From 46631a44217917f8a23b4c019d7f73641e255e67 Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Wed, 24 Jun 2020 14:06:38 +0200 Subject: [PATCH 001/100] updated mapping scholexplorer to OAF --- .../java/eu/dnetlib/dhp/export/DLIToOAF.scala | 16 +++++++++++++++- .../export/SparkExportContentForOpenAire.scala | 11 +++++++---- 2 files changed, 22 insertions(+), 5 deletions(-) diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/DLIToOAF.scala b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/DLIToOAF.scala index 637362acf..967834d69 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/DLIToOAF.scala +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/DLIToOAF.scala @@ -5,7 +5,7 @@ import java.time.format.DateTimeFormatter import eu.dnetlib.dhp.common.PacePerson import eu.dnetlib.dhp.schema.action.AtomicAction -import eu.dnetlib.dhp.schema.oaf.{Author, DataInfo, Dataset, ExternalReference, Field, Instance, KeyValue, Oaf, Publication, Qualifier, Relation, StructuredProperty} +import eu.dnetlib.dhp.schema.oaf.{Author, DataInfo, Dataset, ExternalReference, Field, Instance, KeyValue, Oaf, Publication, Qualifier, Relation, Result, StructuredProperty} import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation} import eu.dnetlib.dhp.utils.DHPUtils import org.apache.commons.lang3.StringUtils @@ -99,6 +99,20 @@ object DLIToOAF { ) + def fixInstance(r:Publication) :Publication = { + val collectedFrom = r.getCollectedfrom.asScala.head + r.getInstance().asScala.foreach(i => i.setCollectedfrom(collectedFrom)) + r + } + + + def fixInstanceDataset(r:Dataset) :Dataset = { + val collectedFrom = r.getCollectedfrom.asScala.head + r.getInstance().asScala.foreach(i => i.setCollectedfrom(collectedFrom)) + r + } + + def toActionSet(item: Oaf): (String, String) = { val mapper = new ObjectMapper() diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/SparkExportContentForOpenAire.scala b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/SparkExportContentForOpenAire.scala index edf951df4..fd8f2d136 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/SparkExportContentForOpenAire.scala +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/SparkExportContentForOpenAire.scala @@ -1,7 +1,7 @@ package eu.dnetlib.dhp.`export` import eu.dnetlib.dhp.application.ArgumentApplicationParser -import eu.dnetlib.dhp.schema.oaf.{Publication, Relation, Dataset => OafDataset} +import eu.dnetlib.dhp.schema.oaf.{Instance, Publication, Relation, Dataset => OafDataset} import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation} import org.apache.commons.io.IOUtils import org.apache.hadoop.io.Text @@ -166,10 +166,13 @@ object SparkExportContentForOpenAire { }).write.mode(SaveMode.Overwrite).save(s"$workingPath/relationAS") - val fRels:Dataset[(String,String)] = spark.read.load(s"$workingPath/relationAS").as[Relation].map(DLIToOAF.toActionSet) - val fpubs:Dataset[(String,String)] = spark.read.load(s"$workingPath/publicationAS").as[Publication].map(DLIToOAF.toActionSet) - val fdats:Dataset[(String,String)] = spark.read.load(s"$workingPath/datasetAS").as[OafDataset].map(DLIToOAF.toActionSet) + spark.read.load(s"$workingPath/publicationAS").as[Publication].map(DLIToOAF.fixInstance).write.mode(SaveMode.Overwrite).save(s"$workingPath/publicationAS_fixed") + spark.read.load(s"$workingPath/datasetAS").as[OafDataset].map(DLIToOAF.fixInstanceDataset).write.mode(SaveMode.Overwrite).save(s"$workingPath/datasetAS_fixed") + + val fRels:Dataset[(String,String)] = spark.read.load(s"$workingPath/relationAS").as[Relation].map(DLIToOAF.toActionSet) + val fpubs:Dataset[(String,String)] = spark.read.load(s"$workingPath/publicationAS_fixed").as[Publication].map(DLIToOAF.toActionSet) + val fdats:Dataset[(String,String)] = spark.read.load(s"$workingPath/datasetAS_fixed").as[OafDataset].map(DLIToOAF.toActionSet) fRels.union(fpubs).union(fdats).rdd.map(s => (new Text(s._1), new Text(s._2))).saveAsHadoopFile(s"$workingPath/rawset", classOf[Text], classOf[Text], classOf[SequenceFileOutputFormat[Text,Text]], classOf[GzipCodec]) } From 8d59fdf34eaa438d0362bd3393b0755570ee7b9d Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 26 Jun 2020 14:32:58 +0200 Subject: [PATCH 002/100] WIP: dataset based PrepareRelationsJob --- .../CreateRelatedEntitiesJob_phase2.java | 17 --- .../dhp/oa/provision/PrepareRelationsJob.java | 128 ++++++++++++++++-- .../dhp/oa/provision/RelationComparator.java | 43 ++++++ .../dhp/oa/provision/RelationList.java | 25 ++++ .../dhp/oa/provision/SortableRelation.java | 80 +++++++++++ .../model/ProvisionModelSupport.java | 7 +- .../provision/model/RelatedEntityWrapper.java | 4 - .../dhp/oa/provision/model/TypedRow.java | 64 --------- .../oa/provision/utils/XmlRecordFactory.java | 4 - 9 files changed, 271 insertions(+), 101 deletions(-) create mode 100644 dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/RelationComparator.java create mode 100644 dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/RelationList.java create mode 100644 dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SortableRelation.java delete mode 100644 dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/TypedRow.java 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 5ef30d6e1..bfcc648a3 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 @@ -19,7 +19,6 @@ import org.apache.spark.sql.expressions.Aggregator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; @@ -28,8 +27,6 @@ import eu.dnetlib.dhp.common.HdfsSupport; import eu.dnetlib.dhp.oa.provision.model.JoinedEntity; import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport; import eu.dnetlib.dhp.oa.provision.model.RelatedEntityWrapper; -import eu.dnetlib.dhp.oa.provision.model.TypedRow; -import eu.dnetlib.dhp.schema.common.EntityType; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.*; import scala.Tuple2; @@ -305,20 +302,6 @@ public class CreateRelatedEntitiesJob_phase2 { private static FilterFunction filterEmptyEntityFn() { return (FilterFunction) v -> Objects.nonNull(v.getEntity()); - /* - * return (FilterFunction) v -> Optional .ofNullable(v.getEntity()) .map(e -> - * StringUtils.isNotBlank(e.getId())) .orElse(false); - */ - } - - 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) { 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 19823120c..bf9806787 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 @@ -3,28 +3,33 @@ package eu.dnetlib.dhp.oa.provision; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import java.util.HashSet; -import java.util.Optional; -import java.util.Set; +import java.util.*; +import java.util.function.Supplier; +import java.util.stream.Collectors; 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; +import org.apache.spark.api.java.function.FilterFunction; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.rdd.RDD; -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.apache.spark.sql.expressions.Aggregator; 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.Iterables; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport; import eu.dnetlib.dhp.oa.provision.model.SortableRelationKey; import eu.dnetlib.dhp.oa.provision.utils.RelationPartitioner; import eu.dnetlib.dhp.schema.oaf.Relation; @@ -102,13 +107,15 @@ public class PrepareRelationsJob { log.info("maxRelations: {}", maxRelations); SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ProvisionModelSupport.getModelClasses()); runWithSparkSession( conf, isSparkSessionManaged, spark -> { removeOutputDir(spark, outputPath); - prepareRelationsRDD( + prepareRelationsDataset( spark, inputRelationsPath, outputPath, relationFilter, maxRelations, relPartitions); }); } @@ -125,9 +132,8 @@ public class PrepareRelationsJob { * @param maxRelations maximum number of allowed outgoing edges * @param relPartitions number of partitions for the output RDD */ - private static void prepareRelationsRDD( - SparkSession spark, String inputRelationsPath, String outputPath, Set relationFilter, int maxRelations, - int relPartitions) { + private static void prepareRelationsRDD(SparkSession spark, String inputRelationsPath, String outputPath, + Set relationFilter, int maxRelations, int relPartitions) { // group by SOURCE and apply limit RDD bySource = readPathRelationRDD(spark, inputRelationsPath) @@ -163,6 +169,108 @@ public class PrepareRelationsJob { .parquet(outputPath); } + private static void prepareRelationsDataset( + SparkSession spark, String inputRelationsPath, String outputPath, Set relationFilter, int maxRelations, + int relPartitions) { + + Dataset bySource = pruneRelations( + spark, inputRelationsPath, relationFilter, maxRelations, relPartitions, + (Function) r -> r.getSource()); + Dataset byTarget = pruneRelations( + spark, inputRelationsPath, relationFilter, maxRelations, relPartitions, + (Function) r -> r.getTarget()); + + bySource + .union(byTarget) + .repartition(relPartitions) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath); + } + + private static Dataset pruneRelations(SparkSession spark, String inputRelationsPath, + Set relationFilter, int maxRelations, int relPartitions, + Function idFn) { + return readRelations(spark, inputRelationsPath, relationFilter, relPartitions) + .groupByKey( + (MapFunction) r -> idFn.call(r), + Encoders.STRING()) + .agg(new RelationAggregator(maxRelations).toColumn()) + .flatMap( + (FlatMapFunction, Relation>) t -> t + ._2() + .getRelations() + .iterator(), + Encoders.bean(Relation.class)); + } + + private static Dataset readRelations(SparkSession spark, String inputRelationsPath, + Set relationFilter, int relPartitions) { + return spark + .read() + .textFile(inputRelationsPath) + .repartition(relPartitions) + .map( + (MapFunction) s -> OBJECT_MAPPER.readValue(s, Relation.class), + Encoders.kryo(Relation.class)) + .filter((FilterFunction) rel -> rel.getDataInfo().getDeletedbyinference() == false) + .filter((FilterFunction) rel -> relationFilter.contains(rel.getRelClass()) == false); + } + + public static class RelationAggregator + extends Aggregator { + + private int maxRelations; + + public RelationAggregator(int maxRelations) { + this.maxRelations = maxRelations; + } + + @Override + public RelationList zero() { + return new RelationList(); + } + + @Override + public RelationList reduce(RelationList b, Relation a) { + b.getRelations().add(a); + return getSortableRelationList(b); + } + + @Override + public RelationList merge(RelationList b1, RelationList b2) { + b1.getRelations().addAll(b2.getRelations()); + return getSortableRelationList(b1); + } + + @Override + public RelationList finish(RelationList r) { + return getSortableRelationList(r); + } + + private RelationList getSortableRelationList(RelationList b1) { + RelationList sr = new RelationList(); + sr + .setRelations( + b1 + .getRelations() + .stream() + .limit(maxRelations) + .collect(Collectors.toCollection(() -> new PriorityQueue<>(new RelationComparator())))); + return sr; + } + + @Override + public Encoder bufferEncoder() { + return Encoders.kryo(RelationList.class); + } + + @Override + public Encoder outputEncoder() { + return Encoders.kryo(RelationList.class); + } + } + /** * Reads a JavaRDD of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline delimited json text * file, diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/RelationComparator.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/RelationComparator.java new file mode 100644 index 000000000..f2209c26c --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/RelationComparator.java @@ -0,0 +1,43 @@ + +package eu.dnetlib.dhp.oa.provision; + +import java.util.Comparator; +import java.util.Map; +import java.util.Optional; + +import com.google.common.collect.ComparisonChain; +import com.google.common.collect.Maps; + +import eu.dnetlib.dhp.schema.oaf.Relation; + +public class RelationComparator implements Comparator { + + private static final Map weights = Maps.newHashMap(); + + static { + weights.put("outcome", 0); + weights.put("supplement", 1); + weights.put("review", 2); + weights.put("citation", 3); + weights.put("affiliation", 4); + weights.put("relationship", 5); + weights.put("publicationDataset", 6); + weights.put("similarity", 7); + + weights.put("provision", 8); + weights.put("participation", 9); + weights.put("dedup", 10); + } + + private Integer getWeight(Relation o) { + return Optional.ofNullable(weights.get(o.getSubRelType())).orElse(Integer.MAX_VALUE); + } + + @Override + public int compare(Relation o1, Relation o2) { + return ComparisonChain + .start() + .compare(getWeight(o1), getWeight(o2)) + .result(); + } +} diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/RelationList.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/RelationList.java new file mode 100644 index 000000000..6e5fd7dba --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/RelationList.java @@ -0,0 +1,25 @@ + +package eu.dnetlib.dhp.oa.provision; + +import java.io.Serializable; +import java.util.PriorityQueue; +import java.util.Queue; + +import eu.dnetlib.dhp.schema.oaf.Relation; + +public class RelationList implements Serializable { + + private Queue relations; + + public RelationList() { + this.relations = new PriorityQueue<>(new RelationComparator()); + } + + public Queue getRelations() { + return relations; + } + + public void setRelations(Queue relations) { + this.relations = relations; + } +} diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SortableRelation.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SortableRelation.java new file mode 100644 index 000000000..8ce92a6a0 --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/SortableRelation.java @@ -0,0 +1,80 @@ + +package eu.dnetlib.dhp.oa.provision; + +import java.io.Serializable; +import java.util.Map; +import java.util.Optional; + +import com.fasterxml.jackson.annotation.JsonIgnore; +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(); + + static { + weights.put("outcome", 0); + weights.put("supplement", 1); + weights.put("review", 2); + weights.put("citation", 3); + weights.put("affiliation", 4); + weights.put("relationship", 5); + weights.put("publicationDataset", 6); + weights.put("similarity", 7); + + weights.put("provision", 8); + weights.put("participation", 9); + weights.put("dedup", 10); + } + + private static final long serialVersionUID = 34753984579L; + + private String groupingKey; + + public static SortableRelation create(Relation r, String groupingKey) { + SortableRelation sr = new SortableRelation(); + sr.setGroupingKey(groupingKey); + sr.setSource(r.getSource()); + sr.setTarget(r.getTarget()); + sr.setRelType(r.getRelType()); + sr.setSubRelType(r.getSubRelType()); + sr.setRelClass(r.getRelClass()); + sr.setDataInfo(r.getDataInfo()); + sr.setCollectedfrom(r.getCollectedfrom()); + sr.setLastupdatetimestamp(r.getLastupdatetimestamp()); + sr.setProperties(r.getProperties()); + sr.setValidated(r.getValidated()); + sr.setValidationDate(r.getValidationDate()); + + return sr; + } + + @JsonIgnore + public Relation asRelation() { + return this; + } + + @Override + public int compareTo(SortableRelation o) { + return ComparisonChain + .start() + .compare(getGroupingKey(), o.getGroupingKey()) + .compare(getWeight(this), getWeight(o)) + .result(); + } + + private Integer getWeight(SortableRelation o) { + return Optional.ofNullable(weights.get(o.getSubRelType())).orElse(Integer.MAX_VALUE); + } + + public String getGroupingKey() { + return groupingKey; + } + + public void setGroupingKey(String groupingKey) { + this.groupingKey = groupingKey; + } +} diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java index 051fe923d..c09ed86e5 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/ProvisionModelSupport.java @@ -5,6 +5,8 @@ import java.util.List; import com.google.common.collect.Lists; +import eu.dnetlib.dhp.oa.provision.RelationList; +import eu.dnetlib.dhp.oa.provision.SortableRelation; import eu.dnetlib.dhp.schema.common.ModelSupport; public class ProvisionModelSupport { @@ -15,11 +17,12 @@ public class ProvisionModelSupport { .addAll( Lists .newArrayList( - TypedRow.class, RelatedEntityWrapper.class, JoinedEntity.class, RelatedEntity.class, - SortableRelationKey.class)); + SortableRelationKey.class, + SortableRelation.class, + RelationList.class)); return modelClasses.toArray(new Class[] {}); } } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/RelatedEntityWrapper.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/RelatedEntityWrapper.java index cbb143ee2..4a4a4a5be 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/RelatedEntityWrapper.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/RelatedEntityWrapper.java @@ -16,10 +16,6 @@ public class RelatedEntityWrapper implements Serializable { } public RelatedEntityWrapper(Relation relation, RelatedEntity target) { - this(null, relation, target); - } - - public RelatedEntityWrapper(TypedRow entity, Relation relation, RelatedEntity target) { this.relation = relation; this.target = target; } 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 deleted file mode 100644 index cbec372e4..000000000 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/TypedRow.java +++ /dev/null @@ -1,64 +0,0 @@ - -package eu.dnetlib.dhp.oa.provision.model; - -import java.io.Serializable; - -import com.google.common.base.Objects; - -public class TypedRow implements Serializable { - - private String id; - - private Boolean deleted; - - private String type; - - private String oaf; - - public String getId() { - return id; - } - - public void setId(String id) { - this.id = id; - } - - public Boolean getDeleted() { - return deleted; - } - - public void setDeleted(Boolean deleted) { - this.deleted = deleted; - } - - public String getType() { - return type; - } - - public void setType(String type) { - this.type = type; - } - - public String getOaf() { - return 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 int hashCode() { - return Objects.hashCode(id); - } -} 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 5d8d9fa20..db9a68d3d 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 @@ -121,10 +121,6 @@ public class XmlRecordFactory implements Serializable { } } - 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)) { From 7817338e0510da3c798f8076625f6962ee207b87 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 26 Jun 2020 17:58:33 +0200 Subject: [PATCH 003/100] added test to verify the relation pre-processing --- .../dhp/oa/provision/PrepareRelationsJob.java | 68 ++++--------- .../dhp/oa/provision/oozie_app/workflow.xml | 1 + .../oa/provision/PrepareRelationsJobTest.java | 93 ++++++++++++++++++ .../eu/dnetlib/dhp/oa/provision/relations.gz | Bin 0 -> 681 bytes .../src/test/resources/log4j.properties | 11 +++ 5 files changed, 122 insertions(+), 51 deletions(-) create mode 100644 dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJobTest.java create mode 100644 dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/relations.gz create mode 100644 dhp-workflows/dhp-graph-provision/src/test/resources/log4j.properties 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 bf9806787..601cf6449 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 @@ -115,7 +115,7 @@ public class PrepareRelationsJob { isSparkSessionManaged, spark -> { removeOutputDir(spark, outputPath); - prepareRelationsDataset( + prepareRelationsRDD( spark, inputRelationsPath, outputPath, relationFilter, maxRelations, relPartitions); }); } @@ -148,21 +148,8 @@ public class PrepareRelationsJob { .map(Tuple2::_2) .rdd(); - // group by TARGET and apply limit - RDD byTarget = readPathRelationRDD(spark, inputRelationsPath) - .filter(rel -> rel.getDataInfo().getDeletedbyinference() == false) - .filter(rel -> relationFilter.contains(rel.getRelClass()) == false) - .mapToPair(r -> new Tuple2<>(SortableRelationKey.create(r, r.getTarget()), r)) - .repartitionAndSortWithinPartitions(new RelationPartitioner(relPartitions)) - .groupBy(Tuple2::_1) - .map(Tuple2::_2) - .map(t -> Iterables.limit(t, maxRelations)) - .flatMap(Iterable::iterator) - .map(Tuple2::_2) - .rdd(); - spark - .createDataset(bySource.union(byTarget), Encoders.bean(Relation.class)) + .createDataset(bySource, Encoders.bean(Relation.class)) .repartition(relPartitions) .write() .mode(SaveMode.Overwrite) @@ -172,41 +159,7 @@ public class PrepareRelationsJob { private static void prepareRelationsDataset( SparkSession spark, String inputRelationsPath, String outputPath, Set relationFilter, int maxRelations, int relPartitions) { - - Dataset bySource = pruneRelations( - spark, inputRelationsPath, relationFilter, maxRelations, relPartitions, - (Function) r -> r.getSource()); - Dataset byTarget = pruneRelations( - spark, inputRelationsPath, relationFilter, maxRelations, relPartitions, - (Function) r -> r.getTarget()); - - bySource - .union(byTarget) - .repartition(relPartitions) - .write() - .mode(SaveMode.Overwrite) - .parquet(outputPath); - } - - private static Dataset pruneRelations(SparkSession spark, String inputRelationsPath, - Set relationFilter, int maxRelations, int relPartitions, - Function idFn) { - return readRelations(spark, inputRelationsPath, relationFilter, relPartitions) - .groupByKey( - (MapFunction) r -> idFn.call(r), - Encoders.STRING()) - .agg(new RelationAggregator(maxRelations).toColumn()) - .flatMap( - (FlatMapFunction, Relation>) t -> t - ._2() - .getRelations() - .iterator(), - Encoders.bean(Relation.class)); - } - - private static Dataset readRelations(SparkSession spark, String inputRelationsPath, - Set relationFilter, int relPartitions) { - return spark + spark .read() .textFile(inputRelationsPath) .repartition(relPartitions) @@ -214,7 +167,20 @@ public class PrepareRelationsJob { (MapFunction) s -> OBJECT_MAPPER.readValue(s, Relation.class), Encoders.kryo(Relation.class)) .filter((FilterFunction) rel -> rel.getDataInfo().getDeletedbyinference() == false) - .filter((FilterFunction) rel -> relationFilter.contains(rel.getRelClass()) == false); + .filter((FilterFunction) rel -> relationFilter.contains(rel.getRelClass()) == false) + .groupByKey( + (MapFunction) Relation::getSource, + Encoders.STRING()) + .agg(new RelationAggregator(maxRelations).toColumn()) + .flatMap( + (FlatMapFunction, Relation>) t -> Iterables + .limit(t._2().getRelations(), maxRelations) + .iterator(), + Encoders.bean(Relation.class)) + .repartition(relPartitions) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath); } public static class RelationAggregator diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml index 0d5121cf1..697a00a09 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml @@ -133,6 +133,7 @@ --inputRelationsPath${inputGraphRootPath}/relation --outputPath${workingDir}/relation + --maxRelations${maxRelations} --relPartitions5000 diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJobTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJobTest.java new file mode 100644 index 000000000..c16bbc6fb --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJobTest.java @@ -0,0 +1,93 @@ + +package eu.dnetlib.dhp.oa.provision; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport; +import eu.dnetlib.dhp.schema.oaf.Relation; +import org.apache.commons.io.FileUtils; +import org.apache.spark.SparkConf; +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.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.junit.jupiter.api.io.TempDir; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + +public class PrepareRelationsJobTest { + + private static final Logger log = LoggerFactory.getLogger(PrepareRelationsJobTest.class); + + public static final String SUBRELTYPE = "subRelType"; + public static final String OUTCOME = "outcome"; + public static final String SUPPLEMENT = "supplement"; + + private static SparkSession spark; + + private static Path workingDir; + + @BeforeAll + public static void setUp() throws IOException { + workingDir = Files.createTempDirectory(PrepareRelationsJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ProvisionModelSupport.getModelClasses()); + + spark = SparkSession + .builder() + .appName(PrepareRelationsJobTest.class.getSimpleName()) + .master("local[*]") + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + public void testRunPrepareRelationsJob(@TempDir Path testPath) throws Exception { + + final int maxRelations = 10; + PrepareRelationsJob + .main( + new String[] { + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-inputRelationsPath", getClass().getResource("relations.gz").getPath(), + "-outputPath", testPath.toString(), + "-relPartitions", "10", + "-relationFilter", "asd", + "-maxRelations", String.valueOf(maxRelations) + }); + + Dataset out = spark.read() + .parquet(testPath.toString()) + .as(Encoders.bean(Relation.class)) + .cache(); + + Assertions.assertEquals(10, out.count()); + + Dataset freq = out.toDF().cube(SUBRELTYPE).count().filter((FilterFunction) value -> !value.isNullAt(0)); + long outcome = freq.filter(freq.col(SUBRELTYPE).equalTo(OUTCOME)).collectAsList().get(0).getAs("count"); + long supplement = freq.filter(freq.col(SUBRELTYPE).equalTo(SUPPLEMENT)).collectAsList().get(0).getAs("count"); + + Assertions.assertTrue(outcome > supplement); + Assertions.assertEquals(7, outcome); + Assertions.assertEquals(3, supplement); + } + +} diff --git a/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/relations.gz b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/relations.gz new file mode 100644 index 0000000000000000000000000000000000000000..13bc01c8c37f15ce901e259b2510c7495cd9316d GIT binary patch literal 681 zcmV;a0#^MWiwFpQ^YvZ;19D|-VRUJ4ZgT+b+cA&RFc`-1olkM{jL=<%FdxQ~3_8E7O|s)qCN@sT zXqQ)2oTKgi7}CLnF~(Qzejj8=HqAkvZA@WSd6x3Rp!;QJ^`yf|V&2&DB$^lLVP9_Kh12Bou}HFgURC!#KYrNko9%A5-!!{z zbNQEELb4LC2N}eDUN*hVr7qXsXD`>wI8LD)r)tXObo_Q3Fiy#gd8^BQba8`WRvKDIfV=!R*u^F)am<`x|EC*~q z#sjt=`(fRFd2GRpFV-_4Y6N6KfCDlizyaA1;DC$>a6ncBI3P1(-GF{NI%|sIm+TRR z-$Pf-$fT%(kzr8 Date: Mon, 29 Jun 2020 08:43:56 +0200 Subject: [PATCH 004/100] all events matchers --- .../dhp/broker/oa/util/EventFinder.java | 41 ++++++++++++------- 1 file changed, 27 insertions(+), 14 deletions(-) diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java index e142b5904..6dfca4fcb 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java @@ -7,7 +7,20 @@ import java.util.List; import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.dhp.broker.model.EventFactory; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; +import eu.dnetlib.dhp.broker.oa.matchers.relatedDatasets.EnrichMissingDatasetIsReferencedBy; +import eu.dnetlib.dhp.broker.oa.matchers.relatedDatasets.EnrichMissingDatasetIsRelatedTo; +import eu.dnetlib.dhp.broker.oa.matchers.relatedDatasets.EnrichMissingDatasetIsSupplementedBy; +import eu.dnetlib.dhp.broker.oa.matchers.relatedDatasets.EnrichMissingDatasetIsSupplementedTo; +import eu.dnetlib.dhp.broker.oa.matchers.relatedDatasets.EnrichMissingDatasetReferences; import eu.dnetlib.dhp.broker.oa.matchers.relatedProjects.EnrichMissingProject; +import eu.dnetlib.dhp.broker.oa.matchers.relatedProjects.EnrichMoreProject; +import eu.dnetlib.dhp.broker.oa.matchers.relatedPublications.EnrichMissingPublicationIsReferencedBy; +import eu.dnetlib.dhp.broker.oa.matchers.relatedPublications.EnrichMissingPublicationIsRelatedTo; +import eu.dnetlib.dhp.broker.oa.matchers.relatedPublications.EnrichMissingPublicationIsSupplementedBy; +import eu.dnetlib.dhp.broker.oa.matchers.relatedPublications.EnrichMissingPublicationIsSupplementedTo; +import eu.dnetlib.dhp.broker.oa.matchers.relatedPublications.EnrichMissingPublicationReferences; +import eu.dnetlib.dhp.broker.oa.matchers.relatedSoftware.EnrichMissingSoftware; +import eu.dnetlib.dhp.broker.oa.matchers.relatedSoftware.EnrichMoreSoftware; import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMissingAbstract; import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMissingAuthorOrcid; import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMissingOpenAccess; @@ -36,20 +49,20 @@ public class EventFinder { // // Advanced matchers matchers.add(new EnrichMissingProject()); - // matchers.add(new EnrichMoreProject()); - // matchers.add(new EnrichMissingSoftware()); - // matchers.add(new EnrichMoreSoftware()); - // matchers.add(new EnrichMissingPublicationIsRelatedTo()); - // matchers.add(new EnrichMissingPublicationIsReferencedBy()); - // matchers.add(new EnrichMissingPublicationReferences()); - // matchers.add(new EnrichMissingPublicationIsSupplementedTo()); - // matchers.add(new EnrichMissingPublicationIsSupplementedBy()); - // matchers.add(new EnrichMissingDatasetIsRelatedTo()); - // matchers.add(new EnrichMissingDatasetIsReferencedBy()); - // matchers.add(new EnrichMissingDatasetReferences()); - // matchers.add(new EnrichMissingDatasetIsSupplementedTo()); - // matchers.add(new EnrichMissingDatasetIsSupplementedBy()); - // matchers.add(new EnrichMissingAbstract()); + matchers.add(new EnrichMoreProject()); + matchers.add(new EnrichMissingSoftware()); + matchers.add(new EnrichMoreSoftware()); + matchers.add(new EnrichMissingPublicationIsRelatedTo()); + matchers.add(new EnrichMissingPublicationIsReferencedBy()); + matchers.add(new EnrichMissingPublicationReferences()); + matchers.add(new EnrichMissingPublicationIsSupplementedTo()); + matchers.add(new EnrichMissingPublicationIsSupplementedBy()); + matchers.add(new EnrichMissingDatasetIsRelatedTo()); + matchers.add(new EnrichMissingDatasetIsReferencedBy()); + matchers.add(new EnrichMissingDatasetReferences()); + matchers.add(new EnrichMissingDatasetIsSupplementedTo()); + matchers.add(new EnrichMissingDatasetIsSupplementedBy()); + matchers.add(new EnrichMissingAbstract()); } public static EventGroup generateEvents(final ResultGroup results, final DedupConfig dedupConfig) { From 6f136734641b167fbbf9325255e1b69bdea8ef7d Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Mon, 29 Jun 2020 16:33:32 +0200 Subject: [PATCH 005/100] accumulators --- dhp-workflows/dhp-broker-events/pom.xml | 10 ++- .../dhp/broker/oa/GenerateEventsJob.java | 21 +++++- .../dnetlib/dhp/broker/oa/IndexOnESJob.java | 71 +++++++++++++++++++ .../dhp/broker/oa/matchers/UpdateMatcher.java | 17 ++++- .../dhp/broker/oa/util/EventFinder.java | 17 +++-- .../oa/generate_all/oozie_app/workflow.xml | 34 ++++++++- .../eu/dnetlib/dhp/broker/oa/index_es.json | 20 ++++++ .../broker/oa/partial/oozie_app/workflow.xml | 29 +++++++- 8 files changed, 207 insertions(+), 12 deletions(-) create mode 100644 dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/IndexOnESJob.java create mode 100644 dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/index_es.json diff --git a/dhp-workflows/dhp-broker-events/pom.xml b/dhp-workflows/dhp-broker-events/pom.xml index 424015a3c..e3182c259 100644 --- a/dhp-workflows/dhp-broker-events/pom.xml +++ b/dhp-workflows/dhp-broker-events/pom.xml @@ -1,5 +1,7 @@ - + dhp-workflows eu.dnetlib.dhp @@ -24,7 +26,11 @@ org.apache.spark spark-sql_2.11 - + + org.elasticsearch + elasticsearch-hadoop + + eu.dnetlib.dhp diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java index dbe2fdd47..30e77be50 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java @@ -3,14 +3,18 @@ package eu.dnetlib.dhp.broker.oa; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; +import java.util.Map; import java.util.Optional; +import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; 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.util.LongAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -18,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.broker.model.Event; +import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; import eu.dnetlib.dhp.broker.oa.util.ClusterUtils; import eu.dnetlib.dhp.broker.oa.util.EventFinder; import eu.dnetlib.dhp.broker.oa.util.EventGroup; @@ -66,12 +71,15 @@ public class GenerateEventsJob { ClusterUtils.removeDir(spark, eventsPath); + final Map accumulators = prepareAccumulators(spark.sparkContext()); + final Dataset groups = ClusterUtils .readPath(spark, workingPath + "/duplicates", ResultGroup.class); final Dataset events = groups .map( - (MapFunction) g -> EventFinder.generateEvents(g, dedupConfig), + (MapFunction) g -> EventFinder + .generateEvents(g, dedupConfig, accumulators), Encoders.bean(EventGroup.class)) .flatMap(group -> group.getData().iterator(), Encoders.bean(Event.class)); @@ -81,6 +89,17 @@ public class GenerateEventsJob { } + public static Map prepareAccumulators(final SparkContext sc) { + + return EventFinder + .getMatchers() + .stream() + .map(UpdateMatcher::accumulatorName) + .distinct() + .collect(Collectors.toMap(s -> s, s -> sc.longAccumulator(s))); + + } + private static DedupConfig loadDedupConfig(final String isLookupUrl, final String profId) throws Exception { final ISLookUpService isLookUpService = ISLookupClientFactory.getLookUpService(isLookupUrl); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/IndexOnESJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/IndexOnESJob.java new file mode 100644 index 000000000..be8d14c5f --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/IndexOnESJob.java @@ -0,0 +1,71 @@ + +package eu.dnetlib.dhp.broker.oa; + +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; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.elasticsearch.spark.rdd.api.java.JavaEsSpark; +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.broker.model.Event; + +public class IndexOnESJob { + + private static final Logger log = LoggerFactory.getLogger(IndexOnESJob.class); + + public static void main(final String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + IndexOnESJob.class + .getResourceAsStream("/eu/dnetlib/dhp/broker/oa/index_es.json"))); + parser.parseArgument(args); + + final SparkConf conf = new SparkConf(); + + final String eventsPath = parser.get("workingPath") + "/events"; + log.info("eventsPath: {}", eventsPath); + + final String index = parser.get("index"); + log.info("index: {}", index); + + final String indexHost = parser.get("esHost"); + log.info("indexHost: {}", indexHost); + + final SparkSession spark = SparkSession.builder().config(conf).getOrCreate(); + + final JavaRDD inputRdd = spark + .read() + .load(eventsPath) + .as(Encoders.bean(Event.class)) + .map(IndexOnESJob::eventAsJsonString, Encoders.STRING()) + .javaRDD(); + + final 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.nodes", indexHost); + esCfg.put("es.mapping.id", "eventId"); // THE PRIMARY KEY + 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); + } + + private static String eventAsJsonString(final Event f) throws JsonProcessingException { + return new ObjectMapper().writeValueAsString(f); + } + +} diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcher.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcher.java index 7f82f9a2b..0618ff7e3 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcher.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcher.java @@ -12,6 +12,7 @@ import java.util.stream.Collectors; import org.apache.commons.codec.digest.DigestUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.spark.util.LongAccumulator; import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.dhp.broker.model.Topic; @@ -36,7 +37,8 @@ public abstract class UpdateMatcher { public Collection> searchUpdatesForRecord(final OaBrokerMainEntity res, final Collection others, - final DedupConfig dedupConfig) { + final DedupConfig dedupConfig, + final Map accumulators) { final Map> infoMap = new HashMap<>(); @@ -67,9 +69,10 @@ public abstract class UpdateMatcher { if (values.isEmpty()) { return new ArrayList<>(); } else if (values.size() > maxNumber) { - System.err.println("Too many events (" + values.size() + ") matched by " + getClass().getSimpleName()); + incrementAccumulator(accumulators, maxNumber); return values.subList(0, maxNumber); } else { + incrementAccumulator(accumulators, values.size()); return values; } } @@ -100,4 +103,14 @@ public abstract class UpdateMatcher { return highlightToStringFunction; } + public String accumulatorName() { + return "event_matcher_" + getClass().getSimpleName().toLowerCase(); + } + + public void incrementAccumulator(final Map accumulators, final long n) { + if (accumulators.containsKey(accumulatorName())) { + accumulators.get(accumulatorName()).add(n); + } + } + } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java index 6dfca4fcb..5ed55247b 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java @@ -3,6 +3,9 @@ package eu.dnetlib.dhp.broker.oa.util; import java.util.ArrayList; import java.util.List; +import java.util.Map; + +import org.apache.spark.util.LongAccumulator; import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.dhp.broker.model.EventFactory; @@ -35,7 +38,7 @@ import eu.dnetlib.pace.config.DedupConfig; public class EventFinder { - private static List> matchers = new ArrayList<>(); + private static final List> matchers = new ArrayList<>(); static { matchers.add(new EnrichMissingAbstract()); matchers.add(new EnrichMissingAuthorOrcid()); @@ -47,7 +50,7 @@ public class EventFinder { matchers.add(new EnrichMorePid()); matchers.add(new EnrichMoreSubject()); - // // Advanced matchers + // Advanced matchers matchers.add(new EnrichMissingProject()); matchers.add(new EnrichMoreProject()); matchers.add(new EnrichMissingSoftware()); @@ -65,12 +68,14 @@ public class EventFinder { matchers.add(new EnrichMissingAbstract()); } - public static EventGroup generateEvents(final ResultGroup results, final DedupConfig dedupConfig) { + public static EventGroup generateEvents(final ResultGroup results, + final DedupConfig dedupConfig, + final Map accumulators) { final List> list = new ArrayList<>(); for (final OaBrokerMainEntity target : results.getData()) { for (final UpdateMatcher matcher : matchers) { - list.addAll(matcher.searchUpdatesForRecord(target, results.getData(), dedupConfig)); + list.addAll(matcher.searchUpdatesForRecord(target, results.getData(), dedupConfig, accumulators)); } } @@ -83,4 +88,8 @@ public class EventFinder { return events; } + public static List> getMatchers() { + return matchers; + } + } diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml index 8752200ff..b8d12c42c 100644 --- a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml @@ -17,7 +17,14 @@ dedupConfProfId the id of a valid Dedup Configuration Profile - + + esIndexName + the elasticsearch index name + + + esIndexHost + the elasticsearch host + sparkDriverMemory memory for driver process @@ -359,6 +366,31 @@ --isLookupUrl${isLookupUrl} --dedupConfProfile${dedupConfProfId} + + + + + + + yarn + cluster + IndexOnESJob + eu.dnetlib.dhp.broker.oa.IndexOnESJob + dhp-broker-events-${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.sql.shuffle.partitions=3840 + + --workingPath${workingPath} + --index${esIndexName} + --esHost${esIndexHost} + diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/index_es.json b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/index_es.json new file mode 100644 index 000000000..ac1dbf786 --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/index_es.json @@ -0,0 +1,20 @@ +[ + { + "paramName": "o", + "paramLongName": "workingPath", + "paramDescription": "the workinh path", + "paramRequired": true + }, + { + "paramName": "idx", + "paramLongName": "index", + "paramDescription": "the ES index", + "paramRequired": true + }, + { + "paramName": "es", + "paramLongName": "esHost", + "paramDescription": "the ES host", + "paramRequired": true + } +] diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml index fd68bfec2..f10c5d804 100644 --- a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml @@ -1,4 +1,4 @@ - + @@ -80,7 +80,7 @@ - + yarn cluster @@ -101,6 +101,31 @@ --isLookupUrl${isLookupUrl} --dedupConfProfile${dedupConfProfId} + + + + + + + yarn + cluster + IndexOnESJob + eu.dnetlib.dhp.broker.oa.IndexOnESJob + dhp-broker-events-${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.sql.shuffle.partitions=3840 + + --workingPath${workingPath} + --index${esIndexName} + --esHost${esIndexHost} + From 59a5421c24976a1d8af397ea77bc72f4ce694a03 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Tue, 30 Jun 2020 16:17:09 +0200 Subject: [PATCH 006/100] indexing, accumulators, limited lists --- .../dhp/broker/oa/GenerateEventsJob.java | 16 +-- .../dnetlib/dhp/broker/oa/IndexOnESJob.java | 7 +- .../dnetlib/dhp/broker/oa/JoinStep1Job.java | 13 +- .../dnetlib/dhp/broker/oa/JoinStep2Job.java | 13 +- .../dnetlib/dhp/broker/oa/JoinStep3Job.java | 13 +- .../dnetlib/dhp/broker/oa/JoinStep4Job.java | 13 +- .../dhp/broker/oa/PrepareGroupsJob.java | 14 +- .../broker/oa/PrepareRelatedDatasetsJob.java | 13 +- .../broker/oa/PrepareRelatedProjectsJob.java | 14 +- .../oa/PrepareRelatedPublicationsJob.java | 13 +- .../broker/oa/PrepareRelatedSoftwaresJob.java | 13 +- .../broker/oa/PrepareSimpleEntititiesJob.java | 13 +- .../dhp/broker/oa/matchers/UpdateMatcher.java | 6 +- .../dhp/broker/oa/util/ClusterUtils.java | 18 +++ .../dhp/broker/oa/util/ConversionUtils.java | 46 ++++--- .../oa/generate_all/oozie_app/workflow.xml | 2 +- .../broker/oa/partial/oozie_app/workflow.xml | 30 +---- .../broker/oa/matchers/UpdateMatcherTest.java | 125 ++++++++++++++++++ .../EnrichMissingPublicationDateTest.java | 57 ++++++++ 19 files changed, 315 insertions(+), 124 deletions(-) create mode 100644 dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcherTest.java create mode 100644 dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingPublicationDateTest.java diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java index 30e77be50..d6ac71429 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java @@ -10,10 +10,8 @@ import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.SparkContext; -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.util.LongAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,17 +71,17 @@ public class GenerateEventsJob { final Map accumulators = prepareAccumulators(spark.sparkContext()); + final LongAccumulator total = spark.sparkContext().longAccumulator("total_events"); + final Dataset groups = ClusterUtils .readPath(spark, workingPath + "/duplicates", ResultGroup.class); - final Dataset events = groups - .map( - (MapFunction) g -> EventFinder - .generateEvents(g, dedupConfig, accumulators), - Encoders.bean(EventGroup.class)) - .flatMap(group -> group.getData().iterator(), Encoders.bean(Event.class)); + final Dataset dataset = groups + .map(g -> EventFinder.generateEvents(g, dedupConfig, accumulators), Encoders.bean(EventGroup.class)) + .flatMap(g -> g.getData().iterator(), Encoders.bean(Event.class)) + .map(e -> ClusterUtils.incrementAccumulator(e, total), Encoders.bean(Event.class)); - events.write().mode(SaveMode.Overwrite).json(eventsPath); + ClusterUtils.save(dataset, eventsPath, Event.class, total); }); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/IndexOnESJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/IndexOnESJob.java index be8d14c5f..36d0ffd1b 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/IndexOnESJob.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/IndexOnESJob.java @@ -18,6 +18,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.broker.model.Event; +import eu.dnetlib.dhp.broker.oa.util.ClusterUtils; public class IndexOnESJob { @@ -45,10 +46,8 @@ public class IndexOnESJob { final SparkSession spark = SparkSession.builder().config(conf).getOrCreate(); - final JavaRDD inputRdd = spark - .read() - .load(eventsPath) - .as(Encoders.bean(Event.class)) + final JavaRDD inputRdd = ClusterUtils + .readPath(spark, eventsPath, Event.class) .map(IndexOnESJob::eventAsJsonString, Encoders.STRING()) .javaRDD(); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep1Job.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep1Job.java index 1be782a12..f9bf2d146 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep1Job.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep1Job.java @@ -10,8 +10,8 @@ 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.TypedColumn; +import org.apache.spark.util.LongAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,6 +52,8 @@ public class JoinStep1Job { ClusterUtils.removeDir(spark, joinedEntitiesPath); + final LongAccumulator total = spark.sparkContext().longAccumulator("total_entities"); + final Dataset sources = ClusterUtils .readPath(spark, workingPath + "/simpleEntities", OaBrokerMainEntity.class); @@ -61,16 +63,15 @@ public class JoinStep1Job { final TypedColumn, OaBrokerMainEntity> aggr = new RelatedProjectAggregator() .toColumn(); - sources + final Dataset dataset = sources .joinWith(typedRels, sources.col("openaireId").equalTo(typedRels.col("source")), "left_outer") .groupByKey( (MapFunction, String>) t -> t._1.getOpenaireId(), Encoders.STRING()) .agg(aggr) - .map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class)) - .write() - .mode(SaveMode.Overwrite) - .json(joinedEntitiesPath); + .map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class)); + + ClusterUtils.save(dataset, joinedEntitiesPath, OaBrokerMainEntity.class, total); }); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep2Job.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep2Job.java index 103d79553..cdcf0add4 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep2Job.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep2Job.java @@ -10,8 +10,8 @@ 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.TypedColumn; +import org.apache.spark.util.LongAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,6 +52,8 @@ public class JoinStep2Job { ClusterUtils.removeDir(spark, joinedEntitiesPath); + final LongAccumulator total = spark.sparkContext().longAccumulator("total_entities"); + final Dataset sources = ClusterUtils .readPath(spark, workingPath + "/joinedEntities_step1", OaBrokerMainEntity.class); @@ -61,16 +63,15 @@ public class JoinStep2Job { final TypedColumn, OaBrokerMainEntity> aggr = new RelatedSoftwareAggregator() .toColumn(); - sources + final Dataset dataset = sources .joinWith(typedRels, sources.col("openaireId").equalTo(typedRels.col("source")), "left_outer") .groupByKey( (MapFunction, String>) t -> t._1.getOpenaireId(), Encoders.STRING()) .agg(aggr) - .map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class)) - .write() - .mode(SaveMode.Overwrite) - .json(joinedEntitiesPath); + .map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class)); + + ClusterUtils.save(dataset, joinedEntitiesPath, OaBrokerMainEntity.class, total); }); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep3Job.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep3Job.java index ceb199dc4..4d06f6f13 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep3Job.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep3Job.java @@ -10,8 +10,8 @@ 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.TypedColumn; +import org.apache.spark.util.LongAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,6 +52,8 @@ public class JoinStep3Job { ClusterUtils.removeDir(spark, joinedEntitiesPath); + final LongAccumulator total = spark.sparkContext().longAccumulator("total_entities"); + final Dataset sources = ClusterUtils .readPath(spark, workingPath + "/joinedEntities_step2", OaBrokerMainEntity.class); @@ -61,16 +63,15 @@ public class JoinStep3Job { final TypedColumn, OaBrokerMainEntity> aggr = new RelatedDatasetAggregator() .toColumn(); - sources + final Dataset dataset = sources .joinWith(typedRels, sources.col("openaireId").equalTo(typedRels.col("source")), "left_outer") .groupByKey( (MapFunction, String>) t -> t._1.getOpenaireId(), Encoders.STRING()) .agg(aggr) - .map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class)) - .write() - .mode(SaveMode.Overwrite) - .json(joinedEntitiesPath); + .map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class)); + + ClusterUtils.save(dataset, joinedEntitiesPath, OaBrokerMainEntity.class, total); }); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep4Job.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep4Job.java index 3067810dd..b53d7e39b 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep4Job.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep4Job.java @@ -10,8 +10,8 @@ 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.TypedColumn; +import org.apache.spark.util.LongAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,6 +52,8 @@ public class JoinStep4Job { ClusterUtils.removeDir(spark, joinedEntitiesPath); + final LongAccumulator total = spark.sparkContext().longAccumulator("total_entities"); + final Dataset sources = ClusterUtils .readPath(spark, workingPath + "/joinedEntities_step3", OaBrokerMainEntity.class); @@ -61,16 +63,15 @@ public class JoinStep4Job { final TypedColumn, OaBrokerMainEntity> aggr = new RelatedPublicationAggregator() .toColumn(); - sources + final Dataset dataset = sources .joinWith(typedRels, sources.col("openaireId").equalTo(typedRels.col("source")), "left_outer") .groupByKey( (MapFunction, String>) t -> t._1.getOpenaireId(), Encoders.STRING()) .agg(aggr) - .map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class)) - .write() - .mode(SaveMode.Overwrite) - .json(joinedEntitiesPath); + .map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class)); + + ClusterUtils.save(dataset, joinedEntitiesPath, OaBrokerMainEntity.class, total); }); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareGroupsJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareGroupsJob.java index 47a9f36c5..eb9add00d 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareGroupsJob.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareGroupsJob.java @@ -10,8 +10,8 @@ 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.TypedColumn; +import org.apache.spark.util.LongAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,6 +57,8 @@ public class PrepareGroupsJob { ClusterUtils.removeDir(spark, groupsPath); + final LongAccumulator total = spark.sparkContext().longAccumulator("total_groups"); + final Dataset results = ClusterUtils .readPath(spark, workingPath + "/joinedEntities_step4", OaBrokerMainEntity.class); @@ -67,20 +69,16 @@ public class PrepareGroupsJob { final TypedColumn, ResultGroup> aggr = new ResultAggregator() .toColumn(); - final Dataset groups = results + final Dataset dataset = results .joinWith(mergedRels, results.col("openaireId").equalTo(mergedRels.col("source")), "inner") .groupByKey( (MapFunction, String>) t -> t._2.getTarget(), Encoders.STRING()) .agg(aggr) - .map( - (MapFunction, ResultGroup>) t -> t._2, Encoders.bean(ResultGroup.class)) + .map(t -> t._2, Encoders.bean(ResultGroup.class)) .filter(rg -> rg.getData().size() > 1); - groups - .write() - .mode(SaveMode.Overwrite) - .json(groupsPath); + ClusterUtils.save(dataset, groupsPath, ResultGroup.class, total); }); } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedDatasetsJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedDatasetsJob.java index 6e006ccf0..0cfc1adcb 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedDatasetsJob.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedDatasetsJob.java @@ -9,7 +9,7 @@ import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.SaveMode; +import org.apache.spark.util.LongAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,6 +54,8 @@ public class PrepareRelatedDatasetsJob { ClusterUtils.removeDir(spark, relsPath); + final LongAccumulator total = spark.sparkContext().longAccumulator("total_rels"); + final Dataset datasets = ClusterUtils .readPath(spark, graphPath + "/dataset", eu.dnetlib.dhp.schema.oaf.Dataset.class) .filter(d -> !ClusterUtils.isDedupRoot(d.getId())) @@ -67,16 +69,15 @@ public class PrepareRelatedDatasetsJob { .filter(r -> !ClusterUtils.isDedupRoot(r.getSource())) .filter(r -> !ClusterUtils.isDedupRoot(r.getTarget())); - rels + final Dataset dataset = rels .joinWith(datasets, datasets.col("openaireId").equalTo(rels.col("target")), "inner") .map(t -> { final RelatedDataset rel = new RelatedDataset(t._1.getSource(), t._2); rel.getRelDataset().setRelType(t._1.getRelClass()); return rel; - }, Encoders.bean(RelatedDataset.class)) - .write() - .mode(SaveMode.Overwrite) - .json(relsPath); + }, Encoders.bean(RelatedDataset.class)); + + ClusterUtils.save(dataset, relsPath, RelatedDataset.class, total); }); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedProjectsJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedProjectsJob.java index 0af5d21b7..e988366c8 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedProjectsJob.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedProjectsJob.java @@ -9,7 +9,7 @@ import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.SaveMode; +import org.apache.spark.util.LongAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,6 +56,8 @@ public class PrepareRelatedProjectsJob { ClusterUtils.removeDir(spark, relsPath); + final LongAccumulator total = spark.sparkContext().longAccumulator("total_rels"); + final Dataset projects = ClusterUtils .readPath(spark, graphPath + "/project", Project.class) .filter(p -> !ClusterUtils.isDedupRoot(p.getId())) @@ -69,12 +71,12 @@ public class PrepareRelatedProjectsJob { .filter(r -> !ClusterUtils.isDedupRoot(r.getSource())) .filter(r -> !ClusterUtils.isDedupRoot(r.getTarget())); - rels + final Dataset dataset = rels .joinWith(projects, projects.col("openaireId").equalTo(rels.col("target")), "inner") - .map(t -> new RelatedProject(t._1.getSource(), t._2), Encoders.bean(RelatedProject.class)) - .write() - .mode(SaveMode.Overwrite) - .json(relsPath); + .map(t -> new RelatedProject(t._1.getSource(), t._2), Encoders.bean(RelatedProject.class)); + + ClusterUtils.save(dataset, relsPath, RelatedProject.class, total); + }); } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedPublicationsJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedPublicationsJob.java index 84752776e..724acc4dc 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedPublicationsJob.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedPublicationsJob.java @@ -9,7 +9,7 @@ import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.SaveMode; +import org.apache.spark.util.LongAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,6 +55,8 @@ public class PrepareRelatedPublicationsJob { ClusterUtils.removeDir(spark, relsPath); + final LongAccumulator total = spark.sparkContext().longAccumulator("total_rels"); + final Dataset pubs = ClusterUtils .readPath(spark, graphPath + "/publication", Publication.class) .filter(p -> !ClusterUtils.isDedupRoot(p.getId())) @@ -70,16 +72,15 @@ public class PrepareRelatedPublicationsJob { .filter(r -> !ClusterUtils.isDedupRoot(r.getSource())) .filter(r -> !ClusterUtils.isDedupRoot(r.getTarget())); - rels + final Dataset dataset = rels .joinWith(pubs, pubs.col("openaireId").equalTo(rels.col("target")), "inner") .map(t -> { final RelatedPublication rel = new RelatedPublication(t._1.getSource(), t._2); rel.getRelPublication().setRelType(t._1.getRelClass()); return rel; - }, Encoders.bean(RelatedPublication.class)) - .write() - .mode(SaveMode.Overwrite) - .json(relsPath); + }, Encoders.bean(RelatedPublication.class)); + + ClusterUtils.save(dataset, relsPath, RelatedPublication.class, total); }); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedSoftwaresJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedSoftwaresJob.java index 0ad753a97..d15565d0d 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedSoftwaresJob.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedSoftwaresJob.java @@ -9,7 +9,7 @@ import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.SaveMode; +import org.apache.spark.util.LongAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,6 +56,8 @@ public class PrepareRelatedSoftwaresJob { ClusterUtils.removeDir(spark, relsPath); + final LongAccumulator total = spark.sparkContext().longAccumulator("total_rels"); + final Dataset softwares = ClusterUtils .readPath(spark, graphPath + "/software", Software.class) .filter(sw -> !ClusterUtils.isDedupRoot(sw.getId())) @@ -69,12 +71,11 @@ public class PrepareRelatedSoftwaresJob { .filter(r -> !ClusterUtils.isDedupRoot(r.getSource())) .filter(r -> !ClusterUtils.isDedupRoot(r.getTarget())); - rels + final Dataset dataset = rels .joinWith(softwares, softwares.col("openaireId").equalTo(rels.col("target")), "inner") - .map(t -> new RelatedSoftware(t._1.getSource(), t._2), Encoders.bean(RelatedSoftware.class)) - .write() - .mode(SaveMode.Overwrite) - .json(relsPath); + .map(t -> new RelatedSoftware(t._1.getSource(), t._2), Encoders.bean(RelatedSoftware.class)); + + ClusterUtils.save(dataset, relsPath, RelatedSoftware.class, total); }); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareSimpleEntititiesJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareSimpleEntititiesJob.java index 1b9c279fd..d3c7113ec 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareSimpleEntititiesJob.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareSimpleEntititiesJob.java @@ -9,8 +9,8 @@ import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; 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.util.LongAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,13 +56,14 @@ public class PrepareSimpleEntititiesJob { ClusterUtils.removeDir(spark, simpleEntitiesPath); - prepareSimpleEntities(spark, graphPath, Publication.class) + final LongAccumulator total = spark.sparkContext().longAccumulator("total_entities"); + + final Dataset dataset = prepareSimpleEntities(spark, graphPath, Publication.class) .union(prepareSimpleEntities(spark, graphPath, eu.dnetlib.dhp.schema.oaf.Dataset.class)) .union(prepareSimpleEntities(spark, graphPath, Software.class)) - .union(prepareSimpleEntities(spark, graphPath, OtherResearchProduct.class)) - .write() - .mode(SaveMode.Overwrite) - .json(simpleEntitiesPath); + .union(prepareSimpleEntities(spark, graphPath, OtherResearchProduct.class)); + + ClusterUtils.save(dataset, simpleEntitiesPath, OaBrokerMainEntity.class, total); }); } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcher.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcher.java index 0618ff7e3..af6ab30a1 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcher.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcher.java @@ -83,8 +83,8 @@ public abstract class UpdateMatcher { return list == null || list.isEmpty() || StringUtils.isBlank(list.get(0)); } - protected boolean isMissing(final String field) { - return StringUtils.isBlank(field); + protected boolean isMissing(final String s) { + return StringUtils.isBlank(s); } public int getMaxNumber() { @@ -108,7 +108,7 @@ public abstract class UpdateMatcher { } public void incrementAccumulator(final Map accumulators, final long n) { - if (accumulators.containsKey(accumulatorName())) { + if (accumulators != null && accumulators.containsKey(accumulatorName())) { accumulators.get(accumulatorName()).add(n); } } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ClusterUtils.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ClusterUtils.java index de9b901d0..2d0106a7a 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ClusterUtils.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ClusterUtils.java @@ -4,7 +4,9 @@ package eu.dnetlib.dhp.broker.oa.util; 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.apache.spark.util.LongAccumulator; import com.fasterxml.jackson.databind.ObjectMapper; @@ -44,4 +46,20 @@ public class ClusterUtils { || s.equals("isSupplementedTo"); } + public static T incrementAccumulator(final T o, final LongAccumulator acc) { + if (acc != null) { + acc.add(1); + } + return o; + } + + public static void save(final Dataset dataset, final String path, final Class clazz, + final LongAccumulator acc) { + dataset + .map(o -> ClusterUtils.incrementAccumulator(o, acc), Encoders.bean(clazz)) + .write() + .mode(SaveMode.Overwrite) + .json(path); + } + } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java index b61d5e7cc..26a9e9471 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java @@ -55,7 +55,7 @@ public class ConversionUtils { res.setLicense(BrokerConstants.OPEN_ACCESS); res.setHostedby(kvValue(i.getHostedby())); return res; - }); + }, 20); } public static OaBrokerTypedValue oafPidToBrokerPid(final StructuredProperty sp) { @@ -75,8 +75,8 @@ public class ConversionUtils { res.setOpenaireId(d.getId()); res.setOriginalId(first(d.getOriginalId())); res.setTitle(structPropValue(d.getTitle())); - res.setPids(mappedList(d.getPid(), ConversionUtils::oafPidToBrokerPid)); - res.setInstances(flatMappedList(d.getInstance(), ConversionUtils::oafInstanceToBrokerInstances)); + res.setPids(mappedList(d.getPid(), ConversionUtils::oafPidToBrokerPid, 20)); + res.setInstances(flatMappedList(d.getInstance(), ConversionUtils::oafInstanceToBrokerInstances, 20)); res.setCollectedFrom(mappedFirst(d.getCollectedfrom(), KeyValue::getValue)); return res; } @@ -90,8 +90,8 @@ public class ConversionUtils { res.setOpenaireId(p.getId()); res.setOriginalId(first(p.getOriginalId())); res.setTitle(structPropValue(p.getTitle())); - res.setPids(mappedList(p.getPid(), ConversionUtils::oafPidToBrokerPid)); - res.setInstances(flatMappedList(p.getInstance(), ConversionUtils::oafInstanceToBrokerInstances)); + res.setPids(mappedList(p.getPid(), ConversionUtils::oafPidToBrokerPid, 20)); + res.setInstances(flatMappedList(p.getInstance(), ConversionUtils::oafInstanceToBrokerInstances, 20)); res.setCollectedFrom(mappedFirst(p.getCollectedfrom(), KeyValue::getValue)); return res; @@ -107,23 +107,25 @@ public class ConversionUtils { res.setOpenaireId(result.getId()); res.setOriginalId(first(result.getOriginalId())); res.setTypology(classId(result.getResulttype())); - res.setTitles(structPropList(result.getTitle())); - res.setAbstracts(fieldList(result.getDescription())); + res.setTitles(structPropList(result.getTitle(), 10)); + res.setAbstracts(fieldList(result.getDescription(), 10)); res.setLanguage(classId(result.getLanguage())); res.setSubjects(structPropTypedList(result.getSubject())); - res.setCreators(mappedList(result.getAuthor(), ConversionUtils::oafAuthorToBrokerAuthor)); + res.setCreators(mappedList(result.getAuthor(), ConversionUtils::oafAuthorToBrokerAuthor, 30)); res.setPublicationdate(fieldValue(result.getDateofacceptance())); res.setPublisher(fieldValue(result.getPublisher())); res.setEmbargoenddate(fieldValue(result.getEmbargoenddate())); - res.setContributor(fieldList(result.getContributor())); + res.setContributor(fieldList(result.getContributor(), 20)); res .setJournal( result instanceof Publication ? oafJournalToBrokerJournal(((Publication) result).getJournal()) : null); res.setCollectedFromId(mappedFirst(result.getCollectedfrom(), KeyValue::getKey)); res.setCollectedFromName(mappedFirst(result.getCollectedfrom(), KeyValue::getValue)); - res.setPids(mappedList(result.getPid(), ConversionUtils::oafPidToBrokerPid)); - res.setInstances(flatMappedList(result.getInstance(), ConversionUtils::oafInstanceToBrokerInstances)); - res.setExternalReferences(mappedList(result.getExternalReference(), ConversionUtils::oafExtRefToBrokerExtRef)); + res.setPids(mappedList(result.getPid(), ConversionUtils::oafPidToBrokerPid, 20)); + res.setInstances(flatMappedList(result.getInstance(), ConversionUtils::oafInstanceToBrokerInstances, 20)); + res + .setExternalReferences( + mappedList(result.getExternalReference(), ConversionUtils::oafExtRefToBrokerExtRef, 20)); return res; } @@ -243,18 +245,25 @@ public class ConversionUtils { : null; } - private static List fieldList(final List> fl) { + private static List fieldList(final List> fl, final long maxSize) { return fl != null - ? fl.stream().map(Field::getValue).filter(StringUtils::isNotBlank).collect(Collectors.toList()) + ? fl + .stream() + .map(Field::getValue) + .map(s -> StringUtils.abbreviate(s, 3000)) // MAX 3000 CHARS + .filter(StringUtils::isNotBlank) + .limit(maxSize) + .collect(Collectors.toList()) : new ArrayList<>(); } - private static List structPropList(final List props) { + private static List structPropList(final List props, final long maxSize) { return props != null ? props .stream() .map(StructuredProperty::getValue) .filter(StringUtils::isNotBlank) + .limit(maxSize) .collect(Collectors.toList()) : new ArrayList<>(); } @@ -271,7 +280,7 @@ public class ConversionUtils { .collect(Collectors.toList()); } - private static List mappedList(final List list, final Function func) { + private static List mappedList(final List list, final Function func, final long maxSize) { if (list == null) { return new ArrayList<>(); } @@ -280,10 +289,12 @@ public class ConversionUtils { .stream() .map(func::apply) .filter(Objects::nonNull) + .limit(maxSize) .collect(Collectors.toList()); } - private static List flatMappedList(final List list, final Function> func) { + private static List flatMappedList(final List list, final Function> func, + final long maxSize) { if (list == null) { return new ArrayList<>(); } @@ -293,6 +304,7 @@ public class ConversionUtils { .map(func::apply) .flatMap(List::stream) .filter(Objects::nonNull) + .limit(maxSize) .collect(Collectors.toList()); } diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml index b8d12c42c..7667bfba7 100644 --- a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml @@ -378,9 +378,9 @@ eu.dnetlib.dhp.broker.oa.IndexOnESJob dhp-broker-events-${projectVersion}.jar - --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.dynamicAllocation.maxExecutors="2" --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml index f10c5d804..9128c9820 100644 --- a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml @@ -78,9 +78,8 @@ Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - + + yarn cluster @@ -101,31 +100,6 @@ --isLookupUrl${isLookupUrl} --dedupConfProfile${dedupConfProfId} - - - - - - - yarn - cluster - IndexOnESJob - eu.dnetlib.dhp.broker.oa.IndexOnESJob - dhp-broker-events-${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.sql.shuffle.partitions=3840 - - --workingPath${workingPath} - --index${esIndexName} - --esHost${esIndexHost} - diff --git a/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcherTest.java b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcherTest.java new file mode 100644 index 000000000..93bc5617f --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcherTest.java @@ -0,0 +1,125 @@ + +package eu.dnetlib.dhp.broker.oa.matchers; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.Arrays; +import java.util.Collection; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import eu.dnetlib.broker.objects.OaBrokerMainEntity; +import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMissingPublicationDate; +import eu.dnetlib.dhp.broker.oa.util.UpdateInfo; + +class UpdateMatcherTest { + + UpdateMatcher matcher = new EnrichMissingPublicationDate(); + + @BeforeEach + void setUp() throws Exception { + } + + @Test + void testSearchUpdatesForRecord_1() { + final OaBrokerMainEntity res = new OaBrokerMainEntity(); + final OaBrokerMainEntity p1 = new OaBrokerMainEntity(); + final OaBrokerMainEntity p2 = new OaBrokerMainEntity(); + final OaBrokerMainEntity p3 = new OaBrokerMainEntity(); + final OaBrokerMainEntity p4 = new OaBrokerMainEntity(); + + final Collection> list = matcher + .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null); + + assertTrue(list.isEmpty()); + } + + @Test + void testSearchUpdatesForRecord_2() { + final OaBrokerMainEntity res = new OaBrokerMainEntity(); + final OaBrokerMainEntity p1 = new OaBrokerMainEntity(); + final OaBrokerMainEntity p2 = new OaBrokerMainEntity(); + final OaBrokerMainEntity p3 = new OaBrokerMainEntity(); + final OaBrokerMainEntity p4 = new OaBrokerMainEntity(); + + res.setPublicationdate("2018"); + + final Collection> list = matcher + .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null); + + assertTrue(list.isEmpty()); + } + + @Test + void testSearchUpdatesForRecord_3() { + final OaBrokerMainEntity res = new OaBrokerMainEntity(); + final OaBrokerMainEntity p1 = new OaBrokerMainEntity(); + final OaBrokerMainEntity p2 = new OaBrokerMainEntity(); + final OaBrokerMainEntity p3 = new OaBrokerMainEntity(); + final OaBrokerMainEntity p4 = new OaBrokerMainEntity(); + + p2.setPublicationdate("2018"); + + final Collection> list = matcher + .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null); + + assertTrue(list.size() == 1); + } + + @Test + void testSearchUpdatesForRecord_4() { + final OaBrokerMainEntity res = new OaBrokerMainEntity(); + final OaBrokerMainEntity p1 = new OaBrokerMainEntity(); + final OaBrokerMainEntity p2 = new OaBrokerMainEntity(); + final OaBrokerMainEntity p3 = new OaBrokerMainEntity(); + final OaBrokerMainEntity p4 = new OaBrokerMainEntity(); + + res.setPublicationdate("2018"); + p2.setPublicationdate("2018"); + + final Collection> list = matcher + .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null); + + assertTrue(list.isEmpty()); + } + + @Test + void testSearchUpdatesForRecord_5() { + final OaBrokerMainEntity res = new OaBrokerMainEntity(); + final OaBrokerMainEntity p1 = new OaBrokerMainEntity(); + final OaBrokerMainEntity p2 = new OaBrokerMainEntity(); + final OaBrokerMainEntity p3 = new OaBrokerMainEntity(); + final OaBrokerMainEntity p4 = new OaBrokerMainEntity(); + res.setPublicationdate("2018"); + p1.setPublicationdate("2018"); + p2.setPublicationdate("2018"); + p3.setPublicationdate("2018"); + p4.setPublicationdate("2018"); + + final Collection> list = matcher + .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null); + + assertTrue(list.isEmpty()); + } + + @Test + void testSearchUpdatesForRecord_6() { + final OaBrokerMainEntity res = new OaBrokerMainEntity(); + final OaBrokerMainEntity p1 = new OaBrokerMainEntity(); + final OaBrokerMainEntity p2 = new OaBrokerMainEntity(); + final OaBrokerMainEntity p3 = new OaBrokerMainEntity(); + final OaBrokerMainEntity p4 = new OaBrokerMainEntity(); + + p1.setPublicationdate("2018"); + p2.setPublicationdate("2018"); + p3.setPublicationdate("2018"); + p4.setPublicationdate("2018"); + + final Collection> list = matcher + .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null); + + assertTrue(list.size() == 1); + } + +} diff --git a/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingPublicationDateTest.java b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingPublicationDateTest.java new file mode 100644 index 000000000..77a19af4c --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingPublicationDateTest.java @@ -0,0 +1,57 @@ + +package eu.dnetlib.dhp.broker.oa.matchers.simple; + +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.List; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import eu.dnetlib.broker.objects.OaBrokerMainEntity; + +class EnrichMissingPublicationDateTest { + + final EnrichMissingPublicationDate matcher = new EnrichMissingPublicationDate(); + + @BeforeEach + void setUp() throws Exception { + } + + @Test + void testFindDifferences_1() { + final OaBrokerMainEntity source = new OaBrokerMainEntity(); + final OaBrokerMainEntity target = new OaBrokerMainEntity(); + final List list = matcher.findDifferences(source, target); + assertTrue(list.isEmpty()); + } + + @Test + void testFindDifferences_2() { + final OaBrokerMainEntity source = new OaBrokerMainEntity(); + final OaBrokerMainEntity target = new OaBrokerMainEntity(); + source.setPublicationdate("2018"); + final List list = matcher.findDifferences(source, target); + assertTrue(list.size() == 1); + } + + @Test + void testFindDifferences_3() { + final OaBrokerMainEntity source = new OaBrokerMainEntity(); + final OaBrokerMainEntity target = new OaBrokerMainEntity(); + target.setPublicationdate("2018"); + final List list = matcher.findDifferences(source, target); + assertTrue(list.isEmpty()); + } + + @Test + void testFindDifferences_4() { + final OaBrokerMainEntity source = new OaBrokerMainEntity(); + final OaBrokerMainEntity target = new OaBrokerMainEntity(); + source.setPublicationdate("2018"); + target.setPublicationdate("2018"); + final List list = matcher.findDifferences(source, target); + assertTrue(list.isEmpty()); + } + +} From 3bcdfbabe9dbce86785e12c1052154c015f9022e Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Wed, 1 Jul 2020 08:42:39 +0200 Subject: [PATCH 007/100] list with limits --- .../AbstractEnrichMissingDataset.java | 6 +++ .../relatedProjects/EnrichMoreProject.java | 6 +++ .../AbstractEnrichMissingPublication.java | 6 +++ .../relatedSoftware/EnrichMoreSoftware.java | 6 +++ .../simple/EnrichMissingAuthorOrcid.java | 6 +++ .../simple/EnrichMissingOpenAccess.java | 6 +++ .../oa/matchers/simple/EnrichMissingPid.java | 3 +- .../matchers/simple/EnrichMissingSubject.java | 7 +++ .../matchers/simple/EnrichMoreOpenAccess.java | 6 +++ .../oa/matchers/simple/EnrichMorePid.java | 7 +++ .../oa/matchers/simple/EnrichMoreSubject.java | 6 +++ .../dhp/broker/oa/util/BrokerConstants.java | 4 ++ .../dhp/broker/oa/util/ConversionUtils.java | 44 +++++++++---------- 13 files changed, 88 insertions(+), 25 deletions(-) diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/AbstractEnrichMissingDataset.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/AbstractEnrichMissingDataset.java index f21c1c7b3..2f73a2448 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/AbstractEnrichMissingDataset.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedDatasets/AbstractEnrichMissingDataset.java @@ -1,6 +1,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.relatedDatasets; +import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -9,6 +10,7 @@ import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.broker.objects.OaBrokerRelatedDataset; import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; +import eu.dnetlib.dhp.broker.oa.util.BrokerConstants; public abstract class AbstractEnrichMissingDataset extends UpdateMatcher { @@ -25,6 +27,10 @@ public abstract class AbstractEnrichMissingDataset extends UpdateMatcher findDifferences(final OaBrokerMainEntity source, final OaBrokerMainEntity target) { + if (target.getDatasets().size() >= BrokerConstants.MAX_LIST_SIZE) { + return new ArrayList<>(); + } + final Set existingDatasets = target .getDatasets() .stream() diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMoreProject.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMoreProject.java index 85b2cbe28..6a10f19be 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMoreProject.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMoreProject.java @@ -1,6 +1,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.relatedProjects; +import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -9,6 +10,7 @@ import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.broker.objects.OaBrokerProject; import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; +import eu.dnetlib.dhp.broker.oa.util.BrokerConstants; public class EnrichMoreProject extends UpdateMatcher { @@ -27,6 +29,10 @@ public class EnrichMoreProject extends UpdateMatcher { protected List findDifferences(final OaBrokerMainEntity source, final OaBrokerMainEntity target) { + if (target.getProjects().size() >= BrokerConstants.MAX_LIST_SIZE) { + return new ArrayList<>(); + } + final Set existingProjects = target .getProjects() .stream() diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/AbstractEnrichMissingPublication.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/AbstractEnrichMissingPublication.java index f951131b1..7ba3e5e02 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/AbstractEnrichMissingPublication.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedPublications/AbstractEnrichMissingPublication.java @@ -1,6 +1,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.relatedPublications; +import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -9,6 +10,7 @@ import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.broker.objects.OaBrokerRelatedPublication; import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; +import eu.dnetlib.dhp.broker.oa.util.BrokerConstants; public abstract class AbstractEnrichMissingPublication extends UpdateMatcher { @@ -27,6 +29,10 @@ public abstract class AbstractEnrichMissingPublication extends UpdateMatcher= BrokerConstants.MAX_LIST_SIZE) { + return new ArrayList<>(); + } + final Set existingPublications = target .getPublications() .stream() diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedSoftware/EnrichMoreSoftware.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedSoftware/EnrichMoreSoftware.java index 2bc370187..a6cd34359 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedSoftware/EnrichMoreSoftware.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedSoftware/EnrichMoreSoftware.java @@ -1,6 +1,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.relatedSoftware; +import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -9,6 +10,7 @@ import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.broker.objects.OaBrokerRelatedSoftware; import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; +import eu.dnetlib.dhp.broker.oa.util.BrokerConstants; public class EnrichMoreSoftware extends UpdateMatcher { @@ -24,6 +26,10 @@ public class EnrichMoreSoftware extends UpdateMatcher { final OaBrokerMainEntity source, final OaBrokerMainEntity target) { + if (target.getSoftwares().size() >= BrokerConstants.MAX_LIST_SIZE) { + return new ArrayList<>(); + } + final Set existingSoftwares = source .getSoftwares() .stream() diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingAuthorOrcid.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingAuthorOrcid.java index 7bbc43fe3..e834d1dde 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingAuthorOrcid.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingAuthorOrcid.java @@ -1,6 +1,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.simple; +import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -11,6 +12,7 @@ import eu.dnetlib.broker.objects.OaBrokerAuthor; import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; +import eu.dnetlib.dhp.broker.oa.util.BrokerConstants; public class EnrichMissingAuthorOrcid extends UpdateMatcher { @@ -25,6 +27,10 @@ public class EnrichMissingAuthorOrcid extends UpdateMatcher { protected List findDifferences(final OaBrokerMainEntity source, final OaBrokerMainEntity target) { + if (target.getCreators().size() >= BrokerConstants.MAX_LIST_SIZE) { + return new ArrayList<>(); + } + final Set existingOrcids = target .getCreators() .stream() diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingOpenAccess.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingOpenAccess.java index 41a00dcd1..8e4f2fcf4 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingOpenAccess.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingOpenAccess.java @@ -1,6 +1,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.simple; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; @@ -23,6 +24,11 @@ public class EnrichMissingOpenAccess extends UpdateMatcher { @Override protected List findDifferences(final OaBrokerMainEntity source, final OaBrokerMainEntity target) { + + if (target.getInstances().size() >= BrokerConstants.MAX_LIST_SIZE) { + return new ArrayList<>(); + } + final long count = target .getInstances() .stream() diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingPid.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingPid.java index 4863bdeb7..4e4003890 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingPid.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingPid.java @@ -22,9 +22,8 @@ public class EnrichMissingPid extends UpdateMatcher { @Override protected List findDifferences(final OaBrokerMainEntity source, final OaBrokerMainEntity target) { - final long count = target.getPids().size(); - if (count > 0) { + if (target.getPids().size() > 0) { return Arrays.asList(); } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingSubject.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingSubject.java index f762e3f52..26ebbb7c0 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingSubject.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMissingSubject.java @@ -1,6 +1,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.simple; +import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -9,6 +10,7 @@ import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.broker.objects.OaBrokerTypedValue; import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; +import eu.dnetlib.dhp.broker.oa.util.BrokerConstants; public class EnrichMissingSubject extends UpdateMatcher { @@ -22,6 +24,11 @@ public class EnrichMissingSubject extends UpdateMatcher { @Override protected List findDifferences(final OaBrokerMainEntity source, final OaBrokerMainEntity target) { + + if (target.getSubjects().size() >= BrokerConstants.MAX_LIST_SIZE) { + return new ArrayList<>(); + } + final Set existingSubject = target .getSubjects() .stream() diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreOpenAccess.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreOpenAccess.java index 9ce362a97..46f6fa80c 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreOpenAccess.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreOpenAccess.java @@ -1,6 +1,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.simple; +import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -23,6 +24,11 @@ public class EnrichMoreOpenAccess extends UpdateMatcher { @Override protected List findDifferences(final OaBrokerMainEntity source, final OaBrokerMainEntity target) { + + if (target.getInstances().size() >= BrokerConstants.MAX_LIST_SIZE) { + return new ArrayList<>(); + } + final Set urls = target .getInstances() .stream() diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMorePid.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMorePid.java index 583960037..609437b9d 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMorePid.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMorePid.java @@ -1,6 +1,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.simple; +import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -9,6 +10,7 @@ import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.broker.objects.OaBrokerTypedValue; import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; +import eu.dnetlib.dhp.broker.oa.util.BrokerConstants; public class EnrichMorePid extends UpdateMatcher { @@ -22,6 +24,11 @@ public class EnrichMorePid extends UpdateMatcher { @Override protected List findDifferences(final OaBrokerMainEntity source, final OaBrokerMainEntity target) { + + if (target.getPids().size() >= BrokerConstants.MAX_LIST_SIZE) { + return new ArrayList<>(); + } + final Set existingPids = target .getPids() .stream() diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubject.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubject.java index 150029462..bbe6609d7 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubject.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/simple/EnrichMoreSubject.java @@ -1,6 +1,7 @@ package eu.dnetlib.dhp.broker.oa.matchers.simple; +import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -9,6 +10,7 @@ import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.broker.objects.OaBrokerTypedValue; import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; +import eu.dnetlib.dhp.broker.oa.util.BrokerConstants; public class EnrichMoreSubject extends UpdateMatcher { @@ -23,6 +25,10 @@ public class EnrichMoreSubject extends UpdateMatcher { protected List findDifferences(final OaBrokerMainEntity source, final OaBrokerMainEntity target) { + if (target.getSubjects().size() >= BrokerConstants.MAX_LIST_SIZE) { + return new ArrayList<>(); + } + final Set existingSubjects = target .getSubjects() .stream() diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/BrokerConstants.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/BrokerConstants.java index 58e41acbb..5308b9dff 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/BrokerConstants.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/BrokerConstants.java @@ -19,6 +19,10 @@ public class BrokerConstants { public static final int MAX_NUMBER_OF_RELS = 20; + public static final int MAX_STRING_SIZE = 3000; + + public static final int MAX_LIST_SIZE = 50; + public static Class[] getModelClasses() { final Set> list = new HashSet<>(); list.addAll(Arrays.asList(ModelSupport.getOafModelClasses())); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java index 26a9e9471..1ce84283a 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java @@ -55,7 +55,7 @@ public class ConversionUtils { res.setLicense(BrokerConstants.OPEN_ACCESS); res.setHostedby(kvValue(i.getHostedby())); return res; - }, 20); + }); } public static OaBrokerTypedValue oafPidToBrokerPid(final StructuredProperty sp) { @@ -75,8 +75,8 @@ public class ConversionUtils { res.setOpenaireId(d.getId()); res.setOriginalId(first(d.getOriginalId())); res.setTitle(structPropValue(d.getTitle())); - res.setPids(mappedList(d.getPid(), ConversionUtils::oafPidToBrokerPid, 20)); - res.setInstances(flatMappedList(d.getInstance(), ConversionUtils::oafInstanceToBrokerInstances, 20)); + res.setPids(mappedList(d.getPid(), ConversionUtils::oafPidToBrokerPid)); + res.setInstances(flatMappedList(d.getInstance(), ConversionUtils::oafInstanceToBrokerInstances)); res.setCollectedFrom(mappedFirst(d.getCollectedfrom(), KeyValue::getValue)); return res; } @@ -90,8 +90,8 @@ public class ConversionUtils { res.setOpenaireId(p.getId()); res.setOriginalId(first(p.getOriginalId())); res.setTitle(structPropValue(p.getTitle())); - res.setPids(mappedList(p.getPid(), ConversionUtils::oafPidToBrokerPid, 20)); - res.setInstances(flatMappedList(p.getInstance(), ConversionUtils::oafInstanceToBrokerInstances, 20)); + res.setPids(mappedList(p.getPid(), ConversionUtils::oafPidToBrokerPid)); + res.setInstances(flatMappedList(p.getInstance(), ConversionUtils::oafInstanceToBrokerInstances)); res.setCollectedFrom(mappedFirst(p.getCollectedfrom(), KeyValue::getValue)); return res; @@ -107,25 +107,24 @@ public class ConversionUtils { res.setOpenaireId(result.getId()); res.setOriginalId(first(result.getOriginalId())); res.setTypology(classId(result.getResulttype())); - res.setTitles(structPropList(result.getTitle(), 10)); - res.setAbstracts(fieldList(result.getDescription(), 10)); + res.setTitles(structPropList(result.getTitle())); + res.setAbstracts(fieldList(result.getDescription())); res.setLanguage(classId(result.getLanguage())); res.setSubjects(structPropTypedList(result.getSubject())); - res.setCreators(mappedList(result.getAuthor(), ConversionUtils::oafAuthorToBrokerAuthor, 30)); + res.setCreators(mappedList(result.getAuthor(), ConversionUtils::oafAuthorToBrokerAuthor)); res.setPublicationdate(fieldValue(result.getDateofacceptance())); res.setPublisher(fieldValue(result.getPublisher())); res.setEmbargoenddate(fieldValue(result.getEmbargoenddate())); - res.setContributor(fieldList(result.getContributor(), 20)); + res.setContributor(fieldList(result.getContributor())); res .setJournal( result instanceof Publication ? oafJournalToBrokerJournal(((Publication) result).getJournal()) : null); res.setCollectedFromId(mappedFirst(result.getCollectedfrom(), KeyValue::getKey)); res.setCollectedFromName(mappedFirst(result.getCollectedfrom(), KeyValue::getValue)); - res.setPids(mappedList(result.getPid(), ConversionUtils::oafPidToBrokerPid, 20)); - res.setInstances(flatMappedList(result.getInstance(), ConversionUtils::oafInstanceToBrokerInstances, 20)); + res.setPids(mappedList(result.getPid(), ConversionUtils::oafPidToBrokerPid)); + res.setInstances(flatMappedList(result.getInstance(), ConversionUtils::oafInstanceToBrokerInstances)); res - .setExternalReferences( - mappedList(result.getExternalReference(), ConversionUtils::oafExtRefToBrokerExtRef, 20)); + .setExternalReferences(mappedList(result.getExternalReference(), ConversionUtils::oafExtRefToBrokerExtRef)); return res; } @@ -245,25 +244,25 @@ public class ConversionUtils { : null; } - private static List fieldList(final List> fl, final long maxSize) { + private static List fieldList(final List> fl) { return fl != null ? fl .stream() .map(Field::getValue) - .map(s -> StringUtils.abbreviate(s, 3000)) // MAX 3000 CHARS + .map(s -> StringUtils.abbreviate(s, BrokerConstants.MAX_STRING_SIZE)) .filter(StringUtils::isNotBlank) - .limit(maxSize) + .limit(BrokerConstants.MAX_LIST_SIZE) .collect(Collectors.toList()) : new ArrayList<>(); } - private static List structPropList(final List props, final long maxSize) { + private static List structPropList(final List props) { return props != null ? props .stream() .map(StructuredProperty::getValue) .filter(StringUtils::isNotBlank) - .limit(maxSize) + .limit(BrokerConstants.MAX_LIST_SIZE) .collect(Collectors.toList()) : new ArrayList<>(); } @@ -280,7 +279,7 @@ public class ConversionUtils { .collect(Collectors.toList()); } - private static List mappedList(final List list, final Function func, final long maxSize) { + private static List mappedList(final List list, final Function func) { if (list == null) { return new ArrayList<>(); } @@ -289,12 +288,11 @@ public class ConversionUtils { .stream() .map(func::apply) .filter(Objects::nonNull) - .limit(maxSize) + .limit(BrokerConstants.MAX_LIST_SIZE) .collect(Collectors.toList()); } - private static List flatMappedList(final List list, final Function> func, - final long maxSize) { + private static List flatMappedList(final List list, final Function> func) { if (list == null) { return new ArrayList<>(); } @@ -304,7 +302,7 @@ public class ConversionUtils { .map(func::apply) .flatMap(List::stream) .filter(Objects::nonNull) - .limit(maxSize) + .limit(BrokerConstants.MAX_LIST_SIZE) .collect(Collectors.toList()); } From e4a29a4513a5021e635991a91b2c24d2faf9f505 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 2 Jul 2020 12:36:33 +0200 Subject: [PATCH 008/100] fixed workflow for the import of the claims alone --- .../{ => oozie_app}/config-default.xml | 0 .../graph/raw_claims/oozie_app/workflow.xml | 161 +++++++++++++++++ .../dhp/oa/graph/raw_claims/workflow.xml | 169 ------------------ 3 files changed, 161 insertions(+), 169 deletions(-) rename dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/{ => oozie_app}/config-default.xml (100%) create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/oozie_app/workflow.xml delete mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/workflow.xml diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/oozie_app/config-default.xml similarity index 100% rename from dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/config-default.xml rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/oozie_app/config-default.xml diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/oozie_app/workflow.xml new file mode 100644 index 000000000..13ec192ef --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/oozie_app/workflow.xml @@ -0,0 +1,161 @@ + + + + graphOutputPath + the target path to store raw graph + + + reuseContent + false + should import content from the aggregator or reuse a previous version + + + contentPath + path location to store (or reuse) content from the aggregator + + + postgresURL + the postgres URL to access to the database + + + postgresUser + the user postgres + + + postgresPassword + the password postgres + + + dbSchema + beta + the database schema according to the D-Net infrastructure (beta or production) + + + mongoURL + mongoDB url, example: mongodb://[username:password@]host[:port] + + + mongoDb + mongo database + + + isLookupUrl + the address of the lookUp service + + + + sparkDriverMemory + memory for driver process + + + sparkExecutorMemory + memory for individual executor + + + sparkExecutorCores + number of cores used by single executor + + + oozieActionShareLibForSpark2 + oozie action sharelib for spark 2.* + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + spark 2.* extra listeners classname + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + spark 2.* sql query execution listeners classname + + + spark2YarnHistoryServerAddress + spark 2.* yarn history server address + + + spark2EventLogDir + spark 2.* event log dir location + + + + + ${jobTracker} + ${nameNode} + + + mapreduce.job.queuename + ${queueName} + + + oozie.launcher.mapred.job.queue.name + ${oozieLauncherQueueName} + + + oozie.action.sharelib.for.spark + ${oozieActionShareLibForSpark2} + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + eu.dnetlib.dhp.oa.graph.raw.MigrateDbEntitiesApplication + --hdfsPath${contentPath}/db_claims + --postgresUrl${postgresURL} + --postgresUser${postgresUser} + --postgresPassword${postgresPassword} + --isLookupUrl${isLookupUrl} + --actionclaims + --dbschema${dbSchema} + + + + + + + + + + + eu.dnetlib.dhp.oa.graph.raw.MigrateMongoMdstoresApplication + -p${contentPath}/odf_claims + -mongourl${mongoURL} + -mongodb${mongoDb} + -fODF + -lstore + -iclaim + + + + + + + + + + + eu.dnetlib.dhp.oa.graph.raw.MigrateMongoMdstoresApplication + -p${contentPath}/oaf_claims + -mongourl${mongoURL} + -mongodb${mongoDb} + -fOAF + -lstore + -iclaim + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/workflow.xml deleted file mode 100644 index 1ac456976..000000000 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/workflow.xml +++ /dev/null @@ -1,169 +0,0 @@ - - - - migrationClaimsPathStep1 - the base path to store hdfs file - - - migrationClaimsPathStep2 - the temporary path to store entities before dispatching - - - migrationClaimsPathStep3 - the graph Raw base path - - - postgresURL - the postgres URL to access to the database - - - postgresUser - the user postgres - - - postgresPassword - the password postgres - - - mongoURL - mongoDB url, example: mongodb://[username:password@]host[:port] - - - mongoDb - mongo database - - - 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} - eu.dnetlib.dhp.migration.step1.MigrateDbEntitiesApplication - -p${migrationClaimsPathStep1}/db_claims - -pgurl${postgresURL} - -pguser${postgresUser} - -pgpasswd${postgresPassword} - -aclaims - - - - - - - - ${jobTracker} - ${nameNode} - eu.dnetlib.dhp.migration.step1.MigrateMongoMdstoresApplication - -p${migrationClaimsPathStep1}/odf_claims - -mongourl${mongoURL} - -mongodb${mongoDb} - -fODF - -lstore - -iclaim - - - - - - - - ${jobTracker} - ${nameNode} - eu.dnetlib.dhp.migration.step1.MigrateMongoMdstoresApplication - -p${migrationClaimsPathStep1}/oaf_claims - -mongourl${mongoURL} - -mongodb${mongoDb} - -fOAF - -lstore - -iclaim - - - - - - - - - - - - - - - - - ${jobTracker} - ${nameNode} - yarn-cluster - cluster - GenerateClaimEntities - eu.dnetlib.dhp.migration.step2.GenerateEntitiesApplication - dhp-aggregation-${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 - -s${migrationClaimsPathStep1}/db_claims,${migrationClaimsPathStep1}/oaf_claims,${migrationClaimsPathStep1}/odf_claims - -t${migrationClaimsPathStep2}/claim_entities - -pgurl${postgresURL} - -pguser${postgresUser} - -pgpasswd${postgresPassword} - - - - - - - - - - - - - - - - - ${jobTracker} - ${nameNode} - yarn-cluster - cluster - GenerateClaimGraph - eu.dnetlib.dhp.migration.step3.DispatchEntitiesApplication - dhp-aggregation-${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 - -s${migrationClaimsPathStep2}/claim_entities - -g${migrationClaimsPathStep3} - - - - - - - \ No newline at end of file From 1d420eedb4135c72ecef00383bff7df5fe5c9aec Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Thu, 2 Jul 2020 12:37:43 +0200 Subject: [PATCH 009/100] added generation of EBI Dataset --- .../eu/dnetlib/dhp/sx/ebi/EBIAggregator.scala | 89 +++++++++++++++++++ .../dhp/sx/ebi/SparkCreateEBIDataFrame.scala | 83 +++++++++++++++++ .../parser/AbstractScholexplorerParser.java | 13 +++ .../parser/DatasetScholexplorerParser.java | 46 +++++++++- .../PublicationScholexplorerParser.java | 1 + .../dnetlib/dhp/sx/ebi/ebi_to_df_params.json | 4 + .../dhp/sx/ebi/oozie_app/config-default.xml | 38 ++++++++ .../dnetlib/dhp/sx/ebi/oozie_app/workflow.xml | 51 +++++++++++ .../eu/dnetlib/dhp/sx/ebi/TestEBITODS.scala | 22 +++++ 9 files changed, 346 insertions(+), 1 deletion(-) create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/EBIAggregator.scala create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/SparkCreateEBIDataFrame.scala create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/ebi_to_df_params.json create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/oozie_app/workflow.xml create mode 100644 dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/ebi/TestEBITODS.scala diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/EBIAggregator.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/EBIAggregator.scala new file mode 100644 index 000000000..41fcd2636 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/EBIAggregator.scala @@ -0,0 +1,89 @@ +package eu.dnetlib.dhp.sx.ebi +import eu.dnetlib.dhp.schema.oaf.{Publication, Relation, Dataset => OafDataset} +import org.apache.spark.sql.{Encoder, Encoders} +import org.apache.spark.sql.expressions.Aggregator + + + +object EBIAggregator { + + def getDatasetAggregator(): Aggregator[(String, OafDataset), OafDataset, OafDataset] = new Aggregator[(String, OafDataset), OafDataset, OafDataset]{ + + override def zero: OafDataset = new OafDataset() + + override def reduce(b: OafDataset, a: (String, OafDataset)): OafDataset = { + b.mergeFrom(a._2) + if (b.getId == null) + b.setId(a._2.getId) + b + } + + + override def merge(wx: OafDataset, wy: OafDataset): OafDataset = { + wx.mergeFrom(wy) + if(wx.getId == null && wy.getId.nonEmpty) + wx.setId(wy.getId) + wx + } + override def finish(reduction: OafDataset): OafDataset = reduction + + override def bufferEncoder: Encoder[OafDataset] = + Encoders.kryo(classOf[OafDataset]) + + override def outputEncoder: Encoder[OafDataset] = + Encoders.kryo(classOf[OafDataset]) + } + + + def getPublicationAggregator(): Aggregator[(String, Publication), Publication, Publication] = new Aggregator[(String, Publication), Publication, Publication]{ + + override def zero: Publication = new Publication() + + override def reduce(b: Publication, a: (String, Publication)): Publication = { + b.mergeFrom(a._2) + if (b.getId == null) + b.setId(a._2.getId) + b + } + + + override def merge(wx: Publication, wy: Publication): Publication = { + wx.mergeFrom(wy) + if(wx.getId == null && wy.getId.nonEmpty) + wx.setId(wy.getId) + wx + } + override def finish(reduction: Publication): Publication = reduction + + override def bufferEncoder: Encoder[Publication] = + Encoders.kryo(classOf[Publication]) + + override def outputEncoder: Encoder[Publication] = + Encoders.kryo(classOf[Publication]) + } + + + def getRelationAggregator(): Aggregator[(String, Relation), Relation, Relation] = new Aggregator[(String, Relation), Relation, Relation]{ + + override def zero: Relation = new Relation() + + override def reduce(b: Relation, a: (String, Relation)): Relation = { + a._2 + } + + + override def merge(a: Relation, b: Relation): Relation = { + if(b!= null) b else a + } + override def finish(reduction: Relation): Relation = reduction + + override def bufferEncoder: Encoder[Relation] = + Encoders.kryo(classOf[Relation]) + + override def outputEncoder: Encoder[Relation] = + Encoders.kryo(classOf[Relation]) + } + + + +} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/SparkCreateEBIDataFrame.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/SparkCreateEBIDataFrame.scala new file mode 100644 index 000000000..c7cc3ed04 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/SparkCreateEBIDataFrame.scala @@ -0,0 +1,83 @@ +package eu.dnetlib.dhp.sx.ebi + +import eu.dnetlib.dhp.application.ArgumentApplicationParser +import eu.dnetlib.dhp.schema.oaf.{Oaf, Publication, Relation, Dataset => OafDataset} +import eu.dnetlib.dhp.sx.graph.parser.{DatasetScholexplorerParser, PublicationScholexplorerParser} +import eu.dnetlib.scholexplorer.relation.RelationMapper +import org.apache.commons.io.IOUtils +import org.apache.spark.SparkConf +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession} +import org.codehaus.jackson.map.{ObjectMapper, SerializationConfig} +import org.slf4j.{Logger, LoggerFactory} +import scala.collection.JavaConverters._ + +object SparkCreateEBIDataFrame { + + + def main(args: Array[String]): Unit = { + val logger: Logger = LoggerFactory.getLogger(SparkCreateEBIDataFrame.getClass) + val conf: SparkConf = new SparkConf() + val parser = new ArgumentApplicationParser(IOUtils.toString(SparkCreateEBIDataFrame.getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/ebi/ebi_to_df_params.json"))) + parser.parseArgument(args) + val spark: SparkSession = + SparkSession + .builder() + .config(conf) + .appName(SparkCreateEBIDataFrame.getClass.getSimpleName) + .master(parser.get("master")).getOrCreate() + + val sc = spark.sparkContext + + + val workingPath = parser.get("workingPath") + val relationMapper = RelationMapper.load + + implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf]) + implicit val datasetEncoder: Encoder[OafDataset] = Encoders.kryo(classOf[OafDataset]) + implicit val pubEncoder: Encoder[Publication] = Encoders.kryo(classOf[Publication]) + implicit val relEncoder: Encoder[Relation] = Encoders.kryo(classOf[Relation]) + + logger.info("Extract Publication and relation from publication_xml") + val oafPubsRDD:RDD[Oaf] = sc.textFile(s"$workingPath/publication_xml").map(s => + { + new ObjectMapper().readValue(s, classOf[String]) + }).flatMap(s => { + val d = new PublicationScholexplorerParser + d.parseObject(s, relationMapper).asScala.iterator}) + + val mapper = new ObjectMapper() + mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT) + spark.createDataset(oafPubsRDD).write.mode(SaveMode.Overwrite).save(s"$workingPath/oaf") + + logger.info("Extract Publication and relation from dataset_xml") + val oafDatsRDD:RDD[Oaf] = sc.textFile(s"$workingPath/_dataset_xml").map(s => + { + new ObjectMapper().readValue(s, classOf[String]) + }).flatMap(s => { + val d = new DatasetScholexplorerParser + d.parseObject(s, relationMapper).asScala.iterator}) + + spark.createDataset(oafDatsRDD).write.mode(SaveMode.Append).save(s"$workingPath/oaf") + val dataset: Dataset[OafDataset] = spark.read.load(s"$workingPath/oaf").as[Oaf].filter(o => o.isInstanceOf[OafDataset]).map(d => d.asInstanceOf[OafDataset]) + val publication: Dataset[Publication] = spark.read.load(s"$workingPath/oaf").as[Oaf].filter(o => o.isInstanceOf[Publication]).map(d => d.asInstanceOf[Publication]) + val relations: Dataset[Relation] = spark.read.load(s"$workingPath/oaf").as[Oaf].filter(o => o.isInstanceOf[Relation]).map(d => d.asInstanceOf[Relation]) + publication.map(d => (d.getId, d))(Encoders.tuple(Encoders.STRING, pubEncoder)) + .groupByKey(_._1)(Encoders.STRING) + .agg(EBIAggregator.getPublicationAggregator().toColumn) + .map(p => p._2) + .write.mode(SaveMode.Overwrite).save(s"$workingPath/publication") + + dataset.map(d => (d.getId, d))(Encoders.tuple(Encoders.STRING, datasetEncoder)) + .groupByKey(_._1)(Encoders.STRING) + .agg(EBIAggregator.getDatasetAggregator().toColumn) + .map(p => p._2) + .write.mode(SaveMode.Overwrite).save(s"$workingPath/dataset") + + relations.map(d => (s"${d.getSource}::${d.getRelType}::${d.getTarget}", d))(Encoders.tuple(Encoders.STRING, relEncoder)) + .groupByKey(_._1)(Encoders.STRING) + .agg(EBIAggregator.getRelationAggregator().toColumn) + .map(p => p._2) + .write.mode(SaveMode.Overwrite).save(s"$workingPath/relation") + } +} 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 0db2b2688..f0c789323 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 @@ -150,6 +150,19 @@ public abstract class AbstractScholexplorerParser { return uk; } + + protected Qualifier generateQualifier(final String classId, final String className, final String schemeId, final String schemeName) { + final Qualifier q = new Qualifier(); + q.setClassid(classId); + q.setClassid(className); + q.setSchemeid(schemeId); + q.setSchemename(schemeName); + return q; + + + + } + protected void generateRelations( RelationMapper relationMapper, Result parsedObject, 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 afba57bb8..c5c6b1493 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 @@ -64,7 +64,6 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser { 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']"); @@ -149,6 +148,42 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser { inferPid(currentPid); parsedObject.setPid(Collections.singletonList(currentPid)); + + String resolvedURL = null; + + switch (currentPid.getQualifier().getClassname().toLowerCase()) { + case "uniprot": + resolvedURL ="https://www.uniprot.org/uniprot/"+currentPid.getValue(); + break; + case "ena": + if (StringUtils.isNotBlank(currentPid.getValue()) && currentPid.getValue().length() > 7) + resolvedURL ="https://www.ebi.ac.uk/ena/data/view/"+currentPid.getValue().substring(0,8); + break; + case "chembl": + resolvedURL ="https://www.ebi.ac.uk/chembl/compound_report_card/"+currentPid.getValue(); + break; + + case "ncbi-n": + resolvedURL ="https://www.ncbi.nlm.nih.gov/nuccore/"+currentPid.getValue(); + break; + case "ncbi-p": + resolvedURL ="https://www.ncbi.nlm.nih.gov/nuccore/"+currentPid.getValue(); + break; + case "genbank": + resolvedURL ="https://www.ncbi.nlm.nih.gov/nuccore/"+currentPid.getValue(); + break; + case "pdb": + resolvedURL ="https://www.ncbi.nlm.nih.gov/nuccore/"+currentPid.getValue(); + break; + case "url": + resolvedURL =currentPid.getValue(); + break; + } + + + + + final String sourceId = generateId( currentPid.getValue(), currentPid.getQualifier().getClassid(), "dataset"); parsedObject.setId(sourceId); @@ -251,6 +286,7 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser { t -> { final StructuredProperty st = new StructuredProperty(); st.setValue(t); + st.setQualifier(generateQualifier( "main title","main title", "dnet:dataCite_title","dnet:dataCite_title")); return st; }) .collect(Collectors.toList())); @@ -282,6 +318,14 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser { .collect(Collectors.toList())); } + + if(StringUtils.isNotBlank(resolvedURL)) { + Instance i = new Instance(); + i.setCollectedfrom(parsedObject.getCollectedfrom().get(0)); + i.setUrl(Collections.singletonList(resolvedURL)); + parsedObject.setInstance(Collections.singletonList(i)); + } + result.add(parsedObject); return result; } catch (Throwable e) { 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 bf59a6f0e..b66bfd161 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 @@ -202,6 +202,7 @@ public class PublicationScholexplorerParser extends AbstractScholexplorerParser t -> { final StructuredProperty st = new StructuredProperty(); st.setValue(t); + st.setQualifier(generateQualifier( "main title","main title", "dnet:dataCite_title","dnet:dataCite_title")); return st; }) .collect(Collectors.toList())); diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/ebi_to_df_params.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/ebi_to_df_params.json new file mode 100644 index 000000000..366f1426e --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/ebi_to_df_params.json @@ -0,0 +1,4 @@ +[ + {"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true}, + {"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true} +] \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/oozie_app/config-default.xml new file mode 100644 index 000000000..cf617a84c --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/oozie_app/config-default.xml @@ -0,0 +1,38 @@ + + + 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 + + + spark2EventLogDir + /user/spark/spark2ApplicationHistory + + + spark2ExtraListeners + "com.cloudera.spark.lineage.NavigatorAppListener" + + + spark2SqlQueryExecutionListeners + "com.cloudera.spark.lineage.NavigatorQueryListener" + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/oozie_app/workflow.xml new file mode 100644 index 000000000..b81669674 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/oozie_app/workflow.xml @@ -0,0 +1,51 @@ + + + + workingPath + the Working 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())}] + + + + + + yarn-cluster + cluster + Create EBI DataSet + eu.dnetlib.dhp.sx.ebi.SparkCreateEBIDataFrame + dhp-doiboost-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.sql.shuffle.partitions=1000 + ${sparkExtraOPT} + + --workingPath${workingPath} + --masteryarn-cluster + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/ebi/TestEBITODS.scala b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/ebi/TestEBITODS.scala new file mode 100644 index 000000000..979bf7e14 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/ebi/TestEBITODS.scala @@ -0,0 +1,22 @@ +package eu.dnetlib.dhp.sx.ebi + +import org.junit.jupiter.api.Test + +class TestEBITODS { + + + @Test + def testEBI():Unit = { + + + + + + + + + } + + + +} From ed1c7e5d757bd4effec6f69cc12390e46c953519 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 2 Jul 2020 12:40:21 +0200 Subject: [PATCH 010/100] fixed workflow for the import of the claims alone --- .../dnetlib/dhp/oa/graph/raw_claims/oozie_app/workflow.xml | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/oozie_app/workflow.xml index 13ec192ef..66eaeeb26 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/oozie_app/workflow.xml @@ -1,9 +1,5 @@ - - graphOutputPath - the target path to store raw graph - reuseContent false @@ -152,7 +148,7 @@ -lstore -iclaim - + From d380b85246bf26398d8e3b1c569a3cf1c5ea71c9 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 2 Jul 2020 12:42:13 +0200 Subject: [PATCH 011/100] unit test for the preparation of the relations --- .../oa/provision/PrepareRelationsJobTest.java | 29 ++++++++++++------- 1 file changed, 18 insertions(+), 11 deletions(-) diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJobTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJobTest.java index c16bbc6fb..528532edd 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJobTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJobTest.java @@ -1,9 +1,10 @@ package eu.dnetlib.dhp.oa.provision; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport; -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.function.FilterFunction; @@ -19,9 +20,10 @@ import org.junit.jupiter.api.io.TempDir; 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 eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport; +import eu.dnetlib.dhp.schema.oaf.Relation; public class PrepareRelationsJobTest { @@ -74,14 +76,19 @@ public class PrepareRelationsJobTest { "-maxRelations", String.valueOf(maxRelations) }); - Dataset out = spark.read() - .parquet(testPath.toString()) - .as(Encoders.bean(Relation.class)) - .cache(); + Dataset out = spark + .read() + .parquet(testPath.toString()) + .as(Encoders.bean(Relation.class)) + .cache(); Assertions.assertEquals(10, out.count()); - Dataset freq = out.toDF().cube(SUBRELTYPE).count().filter((FilterFunction) value -> !value.isNullAt(0)); + Dataset freq = out + .toDF() + .cube(SUBRELTYPE) + .count() + .filter((FilterFunction) value -> !value.isNullAt(0)); long outcome = freq.filter(freq.col(SUBRELTYPE).equalTo(OUTCOME)).collectAsList().get(0).getAs("count"); long supplement = freq.filter(freq.col(SUBRELTYPE).equalTo(SUPPLEMENT)).collectAsList().get(0).getAs("count"); From b413db0bff8d7ae4d5c9bfb4eb38205a198325da Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Thu, 2 Jul 2020 12:43:03 +0200 Subject: [PATCH 012/100] white/blacklists --- dhp-workflows/dhp-broker-events/pom.xml | 2 +- .../dhp/broker/model/EventFactory.java | 3 + .../dhp/broker/model/MappedFields.java | 22 +++++ .../dhp/broker/oa/GenerateEventsJob.java | 38 ++++++++- .../dnetlib/dhp/broker/oa/IndexOnESJob.java | 1 + .../dnetlib/dhp/broker/oa/JoinStep0Job.java | 80 +++++++++++++++++++ .../dnetlib/dhp/broker/oa/JoinStep1Job.java | 2 +- .../oa/PrepareRelatedDatasourcesJob.java | 68 ++++++++++++++++ .../dhp/broker/oa/util/EventFinder.java | 29 ++++++- .../dhp/broker/oa/util/UpdateInfo.java | 3 +- .../withRels/AddDatasourceTypeAggregator.java | 59 ++++++++++++++ .../withRels/SimpleDatasourceInfo.java | 40 ++++++++++ .../oa/generate_all/oozie_app/workflow.xml | 69 +++++++++++++++- .../dhp/broker/oa/generate_events.json | 18 +++++ .../broker/oa/partial/oozie_app/workflow.xml | 15 ++-- 15 files changed, 433 insertions(+), 16 deletions(-) create mode 100644 dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep0Job.java create mode 100644 dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedDatasourcesJob.java create mode 100644 dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/AddDatasourceTypeAggregator.java create mode 100644 dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/SimpleDatasourceInfo.java diff --git a/dhp-workflows/dhp-broker-events/pom.xml b/dhp-workflows/dhp-broker-events/pom.xml index e3182c259..119031b06 100644 --- a/dhp-workflows/dhp-broker-events/pom.xml +++ b/dhp-workflows/dhp-broker-events/pom.xml @@ -59,7 +59,7 @@ eu.dnetlib dnet-openaire-broker-common - [3.0.3,4.0.0) + [3.0.4,4.0.0) diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/model/EventFactory.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/model/EventFactory.java index 315a054d3..49e750698 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/model/EventFactory.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/model/EventFactory.java @@ -42,6 +42,7 @@ public class EventFactory { res.setCreationDate(now); res.setExpiryDate(calculateExpiryDate(now)); res.setInstantMessage(false); + return res; } @@ -53,6 +54,7 @@ public class EventFactory { map.setTargetDatasourceId(target.getCollectedFromId()); map.setTargetDatasourceName(target.getCollectedFromName()); + map.setTargetDatasourceType(target.getCollectedFromType()); map.setTargetResultId(target.getOpenaireId()); @@ -73,6 +75,7 @@ public class EventFactory { map.setTrust(updateInfo.getTrust()); map.setProvenanceDatasourceId(source.getCollectedFromId()); map.setProvenanceDatasourceName(source.getCollectedFromName()); + map.setProvenanceDatasourceType(source.getCollectedFromType()); map.setProvenanceResultId(source.getOpenaireId()); return map; diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/model/MappedFields.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/model/MappedFields.java index 4b0ed171b..2c1be3ba4 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/model/MappedFields.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/model/MappedFields.java @@ -13,6 +13,7 @@ public class MappedFields implements Serializable { private String targetDatasourceId; private String targetDatasourceName; + private String targetDatasourceType; private String targetResultId; private String targetResultTitle; private long targetDateofacceptance; @@ -21,6 +22,7 @@ public class MappedFields implements Serializable { private float trust; private String provenanceDatasourceId; private String provenanceDatasourceName; + private String provenanceDatasourceType; private String provenanceResultId; public String getTargetDatasourceId() { @@ -39,6 +41,14 @@ public class MappedFields implements Serializable { this.targetDatasourceName = targetDatasourceName; } + public String getTargetDatasourceType() { + return targetDatasourceType; + } + + public void setTargetDatasourceType(final String targetDatasourceType) { + this.targetDatasourceType = targetDatasourceType; + } + public String getTargetResultId() { return targetResultId; } @@ -103,6 +113,14 @@ public class MappedFields implements Serializable { this.provenanceDatasourceName = provenanceDatasourceName; } + public String getProvenanceDatasourceType() { + return provenanceDatasourceType; + } + + public void setProvenanceDatasourceType(final String provenanceDatasourceType) { + this.provenanceDatasourceType = provenanceDatasourceType; + } + public String getProvenanceResultId() { return provenanceResultId; } @@ -111,4 +129,8 @@ public class MappedFields implements Serializable { this.provenanceResultId = provenanceResultId; } + public static long getSerialversionuid() { + return serialVersionUID; + } + } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java index d6ac71429..5d3121aed 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java @@ -3,11 +3,15 @@ package eu.dnetlib.dhp.broker.oa; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; +import java.util.Arrays; +import java.util.HashSet; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.spark.SparkConf; import org.apache.spark.SparkContext; import org.apache.spark.sql.Dataset; @@ -59,6 +63,15 @@ public class GenerateEventsJob { final String eventsPath = workingPath + "/events"; log.info("eventsPath: {}", eventsPath); + final Set dsIdWhitelist = parseParamAsList(parser, "datasourceIdWhitelist"); + log.info("datasourceIdWhitelist: {}", StringUtils.join(dsIdWhitelist, ",")); + + final Set dsTypeWhitelist = parseParamAsList(parser, "datasourceTypeWhitelist"); + log.info("datasourceTypeWhitelist: {}", StringUtils.join(dsTypeWhitelist, ",")); + + final Set dsIdBlacklist = parseParamAsList(parser, "datasourceIdBlacklist"); + log.info("datasourceIdBlacklist: {}", StringUtils.join(dsIdBlacklist, ",")); + final SparkConf conf = new SparkConf(); // TODO UNCOMMENT @@ -77,9 +90,12 @@ public class GenerateEventsJob { .readPath(spark, workingPath + "/duplicates", ResultGroup.class); final Dataset dataset = groups - .map(g -> EventFinder.generateEvents(g, dedupConfig, accumulators), Encoders.bean(EventGroup.class)) - .flatMap(g -> g.getData().iterator(), Encoders.bean(Event.class)) - .map(e -> ClusterUtils.incrementAccumulator(e, total), Encoders.bean(Event.class)); + .map( + g -> EventFinder + .generateEvents(g, dsIdWhitelist, dsIdBlacklist, dsTypeWhitelist, dedupConfig, accumulators), + Encoders + .bean(EventGroup.class)) + .flatMap(g -> g.getData().iterator(), Encoders.bean(Event.class)); ClusterUtils.save(dataset, eventsPath, Event.class, total); @@ -87,6 +103,22 @@ public class GenerateEventsJob { } + private static Set parseParamAsList(final ArgumentApplicationParser parser, final String key) { + final String s = parser.get(key).trim(); + + final Set res = new HashSet<>(); + + if (s.length() > 1) { // A value of a single char (for example: '-') indicates an empty list + Arrays + .stream(s.split(",")) + .map(String::trim) + .filter(StringUtils::isNotBlank) + .forEach(res::add); + } + + return res; + } + public static Map prepareAccumulators(final SparkContext sc) { return EventFinder diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/IndexOnESJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/IndexOnESJob.java index 36d0ffd1b..9124d18e3 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/IndexOnESJob.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/IndexOnESJob.java @@ -48,6 +48,7 @@ public class IndexOnESJob { final JavaRDD inputRdd = ClusterUtils .readPath(spark, eventsPath, Event.class) + .limit(10000) // TODO REMOVE .map(IndexOnESJob::eventAsJsonString, Encoders.STRING()) .javaRDD(); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep0Job.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep0Job.java new file mode 100644 index 000000000..eb1825fa5 --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep0Job.java @@ -0,0 +1,80 @@ + +package eu.dnetlib.dhp.broker.oa; + +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.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.TypedColumn; +import org.apache.spark.util.LongAccumulator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.broker.objects.OaBrokerMainEntity; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.broker.oa.util.ClusterUtils; +import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.AddDatasourceTypeAggregator; +import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.SimpleDatasourceInfo; +import scala.Tuple2; + +public class JoinStep0Job { + + private static final Logger log = LoggerFactory.getLogger(JoinStep0Job.class); + + public static void main(final String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + JoinStep0Job.class + .getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json"))); + parser.parseArgument(args); + + final Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + final String graphPath = parser.get("graphPath"); + log.info("graphPath: {}", graphPath); + + final String workingPath = parser.get("workingPath"); + log.info("workingPath: {}", workingPath); + + final String outputPath = workingPath + "/joinedEntities_step0"; + log.info("outputPath: {}", outputPath); + + final SparkConf conf = new SparkConf(); + + runWithSparkSession(conf, isSparkSessionManaged, spark -> { + + ClusterUtils.removeDir(spark, outputPath); + + final LongAccumulator total = spark.sparkContext().longAccumulator("total_entities"); + + final Dataset sources = ClusterUtils + .readPath(spark, workingPath + "/simpleEntities", OaBrokerMainEntity.class); + + final Dataset datasources = ClusterUtils + .readPath(spark, workingPath + "/datasources", SimpleDatasourceInfo.class); + + final TypedColumn, OaBrokerMainEntity> aggr = new AddDatasourceTypeAggregator() + .toColumn(); + + final Dataset dataset = sources + .joinWith(datasources, sources.col("collectedFromId").equalTo(datasources.col("id")), "inner") + .groupByKey(t -> t._1.getOpenaireId(), Encoders.STRING()) + .agg(aggr) + .map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class)); + + ClusterUtils.save(dataset, outputPath, OaBrokerMainEntity.class, total); + + }); + + } + +} diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep1Job.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep1Job.java index f9bf2d146..8e502f736 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep1Job.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep1Job.java @@ -55,7 +55,7 @@ public class JoinStep1Job { final LongAccumulator total = spark.sparkContext().longAccumulator("total_entities"); final Dataset sources = ClusterUtils - .readPath(spark, workingPath + "/simpleEntities", OaBrokerMainEntity.class); + .readPath(spark, workingPath + "/joinedEntities_step0", OaBrokerMainEntity.class); final Dataset typedRels = ClusterUtils .readPath(spark, workingPath + "/relatedProjects", RelatedProject.class); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedDatasourcesJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedDatasourcesJob.java new file mode 100644 index 000000000..30f5ddac3 --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedDatasourcesJob.java @@ -0,0 +1,68 @@ + +package eu.dnetlib.dhp.broker.oa; + +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.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.util.LongAccumulator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.broker.oa.util.ClusterUtils; +import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.SimpleDatasourceInfo; +import eu.dnetlib.dhp.schema.oaf.Datasource; + +public class PrepareRelatedDatasourcesJob { + + private static final Logger log = LoggerFactory.getLogger(PrepareRelatedDatasourcesJob.class); + + public static void main(final String[] args) throws Exception { + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + PrepareRelatedDatasourcesJob.class + .getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json"))); + parser.parseArgument(args); + + final Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + final String graphPath = parser.get("graphPath"); + log.info("graphPath: {}", graphPath); + + final String workingPath = parser.get("workingPath"); + log.info("workingPath: {}", workingPath); + + final String relsPath = workingPath + "/datasources"; + log.info("relsPath: {}", relsPath); + + final SparkConf conf = new SparkConf(); + + runWithSparkSession(conf, isSparkSessionManaged, spark -> { + + ClusterUtils.removeDir(spark, relsPath); + + final LongAccumulator total = spark.sparkContext().longAccumulator("total_datasources"); + + final Dataset dataset = ClusterUtils + .readPath(spark, graphPath + "/datasource", Datasource.class) + .map( + ds -> new SimpleDatasourceInfo(ds.getId(), ds.getDatasourcetype().getClassid()), + Encoders.bean(SimpleDatasourceInfo.class)); + + ClusterUtils.save(dataset, relsPath, SimpleDatasourceInfo.class, total); + + }); + + } + +} diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java index 5ed55247b..e7abae68b 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java @@ -4,8 +4,11 @@ package eu.dnetlib.dhp.broker.oa.util; import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.spark.util.LongAccumulator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.dhp.broker.model.EventFactory; @@ -38,6 +41,8 @@ import eu.dnetlib.pace.config.DedupConfig; public class EventFinder { + private static final Logger log = LoggerFactory.getLogger(EventFinder.class); + private static final List> matchers = new ArrayList<>(); static { matchers.add(new EnrichMissingAbstract()); @@ -69,19 +74,39 @@ public class EventFinder { } public static EventGroup generateEvents(final ResultGroup results, + final Set dsIdWhitelist, + final Set dsIdBlacklist, + final Set dsTypeWhitelist, final DedupConfig dedupConfig, final Map accumulators) { + final List> list = new ArrayList<>(); for (final OaBrokerMainEntity target : results.getData()) { - for (final UpdateMatcher matcher : matchers) { - list.addAll(matcher.searchUpdatesForRecord(target, results.getData(), dedupConfig, accumulators)); + if (verifyTarget(target, dsIdWhitelist, dsIdBlacklist, dsTypeWhitelist)) { + for (final UpdateMatcher matcher : matchers) { + list.addAll(matcher.searchUpdatesForRecord(target, results.getData(), dedupConfig, accumulators)); + } } } return asEventGroup(list); } + private static boolean verifyTarget(final OaBrokerMainEntity target, + final Set dsIdWhitelist, + final Set dsIdBlacklist, + final Set dsTypeWhitelist) { + + if (dsIdWhitelist.contains(target.getCollectedFromId())) { + return true; + } else if (dsIdBlacklist.contains(target.getCollectedFromId())) { + return false; + } else { + return dsTypeWhitelist.contains(target.getCollectedFromType()); + } + } + private static EventGroup asEventGroup(final List> list) { final EventGroup events = new EventGroup(); list.stream().map(EventFactory::newBrokerEvent).forEach(events::addElement); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/UpdateInfo.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/UpdateInfo.java index 048683b50..0586b681e 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/UpdateInfo.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/UpdateInfo.java @@ -113,6 +113,7 @@ public final class UpdateInfo { final String provId = getSource().getOpenaireId(); final String provRepo = getSource().getCollectedFromName(); + final String provType = getSource().getCollectedFromType(); final String provUrl = getSource() .getInstances() @@ -122,7 +123,7 @@ public final class UpdateInfo { .orElse(null); ; - final OaBrokerProvenance provenance = new OaBrokerProvenance(provId, provRepo, provUrl); + final OaBrokerProvenance provenance = new OaBrokerProvenance(provId, provRepo, provType, provUrl); final OaBrokerEventPayload res = new OaBrokerEventPayload(); res.setResult(target); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/AddDatasourceTypeAggregator.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/AddDatasourceTypeAggregator.java new file mode 100644 index 000000000..ccd15c8c6 --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/AddDatasourceTypeAggregator.java @@ -0,0 +1,59 @@ + +package eu.dnetlib.dhp.broker.oa.util.aggregators.withRels; + +import org.apache.commons.lang3.StringUtils; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.expressions.Aggregator; + +import eu.dnetlib.broker.objects.OaBrokerMainEntity; +import scala.Tuple2; + +public class AddDatasourceTypeAggregator + extends Aggregator, OaBrokerMainEntity, OaBrokerMainEntity> { + + /** + * + */ + private static final long serialVersionUID = 8788588975496014728L; + + @Override + public OaBrokerMainEntity zero() { + return new OaBrokerMainEntity(); + } + + @Override + public OaBrokerMainEntity finish(final OaBrokerMainEntity g) { + return g; + } + + @Override + public OaBrokerMainEntity reduce(final OaBrokerMainEntity g, + final Tuple2 t) { + final OaBrokerMainEntity res = StringUtils.isNotBlank(g.getOpenaireId()) ? g : t._1; + if (t._2 != null && StringUtils.isNotBlank(t._2.getType())) { + res.setCollectedFromType(t._2.getType()); + } + return res; + + } + + @Override + public OaBrokerMainEntity merge(final OaBrokerMainEntity g1, final OaBrokerMainEntity g2) { + if (StringUtils.isNotBlank(g1.getOpenaireId()) && StringUtils.isNotBlank(g1.getCollectedFromType())) { + return g1; + } else { + return g2; + } + } + + @Override + public Encoder bufferEncoder() { + return Encoders.bean(OaBrokerMainEntity.class); + } + + @Override + public Encoder outputEncoder() { + return Encoders.bean(OaBrokerMainEntity.class); + } +} diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/SimpleDatasourceInfo.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/SimpleDatasourceInfo.java new file mode 100644 index 000000000..966f63fa0 --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/SimpleDatasourceInfo.java @@ -0,0 +1,40 @@ + +package eu.dnetlib.dhp.broker.oa.util.aggregators.withRels; + +import java.io.Serializable; + +public class SimpleDatasourceInfo implements Serializable { + + /** + * + */ + private static final long serialVersionUID = 2996609859416024734L; + + private String id; + private String type; + + public SimpleDatasourceInfo() { + } + + public SimpleDatasourceInfo(final String id, final String type) { + this.id = id; + this.type = type; + } + + public String getId() { + return id; + } + + public void setId(final String id) { + this.id = id; + } + + public String getType() { + return type; + } + + public void setType(final String type) { + this.type = type; + } + +} diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml index 7667bfba7..a0c7b00db 100644 --- a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml @@ -16,6 +16,21 @@ dedupConfProfId the id of a valid Dedup Configuration Profile + + + datasourceIdWhitelist + - + a white list (comma separeted, - for empty list) of datasource ids + + + datasourceTypeWhitelist + - + a white list (comma separeted, - for empty list) of datasource types + + + datasourceIdBlacklist + - + a black list (comma separeted, - for empty list) of datasource ids esIndexName @@ -96,6 +111,7 @@ + @@ -125,6 +141,30 @@ + + + + yarn + cluster + PrepareRelatedDatasourcesJob + eu.dnetlib.dhp.broker.oa.PrepareRelatedDatasourcesJob + dhp-broker-events-${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.sql.shuffle.partitions=3840 + + --graphPath${graphInputPath} + --workingPath${workingPath} + + + + @@ -223,7 +263,31 @@ - + + + + + yarn + cluster + JoinStep0 + eu.dnetlib.dhp.broker.oa.JoinStep0Job + dhp-broker-events-${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.sql.shuffle.partitions=3840 + + --graphPath${graphInputPath} + --workingPath${workingPath} + + + + @@ -365,6 +429,9 @@ --workingPath${workingPath} --isLookupUrl${isLookupUrl} --dedupConfProfile${dedupConfProfId} + --datasourceIdWhitelist${datasourceIdWhitelist} + --datasourceTypeWhitelist${datasourceTypeWhitelist} + --datasourceIdBlacklist${datasourceIdBlacklist} diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_events.json b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_events.json index 7ae076159..c545884f9 100644 --- a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_events.json +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_events.json @@ -16,5 +16,23 @@ "paramLongName": "dedupConfProfile", "paramDescription": "the id of a valid Dedup Configuration Profile", "paramRequired": true + }, + { + "paramName": "datasourceIdWhitelist", + "paramLongName": "datasourceIdWhitelist", + "paramDescription": "a white list (comma separeted, - for empty list) of datasource ids", + "paramRequired": true + }, + { + "paramName": "datasourceTypeWhitelist", + "paramLongName": "datasourceTypeWhitelist", + "paramDescription": "a white list (comma separeted, - for empty list) of datasource types", + "paramRequired": true + }, + { + "paramName": "datasourceIdBlacklist", + "paramLongName": "datasourceIdBlacklist", + "paramDescription": "a black list (comma separeted, - for empty list) of datasource ids", + "paramRequired": true } ] diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml index 9128c9820..b38290448 100644 --- a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml @@ -73,23 +73,24 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + + yarn cluster - GenerateEventsJob - eu.dnetlib.dhp.broker.oa.GenerateEventsJob + IndexOnESJob + eu.dnetlib.dhp.broker.oa.IndexOnESJob dhp-broker-events-${projectVersion}.jar - --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} + --conf spark.dynamicAllocation.maxExecutors="2" --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} @@ -97,8 +98,8 @@ --conf spark.sql.shuffle.partitions=3840 --workingPath${workingPath} - --isLookupUrl${isLookupUrl} - --dedupConfProfile${dedupConfProfId} + --index${esIndexName} + --esHost${esIndexHost} From 0f77cac4b55d83b2cdc1aa50c0c848f0720c3ed6 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 2 Jul 2020 12:43:51 +0200 Subject: [PATCH 013/100] fix: deduper must use queueMaxSize instead of groupMaxSize for the block definition --- .../src/main/java/eu/dnetlib/dhp/oa/dedup/Deduper.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 c72940deb..180f9f846 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 @@ -37,7 +37,7 @@ public class Deduper implements Serializable { public static JavaPairRDD createSortedBlocks( JavaPairRDD mapDocs, DedupConfig config) { final String of = config.getWf().getOrderField(); - final int maxQueueSize = config.getWf().getGroupMaxSize(); + final int maxQueueSize = config.getWf().getQueueMaxSize(); return mapDocs // the reduce is just to be sure that we haven't document with same id From c3d67f709a2589573330d85c298d33f17370d12c Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 2 Jul 2020 17:35:22 +0200 Subject: [PATCH 014/100] adjusted dedup configuration for result entities: using new wordssuffixprefix clustering function, removed ngrampairs, adjusted queueMaxSize (800) and slidingWindowSize (80) --- .../test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java | 6 +++--- .../resources/eu/dnetlib/dhp/dedup/conf/ds.curr.conf.json | 7 +++---- .../resources/eu/dnetlib/dhp/dedup/conf/orp.curr.conf.json | 7 +++---- .../resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json | 7 +++---- .../resources/eu/dnetlib/dhp/dedup/conf/sw.curr.conf.json | 7 +++---- pom.xml | 2 +- 6 files changed, 16 insertions(+), 20 deletions(-) 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 8dd00be97..88d5f24f9 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 @@ -182,7 +182,7 @@ public class SparkDedupTest implements Serializable { .count(); assertEquals(3432, orgs_simrel); - assertEquals(7152, pubs_simrel); + assertEquals(7054, pubs_simrel); assertEquals(344, sw_simrel); assertEquals(458, ds_simrel); assertEquals(6750, orp_simrel); @@ -234,7 +234,7 @@ public class SparkDedupTest implements Serializable { .count(); assertEquals(1276, orgs_mergerel); - assertEquals(1442, pubs_mergerel); + assertEquals(1440, pubs_mergerel); assertEquals(288, sw_mergerel); assertEquals(472, ds_mergerel); assertEquals(718, orp_mergerel); @@ -423,7 +423,7 @@ public class SparkDedupTest implements Serializable { long relations = jsc.textFile(testDedupGraphBasePath + "/relation").count(); - assertEquals(4975, relations); + assertEquals(4971, relations); // check deletedbyinference final Dataset mergeRels = spark diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/ds.curr.conf.json b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/ds.curr.conf.json index 13b18e1c3..2469b2cc0 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/ds.curr.conf.json +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/ds.curr.conf.json @@ -6,10 +6,10 @@ "subEntityType" : "resulttype", "subEntityValue" : "dataset", "orderField" : "title", - "queueMaxSize" : "2000", + "queueMaxSize" : "800", "groupMaxSize" : "100", "maxChildren" : "100", - "slidingWindowSize" : "200", + "slidingWindowSize" : "80", "rootBuilder" : ["result", "resultProject_outcome_isProducedBy", "resultResult_publicationDataset_isRelatedTo", "resultResult_similarity_isAmongTopNSimilarDocuments", "resultResult_similarity_hasAmongTopNSimilarDocuments", "resultOrganization_affiliation_hasAuthorInstitution", "resultResult_part_hasPart", "resultResult_part_isPartOf", "resultResult_supplement_isSupplementTo", "resultResult_supplement_isSupplementedBy", "resultResult_version_isVersionOf" ], "includeChildren" : "true", "idPath" : "$.id", @@ -17,8 +17,7 @@ }, "pace" : { "clustering" : [ - { "name" : "ngrampairs", "fields" : [ "title" ], "params" : { "max" : "1", "ngramLen" : "3"} }, - { "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } }, + { "name" : "wordssuffixprefix", "fields" : [ "title" ], "params" : { "max" : "2", "len" : "3" } }, { "name" : "lowercase", "fields" : [ "doi" ], "params" : { } } ], "decisionTree" : { diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/orp.curr.conf.json b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/orp.curr.conf.json index 5fb2a171a..4adcc0439 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/orp.curr.conf.json +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/orp.curr.conf.json @@ -6,10 +6,10 @@ "subEntityType" : "resulttype", "subEntityValue" : "otherresearchproduct", "orderField" : "title", - "queueMaxSize" : "2000", + "queueMaxSize" : "800", "groupMaxSize" : "100", "maxChildren" : "100", - "slidingWindowSize" : "200", + "slidingWindowSize" : "80", "rootBuilder" : [ "result", "resultProject_outcome_isProducedBy", "resultResult_publicationDataset_isRelatedTo", "resultResult_similarity_isAmongTopNSimilarDocuments", "resultResult_similarity_hasAmongTopNSimilarDocuments", "resultOrganization_affiliation_hasAuthorInstitution", "resultResult_part_hasPart", "resultResult_part_isPartOf", "resultResult_supplement_isSupplementTo", "resultResult_supplement_isSupplementedBy", "resultResult_version_isVersionOf" ], "includeChildren" : "true", "idPath" : "$.id", @@ -17,8 +17,7 @@ }, "pace" : { "clustering" : [ - { "name" : "ngrampairs", "fields" : [ "title" ], "params" : { "max" : "1", "ngramLen" : "3"} }, - { "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } }, + { "name" : "wordssuffixprefix", "fields" : [ "title" ], "params" : { "max" : "2", "len" : "3" } }, { "name" : "lowercase", "fields" : [ "doi" ], "params" : { } } ], "decisionTree" : { diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json index d471ccb89..ef0b26af4 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json @@ -6,10 +6,10 @@ "subEntityType": "resulttype", "subEntityValue": "publication", "orderField": "title", - "queueMaxSize": "2000", + "queueMaxSize": "800", "groupMaxSize": "100", "maxChildren": "100", - "slidingWindowSize": "200", + "slidingWindowSize": "80", "rootBuilder": [ "result", "resultProject_outcome_isProducedBy", @@ -29,8 +29,7 @@ }, "pace": { "clustering" : [ - { "name" : "ngrampairs", "fields" : [ "title" ], "params" : { "max" : "1", "ngramLen" : "3"} }, - { "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } }, + { "name" : "wordssuffixprefix", "fields" : [ "title" ], "params" : { "max" : "2", "len" : "3" } }, { "name" : "lowercase", "fields" : [ "doi" ], "params" : { } } ], "decisionTree": { diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/sw.curr.conf.json b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/sw.curr.conf.json index f4a107c74..623abbf9f 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/sw.curr.conf.json +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/sw.curr.conf.json @@ -6,10 +6,10 @@ "subEntityType" : "resulttype", "subEntityValue" : "software", "orderField" : "title", - "queueMaxSize" : "2000", + "queueMaxSize" : "800", "groupMaxSize" : "100", "maxChildren" : "100", - "slidingWindowSize" : "200", + "slidingWindowSize" : "80", "rootBuilder" : [ "result", "resultProject_outcome_isProducedBy", "resultResult_publicationDataset_isRelatedTo", "resultResult_similarity_isAmongTopNSimilarDocuments", "resultResult_similarity_hasAmongTopNSimilarDocuments", "resultOrganization_affiliation_hasAuthorInstitution", "resultResult_part_hasPart", "resultResult_part_isPartOf", "resultResult_supplement_isSupplementTo", "resultResult_supplement_isSupplementedBy", "resultResult_version_isVersionOf" ], "includeChildren" : "true", "idPath" : "$.id", @@ -17,8 +17,7 @@ }, "pace" : { "clustering" : [ - { "name" : "ngrampairs", "fields" : [ "title" ], "params" : { "max" : "1", "ngramLen" : "3"} }, - { "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } }, + { "name" : "wordssuffixprefix", "fields" : [ "title" ], "params" : { "max" : "2", "len" : "3" } }, { "name" : "lowercase", "fields" : [ "doi", "url" ], "params" : { } } ], "decisionTree": { diff --git a/pom.xml b/pom.xml index 89b7e8829..4619f3174 100644 --- a/pom.xml +++ b/pom.xml @@ -315,7 +315,7 @@ eu.dnetlib dnet-pace-core - 4.0.1 + 4.0.2 eu.dnetlib From 04bebb708c2bfa76362ce9b4cf5046193ffcf6e7 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Fri, 3 Jul 2020 11:48:12 +0200 Subject: [PATCH 015/100] some fixes --- .../relatedProjects/EnrichMissingProject.java | 2 +- .../relatedProjects/EnrichMoreProject.java | 14 ++--- .../dhp/broker/oa/util/ConversionUtils.java | 61 ++++++------------- 3 files changed, 25 insertions(+), 52 deletions(-) diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMissingProject.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMissingProject.java index 4b563d381..ab2735f2a 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMissingProject.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMissingProject.java @@ -15,7 +15,7 @@ public class EnrichMissingProject extends UpdateMatcher { super(20, prj -> Topic.ENRICH_MISSING_PROJECT, (p, prj) -> p.getProjects().add(prj), - prj -> prj.getFunder() + "::" + prj.getFundingProgram() + prj.getCode()); + prj -> prj.getOpenaireId()); } @Override diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMoreProject.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMoreProject.java index 6a10f19be..c38fd81a6 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMoreProject.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMoreProject.java @@ -18,31 +18,25 @@ public class EnrichMoreProject extends UpdateMatcher { super(20, prj -> Topic.ENRICH_MORE_PROJECT, (p, prj) -> p.getProjects().add(prj), - prj -> projectAsString(prj)); - } - - private static String projectAsString(final OaBrokerProject prj) { - return prj.getFunder() + "::" + prj.getFundingProgram() + "::" + prj.getCode(); + prj -> prj.getOpenaireId()); } @Override protected List findDifferences(final OaBrokerMainEntity source, final OaBrokerMainEntity target) { - if (target.getProjects().size() >= BrokerConstants.MAX_LIST_SIZE) { - return new ArrayList<>(); - } + if (target.getProjects().size() >= BrokerConstants.MAX_LIST_SIZE) { return new ArrayList<>(); } final Set existingProjects = target .getProjects() .stream() - .map(EnrichMoreProject::projectAsString) + .map(p -> p.getOpenaireId()) .collect(Collectors.toSet()); return source .getProjects() .stream() - .filter(p -> !existingProjects.contains(projectAsString(p))) + .filter(p -> !existingProjects.contains(p.getOpenaireId())) .collect(Collectors.toList()); } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java index 1ce84283a..3a2cdc5f3 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java @@ -44,9 +44,7 @@ public class ConversionUtils { private static final Logger log = LoggerFactory.getLogger(ConversionUtils.class); public static List oafInstanceToBrokerInstances(final Instance i) { - if (i == null) { - return new ArrayList<>(); - } + if (i == null) { return new ArrayList<>(); } return mappedList(i.getUrl(), url -> { final OaBrokerInstance res = new OaBrokerInstance(); @@ -67,9 +65,7 @@ public class ConversionUtils { } public static final OaBrokerRelatedDataset oafDatasetToBrokerDataset(final Dataset d) { - if (d == null) { - return null; - } + if (d == null) { return null; } final OaBrokerRelatedDataset res = new OaBrokerRelatedDataset(); res.setOpenaireId(d.getId()); @@ -82,9 +78,7 @@ public class ConversionUtils { } public static OaBrokerRelatedPublication oafPublicationToBrokerPublication(final Publication p) { - if (p == null) { - return null; - } + if (p == null) { return null; } final OaBrokerRelatedPublication res = new OaBrokerRelatedPublication(); res.setOpenaireId(p.getId()); @@ -98,9 +92,7 @@ public class ConversionUtils { } public static final OaBrokerMainEntity oafResultToBrokerResult(final Result result) { - if (result == null) { - return null; - } + if (result == null) { return null; } final OaBrokerMainEntity res = new OaBrokerMainEntity(); @@ -117,8 +109,7 @@ public class ConversionUtils { res.setEmbargoenddate(fieldValue(result.getEmbargoenddate())); res.setContributor(fieldList(result.getContributor())); res - .setJournal( - result instanceof Publication ? oafJournalToBrokerJournal(((Publication) result).getJournal()) : null); + .setJournal(result instanceof Publication ? oafJournalToBrokerJournal(((Publication) result).getJournal()) : null); res.setCollectedFromId(mappedFirst(result.getCollectedfrom(), KeyValue::getKey)); res.setCollectedFromName(mappedFirst(result.getCollectedfrom(), KeyValue::getValue)); res.setPids(mappedList(result.getPid(), ConversionUtils::oafPidToBrokerPid)); @@ -130,9 +121,7 @@ public class ConversionUtils { } private static OaBrokerAuthor oafAuthorToBrokerAuthor(final Author author) { - if (author == null) { - return null; - } + if (author == null) { return null; } final String pids = author.getPid() != null ? author .getPid() @@ -142,6 +131,7 @@ public class ConversionUtils { .filter(pid -> pid.getQualifier().getClassid() != null) .filter(pid -> pid.getQualifier().getClassid().equalsIgnoreCase("orcid")) .map(pid -> pid.getValue()) + .map(pid -> cleanOrcid(pid)) .filter(StringUtils::isNotBlank) .findFirst() .orElse(null) : null; @@ -149,10 +139,13 @@ public class ConversionUtils { return new OaBrokerAuthor(author.getFullname(), pids); } + private static String cleanOrcid(final String s) { + final String match = "//orcid.org/"; + return s.contains(match) ? StringUtils.substringAfter(s, match) : s; + } + private static OaBrokerJournal oafJournalToBrokerJournal(final Journal journal) { - if (journal == null) { - return null; - } + if (journal == null) { return null; } final OaBrokerJournal res = new OaBrokerJournal(); res.setName(journal.getName()); @@ -164,9 +157,7 @@ public class ConversionUtils { } private static OaBrokerExternalReference oafExtRefToBrokerExtRef(final ExternalReference ref) { - if (ref == null) { - return null; - } + if (ref == null) { return null; } final OaBrokerExternalReference res = new OaBrokerExternalReference(); res.setRefidentifier(ref.getRefidentifier()); @@ -177,9 +168,7 @@ public class ConversionUtils { } public static final OaBrokerProject oafProjectToBrokerProject(final Project p) { - if (p == null) { - return null; - } + if (p == null) { return null; } final OaBrokerProject res = new OaBrokerProject(); res.setOpenaireId(p.getId()); @@ -203,9 +192,7 @@ public class ConversionUtils { } public static final OaBrokerRelatedSoftware oafSoftwareToBrokerSoftware(final Software sw) { - if (sw == null) { - return null; - } + if (sw == null) { return null; } final OaBrokerRelatedSoftware res = new OaBrokerRelatedSoftware(); res.setOpenaireId(sw.getId()); @@ -268,9 +255,7 @@ public class ConversionUtils { } private static List structPropTypedList(final List list) { - if (list == null) { - return new ArrayList<>(); - } + if (list == null) { return new ArrayList<>(); } return list .stream() @@ -280,9 +265,7 @@ public class ConversionUtils { } private static List mappedList(final List list, final Function func) { - if (list == null) { - return new ArrayList<>(); - } + if (list == null) { return new ArrayList<>(); } return list .stream() @@ -293,9 +276,7 @@ public class ConversionUtils { } private static List flatMappedList(final List list, final Function> func) { - if (list == null) { - return new ArrayList<>(); - } + if (list == null) { return new ArrayList<>(); } return list .stream() @@ -307,9 +288,7 @@ public class ConversionUtils { } private static T mappedFirst(final List list, final Function func) { - if (list == null) { - return null; - } + if (list == null) { return null; } return list .stream() From b383ed42fa309b143fd5fba08e8297468c182b4b Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 7 Jul 2020 14:21:28 +0200 Subject: [PATCH 016/100] pass optional parameter relationFilter to the PrepareRelationJob implementation --- .../java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJob.java | 1 + .../resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml | 1 + 2 files changed, 2 insertions(+) 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 601cf6449..eb63d4423 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 @@ -156,6 +156,7 @@ public class PrepareRelationsJob { .parquet(outputPath); } + // experimental private static void prepareRelationsDataset( SparkSession spark, String inputRelationsPath, String outputPath, Set relationFilter, int maxRelations, int relPartitions) { diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml index 697a00a09..faa81ad64 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml @@ -134,6 +134,7 @@ --inputRelationsPath${inputGraphRootPath}/relation --outputPath${workingDir}/relation --maxRelations${maxRelations} + --relationFilter${relationFilter} --relPartitions5000 From 8af8e7481ad7762d7c0b83a7c9bfe487a322b103 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 7 Jul 2020 14:23:34 +0200 Subject: [PATCH 017/100] code formatting --- .../relatedProjects/EnrichMoreProject.java | 4 +- .../dhp/broker/oa/util/ConversionUtils.java | 55 ++++++++++++++----- 2 files changed, 44 insertions(+), 15 deletions(-) diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMoreProject.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMoreProject.java index c38fd81a6..85086a6df 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMoreProject.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMoreProject.java @@ -25,7 +25,9 @@ public class EnrichMoreProject extends UpdateMatcher { protected List findDifferences(final OaBrokerMainEntity source, final OaBrokerMainEntity target) { - if (target.getProjects().size() >= BrokerConstants.MAX_LIST_SIZE) { return new ArrayList<>(); } + if (target.getProjects().size() >= BrokerConstants.MAX_LIST_SIZE) { + return new ArrayList<>(); + } final Set existingProjects = target .getProjects() diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java index 3a2cdc5f3..d00c5b817 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java @@ -44,7 +44,9 @@ public class ConversionUtils { private static final Logger log = LoggerFactory.getLogger(ConversionUtils.class); public static List oafInstanceToBrokerInstances(final Instance i) { - if (i == null) { return new ArrayList<>(); } + if (i == null) { + return new ArrayList<>(); + } return mappedList(i.getUrl(), url -> { final OaBrokerInstance res = new OaBrokerInstance(); @@ -65,7 +67,9 @@ public class ConversionUtils { } public static final OaBrokerRelatedDataset oafDatasetToBrokerDataset(final Dataset d) { - if (d == null) { return null; } + if (d == null) { + return null; + } final OaBrokerRelatedDataset res = new OaBrokerRelatedDataset(); res.setOpenaireId(d.getId()); @@ -78,7 +82,9 @@ public class ConversionUtils { } public static OaBrokerRelatedPublication oafPublicationToBrokerPublication(final Publication p) { - if (p == null) { return null; } + if (p == null) { + return null; + } final OaBrokerRelatedPublication res = new OaBrokerRelatedPublication(); res.setOpenaireId(p.getId()); @@ -92,7 +98,9 @@ public class ConversionUtils { } public static final OaBrokerMainEntity oafResultToBrokerResult(final Result result) { - if (result == null) { return null; } + if (result == null) { + return null; + } final OaBrokerMainEntity res = new OaBrokerMainEntity(); @@ -109,7 +117,8 @@ public class ConversionUtils { res.setEmbargoenddate(fieldValue(result.getEmbargoenddate())); res.setContributor(fieldList(result.getContributor())); res - .setJournal(result instanceof Publication ? oafJournalToBrokerJournal(((Publication) result).getJournal()) : null); + .setJournal( + result instanceof Publication ? oafJournalToBrokerJournal(((Publication) result).getJournal()) : null); res.setCollectedFromId(mappedFirst(result.getCollectedfrom(), KeyValue::getKey)); res.setCollectedFromName(mappedFirst(result.getCollectedfrom(), KeyValue::getValue)); res.setPids(mappedList(result.getPid(), ConversionUtils::oafPidToBrokerPid)); @@ -121,7 +130,9 @@ public class ConversionUtils { } private static OaBrokerAuthor oafAuthorToBrokerAuthor(final Author author) { - if (author == null) { return null; } + if (author == null) { + return null; + } final String pids = author.getPid() != null ? author .getPid() @@ -145,7 +156,9 @@ public class ConversionUtils { } private static OaBrokerJournal oafJournalToBrokerJournal(final Journal journal) { - if (journal == null) { return null; } + if (journal == null) { + return null; + } final OaBrokerJournal res = new OaBrokerJournal(); res.setName(journal.getName()); @@ -157,7 +170,9 @@ public class ConversionUtils { } private static OaBrokerExternalReference oafExtRefToBrokerExtRef(final ExternalReference ref) { - if (ref == null) { return null; } + if (ref == null) { + return null; + } final OaBrokerExternalReference res = new OaBrokerExternalReference(); res.setRefidentifier(ref.getRefidentifier()); @@ -168,7 +183,9 @@ public class ConversionUtils { } public static final OaBrokerProject oafProjectToBrokerProject(final Project p) { - if (p == null) { return null; } + if (p == null) { + return null; + } final OaBrokerProject res = new OaBrokerProject(); res.setOpenaireId(p.getId()); @@ -192,7 +209,9 @@ public class ConversionUtils { } public static final OaBrokerRelatedSoftware oafSoftwareToBrokerSoftware(final Software sw) { - if (sw == null) { return null; } + if (sw == null) { + return null; + } final OaBrokerRelatedSoftware res = new OaBrokerRelatedSoftware(); res.setOpenaireId(sw.getId()); @@ -255,7 +274,9 @@ public class ConversionUtils { } private static List structPropTypedList(final List list) { - if (list == null) { return new ArrayList<>(); } + if (list == null) { + return new ArrayList<>(); + } return list .stream() @@ -265,7 +286,9 @@ public class ConversionUtils { } private static List mappedList(final List list, final Function func) { - if (list == null) { return new ArrayList<>(); } + if (list == null) { + return new ArrayList<>(); + } return list .stream() @@ -276,7 +299,9 @@ public class ConversionUtils { } private static List flatMappedList(final List list, final Function> func) { - if (list == null) { return new ArrayList<>(); } + if (list == null) { + return new ArrayList<>(); + } return list .stream() @@ -288,7 +313,9 @@ public class ConversionUtils { } private static T mappedFirst(final List list, final Function func) { - if (list == null) { return null; } + if (list == null) { + return null; + } return list .stream() From efadbdb2bc33d4df424d5f9699ea3507a04a05af Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Tue, 7 Jul 2020 15:37:13 +0200 Subject: [PATCH 018/100] fixed a bug with duplicated events --- .../dhp/broker/oa/CheckDuplictedIdsJob.java | 112 ++++++++++++++++++ .../dhp/broker/oa/GenerateEventsJob.java | 24 +--- .../dnetlib/dhp/broker/oa/IndexOnESJob.java | 2 +- .../relatedProjects/EnrichMoreProject.java | 4 +- .../dhp/broker/oa/util/ClusterUtils.java | 26 +++- .../dhp/broker/oa/util/ConversionUtils.java | 55 ++++++--- .../dhp/broker/oa/util/EventFinder.java | 1 - .../oa/generate_all/oozie_app/workflow.xml | 2 +- .../broker/oa/partial/oozie_app/workflow.xml | 13 +- 9 files changed, 192 insertions(+), 47 deletions(-) create mode 100644 dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/CheckDuplictedIdsJob.java diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/CheckDuplictedIdsJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/CheckDuplictedIdsJob.java new file mode 100644 index 000000000..5ca865e8f --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/CheckDuplictedIdsJob.java @@ -0,0 +1,112 @@ + +package eu.dnetlib.dhp.broker.oa; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.TypedColumn; +import org.apache.spark.sql.expressions.Aggregator; +import org.apache.spark.util.LongAccumulator; +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.broker.model.Event; +import eu.dnetlib.dhp.broker.oa.util.ClusterUtils; +import scala.Tuple2; + +public class CheckDuplictedIdsJob { + + private static final Logger log = LoggerFactory.getLogger(CheckDuplictedIdsJob.class); + + public static void main(final String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + CheckDuplictedIdsJob.class + .getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json"))); + parser.parseArgument(args); + + final SparkConf conf = new SparkConf(); + + final String eventsPath = parser.get("workingPath") + "/events"; + log.info("eventsPath: {}", eventsPath); + + final String countPath = parser.get("workingPath") + "/counts"; + log.info("countPath: {}", countPath); + + final SparkSession spark = SparkSession.builder().config(conf).getOrCreate(); + + final LongAccumulator total = spark.sparkContext().longAccumulator("invaild_event_id"); + + final TypedColumn, Tuple2> agg = new CountAggregator().toColumn(); + + ClusterUtils + .readPath(spark, eventsPath, Event.class) + .map(e -> new Tuple2<>(e.getEventId(), 1l), Encoders.tuple(Encoders.STRING(), Encoders.LONG())) + .groupByKey(t -> t._1, Encoders.STRING()) + .agg(agg) + .map(t -> t._2, Encoders.tuple(Encoders.STRING(), Encoders.LONG())) + .filter(t -> t._2 > 1) + .map(o -> ClusterUtils.incrementAccumulator(o, total), Encoders.tuple(Encoders.STRING(), Encoders.LONG())) + .write() + .mode(SaveMode.Overwrite) + .json(countPath); + ; + + } + + private static String eventAsJsonString(final Event f) throws JsonProcessingException { + return new ObjectMapper().writeValueAsString(f); + } + +} + +class CountAggregator extends Aggregator, Tuple2, Tuple2> { + + /** + * + */ + private static final long serialVersionUID = 1395935985734672538L; + + @Override + public Encoder> bufferEncoder() { + return Encoders.tuple(Encoders.STRING(), Encoders.LONG()); + } + + @Override + public Tuple2 finish(final Tuple2 arg0) { + return arg0; + } + + @Override + public Tuple2 merge(final Tuple2 arg0, final Tuple2 arg1) { + final String s = StringUtils.defaultIfBlank(arg0._1, arg1._1); + return new Tuple2<>(s, arg0._2 + arg1._2); + } + + @Override + public Encoder> outputEncoder() { + return Encoders.tuple(Encoders.STRING(), Encoders.LONG()); + } + + @Override + public Tuple2 reduce(final Tuple2 arg0, final Tuple2 arg1) { + final String s = StringUtils.defaultIfBlank(arg0._1, arg1._1); + return new Tuple2<>(s, arg0._2 + arg1._2); + } + + @Override + public Tuple2 zero() { + return new Tuple2<>(null, 0l); + } + +} diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java index 5d3121aed..a2d92e149 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java @@ -3,8 +3,6 @@ package eu.dnetlib.dhp.broker.oa; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import java.util.Arrays; -import java.util.HashSet; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -63,13 +61,13 @@ public class GenerateEventsJob { final String eventsPath = workingPath + "/events"; log.info("eventsPath: {}", eventsPath); - final Set dsIdWhitelist = parseParamAsList(parser, "datasourceIdWhitelist"); + final Set dsIdWhitelist = ClusterUtils.parseParamAsList(parser, "datasourceIdWhitelist"); log.info("datasourceIdWhitelist: {}", StringUtils.join(dsIdWhitelist, ",")); - final Set dsTypeWhitelist = parseParamAsList(parser, "datasourceTypeWhitelist"); + final Set dsTypeWhitelist = ClusterUtils.parseParamAsList(parser, "datasourceTypeWhitelist"); log.info("datasourceTypeWhitelist: {}", StringUtils.join(dsTypeWhitelist, ",")); - final Set dsIdBlacklist = parseParamAsList(parser, "datasourceIdBlacklist"); + final Set dsIdBlacklist = ClusterUtils.parseParamAsList(parser, "datasourceIdBlacklist"); log.info("datasourceIdBlacklist: {}", StringUtils.join(dsIdBlacklist, ",")); final SparkConf conf = new SparkConf(); @@ -103,22 +101,6 @@ public class GenerateEventsJob { } - private static Set parseParamAsList(final ArgumentApplicationParser parser, final String key) { - final String s = parser.get(key).trim(); - - final Set res = new HashSet<>(); - - if (s.length() > 1) { // A value of a single char (for example: '-') indicates an empty list - Arrays - .stream(s.split(",")) - .map(String::trim) - .filter(StringUtils::isNotBlank) - .forEach(res::add); - } - - return res; - } - public static Map prepareAccumulators(final SparkContext sc) { return EventFinder diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/IndexOnESJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/IndexOnESJob.java index 9124d18e3..806147bdd 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/IndexOnESJob.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/IndexOnESJob.java @@ -48,7 +48,7 @@ public class IndexOnESJob { final JavaRDD inputRdd = ClusterUtils .readPath(spark, eventsPath, Event.class) - .limit(10000) // TODO REMOVE + // .limit(10000) // TODO REMOVE .map(IndexOnESJob::eventAsJsonString, Encoders.STRING()) .javaRDD(); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMoreProject.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMoreProject.java index c38fd81a6..85086a6df 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMoreProject.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/relatedProjects/EnrichMoreProject.java @@ -25,7 +25,9 @@ public class EnrichMoreProject extends UpdateMatcher { protected List findDifferences(final OaBrokerMainEntity source, final OaBrokerMainEntity target) { - if (target.getProjects().size() >= BrokerConstants.MAX_LIST_SIZE) { return new ArrayList<>(); } + if (target.getProjects().size() >= BrokerConstants.MAX_LIST_SIZE) { + return new ArrayList<>(); + } final Set existingProjects = target .getProjects() diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ClusterUtils.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ClusterUtils.java index 2d0106a7a..d8b8dd807 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ClusterUtils.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ClusterUtils.java @@ -1,6 +1,11 @@ package eu.dnetlib.dhp.broker.oa.util; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; + +import org.apache.commons.lang3.StringUtils; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; @@ -10,6 +15,7 @@ import org.apache.spark.util.LongAccumulator; import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.common.HdfsSupport; public class ClusterUtils { @@ -53,7 +59,9 @@ public class ClusterUtils { return o; } - public static void save(final Dataset dataset, final String path, final Class clazz, + public static void save(final Dataset dataset, + final String path, + final Class clazz, final LongAccumulator acc) { dataset .map(o -> ClusterUtils.incrementAccumulator(o, acc), Encoders.bean(clazz)) @@ -62,4 +70,20 @@ public class ClusterUtils { .json(path); } + public static Set parseParamAsList(final ArgumentApplicationParser parser, final String key) { + final String s = parser.get(key).trim(); + + final Set res = new HashSet<>(); + + if (s.length() > 1) { // A value of a single char (for example: '-') indicates an empty list + Arrays + .stream(s.split(",")) + .map(String::trim) + .filter(StringUtils::isNotBlank) + .forEach(res::add); + } + + return res; + } + } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java index 3a2cdc5f3..d00c5b817 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java @@ -44,7 +44,9 @@ public class ConversionUtils { private static final Logger log = LoggerFactory.getLogger(ConversionUtils.class); public static List oafInstanceToBrokerInstances(final Instance i) { - if (i == null) { return new ArrayList<>(); } + if (i == null) { + return new ArrayList<>(); + } return mappedList(i.getUrl(), url -> { final OaBrokerInstance res = new OaBrokerInstance(); @@ -65,7 +67,9 @@ public class ConversionUtils { } public static final OaBrokerRelatedDataset oafDatasetToBrokerDataset(final Dataset d) { - if (d == null) { return null; } + if (d == null) { + return null; + } final OaBrokerRelatedDataset res = new OaBrokerRelatedDataset(); res.setOpenaireId(d.getId()); @@ -78,7 +82,9 @@ public class ConversionUtils { } public static OaBrokerRelatedPublication oafPublicationToBrokerPublication(final Publication p) { - if (p == null) { return null; } + if (p == null) { + return null; + } final OaBrokerRelatedPublication res = new OaBrokerRelatedPublication(); res.setOpenaireId(p.getId()); @@ -92,7 +98,9 @@ public class ConversionUtils { } public static final OaBrokerMainEntity oafResultToBrokerResult(final Result result) { - if (result == null) { return null; } + if (result == null) { + return null; + } final OaBrokerMainEntity res = new OaBrokerMainEntity(); @@ -109,7 +117,8 @@ public class ConversionUtils { res.setEmbargoenddate(fieldValue(result.getEmbargoenddate())); res.setContributor(fieldList(result.getContributor())); res - .setJournal(result instanceof Publication ? oafJournalToBrokerJournal(((Publication) result).getJournal()) : null); + .setJournal( + result instanceof Publication ? oafJournalToBrokerJournal(((Publication) result).getJournal()) : null); res.setCollectedFromId(mappedFirst(result.getCollectedfrom(), KeyValue::getKey)); res.setCollectedFromName(mappedFirst(result.getCollectedfrom(), KeyValue::getValue)); res.setPids(mappedList(result.getPid(), ConversionUtils::oafPidToBrokerPid)); @@ -121,7 +130,9 @@ public class ConversionUtils { } private static OaBrokerAuthor oafAuthorToBrokerAuthor(final Author author) { - if (author == null) { return null; } + if (author == null) { + return null; + } final String pids = author.getPid() != null ? author .getPid() @@ -145,7 +156,9 @@ public class ConversionUtils { } private static OaBrokerJournal oafJournalToBrokerJournal(final Journal journal) { - if (journal == null) { return null; } + if (journal == null) { + return null; + } final OaBrokerJournal res = new OaBrokerJournal(); res.setName(journal.getName()); @@ -157,7 +170,9 @@ public class ConversionUtils { } private static OaBrokerExternalReference oafExtRefToBrokerExtRef(final ExternalReference ref) { - if (ref == null) { return null; } + if (ref == null) { + return null; + } final OaBrokerExternalReference res = new OaBrokerExternalReference(); res.setRefidentifier(ref.getRefidentifier()); @@ -168,7 +183,9 @@ public class ConversionUtils { } public static final OaBrokerProject oafProjectToBrokerProject(final Project p) { - if (p == null) { return null; } + if (p == null) { + return null; + } final OaBrokerProject res = new OaBrokerProject(); res.setOpenaireId(p.getId()); @@ -192,7 +209,9 @@ public class ConversionUtils { } public static final OaBrokerRelatedSoftware oafSoftwareToBrokerSoftware(final Software sw) { - if (sw == null) { return null; } + if (sw == null) { + return null; + } final OaBrokerRelatedSoftware res = new OaBrokerRelatedSoftware(); res.setOpenaireId(sw.getId()); @@ -255,7 +274,9 @@ public class ConversionUtils { } private static List structPropTypedList(final List list) { - if (list == null) { return new ArrayList<>(); } + if (list == null) { + return new ArrayList<>(); + } return list .stream() @@ -265,7 +286,9 @@ public class ConversionUtils { } private static List mappedList(final List list, final Function func) { - if (list == null) { return new ArrayList<>(); } + if (list == null) { + return new ArrayList<>(); + } return list .stream() @@ -276,7 +299,9 @@ public class ConversionUtils { } private static List flatMappedList(final List list, final Function> func) { - if (list == null) { return new ArrayList<>(); } + if (list == null) { + return new ArrayList<>(); + } return list .stream() @@ -288,7 +313,9 @@ public class ConversionUtils { } private static T mappedFirst(final List list, final Function func) { - if (list == null) { return null; } + if (list == null) { + return null; + } return list .stream() diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java index e7abae68b..593e66d43 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java @@ -70,7 +70,6 @@ public class EventFinder { matchers.add(new EnrichMissingDatasetReferences()); matchers.add(new EnrichMissingDatasetIsSupplementedTo()); matchers.add(new EnrichMissingDatasetIsSupplementedBy()); - matchers.add(new EnrichMissingAbstract()); } public static EventGroup generateEvents(final ResultGroup results, diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml index a0c7b00db..b85c60fdf 100644 --- a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml @@ -447,7 +447,7 @@ --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} - --conf spark.dynamicAllocation.maxExecutors="2" + --conf spark.dynamicAllocation.maxExecutors="8" --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml index b38290448..392271260 100644 --- a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml @@ -73,33 +73,32 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + yarn cluster - IndexOnESJob - eu.dnetlib.dhp.broker.oa.IndexOnESJob + Count + eu.dnetlib.dhp.broker.oa.CheckDuplictedIdsJob dhp-broker-events-${projectVersion}.jar + --executor-cores=${sparkExecutorCores} --executor-memory=${sparkExecutorMemory} --driver-memory=${sparkDriverMemory} - --conf spark.dynamicAllocation.maxExecutors="2" --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.shuffle.partitions=3840 + --graphPath${graphInputPath} --workingPath${workingPath} - --index${esIndexName} - --esHost${esIndexHost} From e2ea30f89d7125770433120a60e8e262e1b4c654 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 8 Jul 2020 12:16:24 +0200 Subject: [PATCH 019/100] updated graph construction workflow definition: cleaning wf moved at the bottom to include cleaning of the information produced by the enrichment workflows --- .../dhp/wf/profiles/graph_construction.xml | 159 +++++++++++------- 1 file changed, 99 insertions(+), 60 deletions(-) diff --git a/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_construction.xml b/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_construction.xml index 819b3e12d..4d77883b4 100644 --- a/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_construction.xml +++ b/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_construction.xml @@ -11,6 +11,29 @@ Data Provision 30 + + + reuse cached content from the aggregation system + + reuseContent + true + + + + + + + + set the aggregator content path + + contentPath + /tmp/beta_aggregator + + + + + + Set the path containing the AGGREGATOR graph @@ -62,87 +85,94 @@ - - Set the target path to store the CLEANED graph + + Set the target path to store the ORCID enriched graph - cleanedGraphPath - /tmp/beta_provision/graph/05_graph_cleaned + orcidGraphPath + /tmp/beta_provision/graph/05_graph_orcid - - Set the target path to store the ORCID enriched graph - - orcidGraphPath - /tmp/beta_provision/graph/06_graph_orcid - - - - - Set the target path to store the BULK TAGGED graph bulkTaggingGraphPath - /tmp/beta_provision/graph/07_graph_bulktagging + /tmp/beta_provision/graph/06_graph_bulktagging + Set the target path to store the AFFILIATION from INSTITUTIONAL REPOS graph affiliationGraphPath - /tmp/beta_provision/graph/08_graph_affiliation + /tmp/beta_provision/graph/07_graph_affiliation + Set the target path to store the COMMUNITY from SELECTED SOURCES graph communityOrganizationGraphPath - /tmp/beta_provision/graph/09_graph_comunity_organization + /tmp/beta_provision/graph/08_graph_comunity_organization + Set the target path to store the FUNDING from SEMANTIC RELATION graph fundingGraphPath - /tmp/beta_provision/graph/10_graph_funding + /tmp/beta_provision/graph/09_graph_funding + Set the target path to store the COMMUNITY from SEMANTIC RELATION graph communitySemRelGraphPath - /tmp/beta_provision/graph/11_graph_comunity_sem_rel + /tmp/beta_provision/graph/10_graph_comunity_sem_rel + Set the target path to store the COUNTRY enriched graph countryGraphPath - /tmp/beta_provision/graph/12_graph_country + /tmp/beta_provision/graph/11_graph_country + + + Set the target path to store the CLEANED graph + + cleanedGraphPath + /tmp/beta_provision/graph/12_graph_cleaned + + + + + + Set the target path to store the blacklisted graph @@ -153,6 +183,7 @@ + Set the lookup address @@ -163,6 +194,7 @@ + Set the map of paths for the Bulk Tagging @@ -173,6 +205,7 @@ + Set the map of associations organization, community list for the propagation of community to result through organization @@ -185,6 +218,7 @@ + Set the dedup orchestrator name @@ -195,6 +229,7 @@ + declares the ActionSet ids to promote in the RAW graph @@ -205,6 +240,7 @@ + declares the ActionSet ids to promote in the INFERRED graph @@ -215,6 +251,7 @@ + wait configurations @@ -222,6 +259,7 @@ + create the AGGREGATOR graph @@ -230,7 +268,9 @@ { 'graphOutputPath' : 'aggregatorGraphPath', - 'isLookupUrl' : 'isLookUpUrl' + 'isLookupUrl' : 'isLookUpUrl', + 'reuseContent' : 'reuseContent', + 'contentPath' : 'contentPath' } @@ -241,8 +281,6 @@ 'postgresURL' : 'jdbc:postgresql://beta.services.openaire.eu:5432/dnet_openaireplus', 'postgresUser' : 'dnet', 'postgresPassword' : '', - 'reuseContent' : 'false', - 'contentPath' : '/tmp/beta_provision/aggregator', 'workingDir' : '/tmp/beta_provision/working_dir/aggregator' } @@ -252,6 +290,7 @@ + create the RAW graph @@ -289,6 +328,7 @@ + search for duplicates in the raw graph @@ -314,6 +354,7 @@ + create the INFERRED graph @@ -351,6 +392,7 @@ + mark duplicates as deleted and redistribute the relationships @@ -375,41 +417,6 @@ - - clean the properties in the graph typed as Qualifier according to the vocabulary indicated in schemeid - - executeOozieJob - IIS - - { - 'graphInputPath' : 'consistentGraphPath', - 'graphOutputPath': 'cleanedGraphPath', - 'isLookupUrl': 'isLookUpUrl' - } - - - { - 'oozie.wf.application.path' : '/lib/dnet/oa/graph/clean/oozie_app', - 'workingPath' : '/tmp/beta_provision/working_dir/clean' - } - - build-report - - - - - - - - Do we skip the graph enrichment steps? (Yes to prepare the graph for the IIS) - - NO - - - - - - propagates ORCID among results linked by allowedsemrels semantic relationships @@ -417,7 +424,7 @@ IIS { - 'sourcePath' : 'cleanedGraphPath', + 'sourcePath' : 'consistentGraphPath', 'outputPath': 'orcidGraphPath' } @@ -435,6 +442,7 @@ + mark results respecting some rules as belonging to communities @@ -460,6 +468,7 @@ + creates relashionships between results and organizations when the organizations are associated to institutional repositories @@ -484,6 +493,7 @@ + marks as belonging to communities the result collected from datasources related to the organizations specified in the organizationCommunityMap @@ -509,6 +519,7 @@ + created relation between projects and results linked to other results trough allowedsemrel semantic relations linked to projects @@ -534,6 +545,7 @@ + tag as belonging to communitites result in in allowedsemrels relation with other result already linked to communities @@ -560,6 +572,7 @@ + associated to results colleced from allowedtypes and those in the whithelist the country of the organization(s) handling the datasource it is collected from @@ -584,10 +597,36 @@ build-report + + + + + + + clean the properties in the graph typed as Qualifier according to the vocabulary indicated in schemeid + + executeOozieJob + IIS + + { + 'graphInputPath' : 'countryGraphPath', + 'graphOutputPath': 'cleanedGraphPath', + 'isLookupUrl': 'isLookUpUrl' + } + + + { + 'oozie.wf.application.path' : '/lib/dnet/oa/graph/clean/oozie_app', + 'workingPath' : '/tmp/beta_provision/working_dir/clean' + } + + build-report + + removes blacklisted relations @@ -595,7 +634,7 @@ IIS { - 'sourcePath' : 'countryGraphPath', + 'sourcePath' : 'cleanedGraphPath', 'outputPath': 'blacklistedGraphPath' } From 610d377d57f43613b4f3a9167a5a82b3825f0fe1 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 8 Jul 2020 16:54:26 +0200 Subject: [PATCH 020/100] first implementation of the BETA & PROD graphs merge procedure --- .../oa/graph/merge/MergeGraphSparkJob.java | 130 +++ .../graph/merge/oozie_app/config-default.xml | 18 + .../dhp/oa/graph/merge/oozie_app/workflow.xml | 282 +++++++ .../dhp/oa/graph/merge_graphs_parameters.json | 32 + .../wf/profiles/graph_beta_construction.xml | 779 ++++++++++++++++++ ...uction.xml => graph_core_construction.xml} | 2 +- 6 files changed, 1242 insertions(+), 1 deletion(-) create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJob.java create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge/oozie_app/workflow.xml create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge_graphs_parameters.json create mode 100644 dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_beta_construction.xml rename dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/{graph_construction.xml => graph_core_construction.xml} (99%) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJob.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJob.java new file mode 100644 index 000000000..d33b9e846 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJob.java @@ -0,0 +1,130 @@ +package eu.dnetlib.dhp.oa.graph.merge; + +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.oa.graph.clean.CleanGraphSparkJob; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.FilterFunction; +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 scala.Tuple2; + +import java.util.Objects; +import java.util.Optional; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +/** + * Combines the content from two aggregator graph tables of the same type, entities (or relationships) with the same ids + * are picked preferring those from the BETA aggregator rather then from PROD. The identity of a relationship is defined + * by eu.dnetlib.dhp.schema.common.ModelSupport#idFn() + */ +public class MergeGraphSparkJob { + + private static final Logger log = LoggerFactory.getLogger(CleanGraphSparkJob.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws Exception { + + String jsonConfiguration = IOUtils + .toString( + CleanGraphSparkJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/graph/merge_graphs_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 betaInputPath = parser.get("betaInputPath"); + log.info("betaInputPath: {}", betaInputPath); + + String prodInputPath = parser.get("prodInputPath"); + log.info("prodInputPath: {}", prodInputPath); + + String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + + String graphTableClassName = parser.get("graphTableClassName"); + log.info("graphTableClassName: {}", graphTableClassName); + + Class entityClazz = (Class) Class.forName(graphTableClassName); + + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); + + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + mergeGraphTable(spark, betaInputPath, prodInputPath, entityClazz, entityClazz, outputPath); + }); + } + + private static

void mergeGraphTable( + SparkSession spark, + String betaInputPath, + String prodInputPath, + Class

p_clazz, + Class b_clazz, + String outputPath) { + + Dataset> beta = readTableFromPath(spark, betaInputPath, b_clazz); + Dataset> prod = readTableFromPath(spark, prodInputPath, p_clazz); + + prod.joinWith(beta, prod.col("_1").equalTo(beta.col("_1")), "full_outer") + .map((MapFunction, Tuple2>, P>) value -> { + Optional

p = Optional.ofNullable(value._1()).map(Tuple2::_2); + Optional b = Optional.ofNullable(value._2()).map(Tuple2::_2); + if (p.isPresent() & !b.isPresent()) { + return p.get(); + } + if (b.isPresent()) { + return (P) b.get(); + } + return null; + }, Encoders.bean(p_clazz)) + .filter((FilterFunction

) Objects::nonNull) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath); + } + + private static Dataset> readTableFromPath( + SparkSession spark, String inputEntityPath, Class clazz) { + + log.info("Reading Graph table from: {}", inputEntityPath); + return spark + .read() + .textFile(inputEntityPath) + .map( + (MapFunction>) value -> { + final T t = OBJECT_MAPPER.readValue(value, clazz); + final String id = ModelSupport.idFn().apply(t); + return new Tuple2<>(id, t); + }, + Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz))); + } + + private static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } + +} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge/oozie_app/config-default.xml new file mode 100644 index 000000000..2e0ed9aee --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge/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-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge/oozie_app/workflow.xml new file mode 100644 index 000000000..0a512fb6a --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge/oozie_app/workflow.xml @@ -0,0 +1,282 @@ + + + + + betaInputGgraphPath + the beta graph root path + + + prodInputGgraphPath + the production graph root path + + + graphOutputPath + the output merged graph root path + + + + sparkDriverMemory + memory for driver process + + + sparkExecutorMemory + memory for individual executor + + + sparkExecutorCores + number of cores used by single executor + + + oozieActionShareLibForSpark2 + oozie action sharelib for spark 2.* + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + spark 2.* extra listeners classname + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + spark 2.* sql query execution listeners classname + + + spark2YarnHistoryServerAddress + spark 2.* yarn history server address + + + spark2EventLogDir + spark 2.* event log dir location + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + + + + + + + yarn + cluster + Merge publications + eu.dnetlib.dhp.oa.graph.merge.MergeGraphSparkJob + dhp-graph-mapper-${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.sql.shuffle.partitions=7680 + + --betaInputPath${betaInputGgraphPath}/publication + --prodInputPath${prodInputGgraphPath}/publication + --outputPath${graphOutputPath}/publication + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Publication + + + + + + + + yarn + cluster + Merge datasets + eu.dnetlib.dhp.oa.graph.merge.MergeGraphSparkJob + dhp-graph-mapper-${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.sql.shuffle.partitions=7680 + + --betaInputPath${betaInputGgraphPath}/dataset + --prodInputPath${prodInputGgraphPath}/dataset + --outputPath${graphOutputPath}/dataset + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Dataset + + + + + + + + yarn + cluster + Merge otherresearchproducts + eu.dnetlib.dhp.oa.graph.merge.MergeGraphSparkJob + dhp-graph-mapper-${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.sql.shuffle.partitions=7680 + + --betaInputPath${betaInputGgraphPath}/otherresearchproduct + --prodInputPath${prodInputGgraphPath}/otherresearchproduct + --outputPath${graphOutputPath}/otherresearchproduct + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct + + + + + + + + yarn + cluster + Merge softwares + eu.dnetlib.dhp.oa.graph.merge.MergeGraphSparkJob + dhp-graph-mapper-${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.sql.shuffle.partitions=7680 + + --betaInputPath${betaInputGgraphPath}/software + --prodInputPath${prodInputGgraphPath}/software + --outputPath${graphOutputPath}/software + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Software + + + + + + + + yarn + cluster + Merge datasources + eu.dnetlib.dhp.oa.graph.merge.MergeGraphSparkJob + dhp-graph-mapper-${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.sql.shuffle.partitions=7680 + + --betaInputPath${betaInputGgraphPath}/datasource + --prodInputPath${prodInputGgraphPath}/datasource + --outputPath${graphOutputPath}/datasource + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Datasource + + + + + + + + yarn + cluster + Merge organizations + eu.dnetlib.dhp.oa.graph.merge.MergeGraphSparkJob + dhp-graph-mapper-${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.sql.shuffle.partitions=7680 + + --betaInputPath${betaInputGgraphPath}/organization + --prodInputPath${prodInputGgraphPath}/organization + --outputPath${graphOutputPath}/organization + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Organization + + + + + + + + yarn + cluster + Merge projects + eu.dnetlib.dhp.oa.graph.merge.MergeGraphSparkJob + dhp-graph-mapper-${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.sql.shuffle.partitions=7680 + + --betaInputPath${betaInputGgraphPath}/project + --prodInputPath${prodInputGgraphPath}/project + --outputPath${graphOutputPath}/project + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Project + + + + + + + + yarn + cluster + Merge relations + eu.dnetlib.dhp.oa.graph.merge.MergeGraphSparkJob + dhp-graph-mapper-${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.sql.shuffle.partitions=7680 + + + --betaInputPath${betaInputGgraphPath}/relation + --prodInputPath${prodInputGgraphPath}/relation + --outputPath${graphOutputPath}/relation + --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Relation + + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge_graphs_parameters.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge_graphs_parameters.json new file mode 100644 index 000000000..6018b7e93 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge_graphs_parameters.json @@ -0,0 +1,32 @@ +[ + { + "paramName": "issm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "when true will stop SparkSession after job execution", + "paramRequired": false + }, + { + "paramName": "bin", + "paramLongName": "betaInputPath", + "paramDescription": "the beta graph root path", + "paramRequired": true + }, + { + "paramName": "pin", + "paramLongName": "prodInputPath", + "paramDescription": "the production graph root path", + "paramRequired": true + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the output merged graph root path", + "paramRequired": true + }, + { + "paramName": "class", + "paramLongName": "graphTableClassName", + "paramDescription": "class name moelling the graph table", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_beta_construction.xml b/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_beta_construction.xml new file mode 100644 index 000000000..08ed24cd0 --- /dev/null +++ b/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_beta_construction.xml @@ -0,0 +1,779 @@ + +

+ + + + + +
+ + Graph Construction [HYBRID] + Data Provision + 30 + + + + reuse cached content from the PROD aggregation system + + reuseProdContent + true + + + + + + + + set the PROD aggregator content path + + prodContentPath + /tmp/core_aggregator + + + + + + + + Set the path containing the PROD AGGREGATOR graph + + prodAggregatorGraphPath + /tmp/core_provision/graph/00_prod_graph_aggregator + + + + + + + + reuse cached content from the BETA aggregation system + + reuseBetaContent + true + + + + + + + + set the BETA aggregator content path + + betaContentPath + /tmp/beta_aggregator + + + + + + + + Set the path containing the BETA AGGREGATOR graph + + betaAggregatorGraphPath + /tmp/core_provision/graph/00_beta_graph_aggregator + + + + + + + + Set the IS lookup service address + + isLookUpUrl + http://services.openaire.eu:8280/is/services/isLookUp?wsdl + + + + + + + + Set the target path to store the MERGED graph + + mergedGraphPath + /tmp/core_provision/graph/01_graph_merged + + + + + + + + Set the target path to store the RAW graph + + rawGraphPath + /tmp/core_provision/graph/02_graph_raw + + + + + + + + Set the target path to store the DEDUPED graph + + dedupGraphPath + /tmp/core_provision/graph/03_graph_dedup + + + + + + + + Set the target path to store the INFERRED graph + + inferredGraphPath + /tmp/core_provision/graph/04_graph_inferred + + + + + + + + Set the target path to store the CONSISTENCY graph + + consistentGraphPath + /tmp/core_provision/graph/05_graph_consistent + + + + + + + + Set the target path to store the ORCID enriched graph + + orcidGraphPath + /tmp/core_provision/graph/06_graph_orcid + + + + + + + + Set the target path to store the BULK TAGGED graph + + bulkTaggingGraphPath + /tmp/core_provision/graph/07_graph_bulktagging + + + + + + + + Set the target path to store the AFFILIATION from INSTITUTIONAL REPOS graph + + affiliationGraphPath + /tmp/core_provision/graph/08_graph_affiliation + + + + + + + + Set the target path to store the COMMUNITY from SELECTED SOURCES graph + + communityOrganizationGraphPath + /tmp/core_provision/graph/09_graph_comunity_organization + + + + + + + + Set the target path to store the FUNDING from SEMANTIC RELATION graph + + fundingGraphPath + /tmp/core_provision/graph/10_graph_funding + + + + + + + + Set the target path to store the COMMUNITY from SEMANTIC RELATION graph + + communitySemRelGraphPath + /tmp/core_provision/graph/11_graph_comunity_sem_rel + + + + + + + + Set the target path to store the COUNTRY enriched graph + + countryGraphPath + /tmp/core_provision/graph/12_graph_country + + + + + + + + Set the target path to store the CLEANED graph + + cleanedGraphPath + /tmp/core_provision/graph/13_graph_cleaned + + + + + + + + Set the target path to store the blacklisted graph + + blacklistedGraphPath + /tmp/core_provision/graph/14_graph_blacklisted + + + + + + + + Set the map of paths for the Bulk Tagging + + bulkTaggingPathMap + {"author" : "$['author'][*]['fullname']", "title" : "$['title'][*]['value']", "orcid" : "$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']", "contributor" : "$['contributor'][*]['value']", "description" : "$['description'][*]['value']"} + + + + + + + + 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"]} + + + + + + + + + Set the dedup orchestrator name + + dedupConfig + decisiontree-dedup-test + + + + + + + + declares the ActionSet ids to promote in the RAW graph + + actionSetIdsRawGraph + scholexplorer-dump,gridac-dump,doiboost-organizations,doiboost,orcidworks-no-doi,iis-wos-entities,iis-entities-software,iis-entities-patent + + + + + + + + declares the ActionSet ids to promote in the INFERRED graph + + actionSetIdsIISGraph + iis-researchinitiative,iis-document-citations,iis-document-affiliation,iis-document-classes,iis-document-similarities,iis-referenced-datasets-main,iis-referenced-datasets-preprocessing,iis-referenced-projects-main,iis-referenced-projects-preprocessing,iis-referenceextraction-pdb,document_software_url,iis-extracted-metadata,iis-communities,iis-referenced-patents,iis-covid-19 + + + + + + + + wait configurations + + + + + + + + + + create the AGGREGATOR graph + + executeOozieJob + IIS + + { + 'graphOutputPath' : 'betaAggregatorGraphPath', + 'isLookupUrl' : 'isLookUpUrl', + 'reuseContent' : 'reuseBetaContent', + 'contentPath' : 'betaContentPath' + } + + + { + 'oozie.wf.application.path' : '/lib/dnet/oa/graph/raw_all/oozie_app', + 'mongoURL' : 'mongodb://beta.services.openaire.eu', + 'mongoDb' : 'mdstore', + 'postgresURL' : 'jdbc:postgresql://beta.services.openaire.eu:5432/dnet_openaireplus', + 'postgresUser' : 'dnet', + 'postgresPassword' : '', + 'workingDir' : '/tmp/core_provision/working_dir/beta_aggregator' + } + + build-report + + + + + + + + create the AGGREGATOR graph + + executeOozieJob + IIS + + { + 'graphOutputPath' : 'prodAggregatorGraphPath', + 'isLookupUrl' : 'isLookUpUrl', + 'reuseContent' : 'reuseProdContent', + 'contentPath' : 'prodContentPath' + } + + + { + 'oozie.wf.application.path' : '/lib/dnet/oa/graph/raw_all/oozie_app', + 'mongoURL' : 'mongodb://services.openaire.eu', + 'mongoDb' : 'mdstore', + 'postgresURL' : 'jdbc:postgresql://postgresql.services.openaire.eu:5432/dnet_openaireplus', + 'postgresUser' : 'dnet', + 'postgresPassword' : '', + 'workingDir' : '/tmp/core_provision/working_dir/prod_aggregator' + } + + build-report + + + + + + + + wait configurations + + + + + + + + create the AGGREGATOR graph + + executeOozieJob + IIS + + { + 'betaInputGgraphPath' : 'betaAggregatorGraphPath', + 'prodInputGgraphPath' : 'prodAggregatorGraphPath', + 'graphOutputPath' : 'mergedGraphPath' + } + + + { + 'oozie.wf.application.path' : '/lib/dnet/oa/graph/merge/oozie_app', + 'workingDir' : '/tmp/core_provision/working_dir/merge_graph' + } + + build-report + + + + + + + + create the RAW graph + + executeOozieJob + IIS + + { + 'inputActionSetIds' : 'actionSetIdsRawGraph', + 'inputGraphRootPath' : 'mergedGraphPath', + 'outputGraphRootPath' : 'rawGraphPath', + 'isLookupUrl' : 'isLookUpUrl' + } + + + { + 'oozie.wf.application.path' : '/lib/dnet/actionmanager/wf/main/oozie_app', + 'sparkExecutorCores' : '3', + 'sparkExecutorMemory' : '10G', + 'activePromoteDatasetActionPayload' : 'true', + 'activePromoteDatasourceActionPayload' : 'true', + 'activePromoteOrganizationActionPayload' : 'true', + 'activePromoteOtherResearchProductActionPayload' : 'true', + 'activePromoteProjectActionPayload' : 'true', + 'activePromotePublicationActionPayload' : 'true', + 'activePromoteRelationActionPayload' : 'true', + 'activePromoteResultActionPayload' : 'true', + 'activePromoteSoftwareActionPayload' : 'true', + 'mergeAndGetStrategy' : 'MERGE_FROM_AND_GET', + 'workingDir' : '/tmp/core_provision/working_dir/promoteActionsRaw' + } + + build-report + + + + + + + + search for duplicates in the raw graph + + executeOozieJob + IIS + + { + 'actionSetId' : 'dedupConfig', + 'graphBasePath' : 'rawGraphPath', + 'dedupGraphPath': 'dedupGraphPath', + 'isLookUpUrl' : 'isLookUpUrl' + } + + + { + 'oozie.wf.application.path' : '/lib/dnet/oa/dedup/scan/oozie_app', + 'workingPath' : '/tmp/core_provision/working_dir/dedup' + } + + build-report + + + + + + + + create the INFERRED graph + + executeOozieJob + IIS + + { + 'inputActionSetIds' : 'actionSetIdsIISGraph', + 'inputGraphRootPath' : 'dedupGraphPath', + 'outputGraphRootPath' : 'inferredGraphPath', + 'isLookupUrl' : 'isLookUpUrl' + } + + + { + 'oozie.wf.application.path' : '/lib/dnet/actionmanager/wf/main/oozie_app', + 'sparkExecutorCores' : '3', + 'sparkExecutorMemory' : '10G', + 'activePromoteDatasetActionPayload' : 'true', + 'activePromoteDatasourceActionPayload' : 'true', + 'activePromoteOrganizationActionPayload' : 'true', + 'activePromoteOtherResearchProductActionPayload' : 'true', + 'activePromoteProjectActionPayload' : 'true', + 'activePromotePublicationActionPayload' : 'true', + 'activePromoteRelationActionPayload' : 'true', + 'activePromoteResultActionPayload' : 'true', + 'activePromoteSoftwareActionPayload' : 'true', + 'mergeAndGetStrategy' : 'MERGE_FROM_AND_GET', + 'workingDir' : '/tmp/core_provision/working_dir/promoteActionsIIS' + } + + build-report + + + + + + + + mark duplicates as deleted and redistribute the relationships + + executeOozieJob + IIS + + { + 'graphBasePath' : 'inferredGraphPath', + 'dedupGraphPath': 'consistentGraphPath' + } + + + { + 'oozie.wf.application.path' : '/lib/dnet/oa/dedup/consistency/oozie_app', + 'workingPath' : '/tmp/core_provision/working_dir/dedup' + } + + build-report + + + + + + + + propagates ORCID among results linked by allowedsemrels semantic relationships + + executeOozieJob + IIS + + { + 'sourcePath' : 'consistentGraphPath', + 'outputPath': 'orcidGraphPath' + } + + + { + 'oozie.wf.application.path' : '/lib/dnet/oa/enrichment/orcidtoresultfromsemrel/oozie_app', + 'workingDir' : '/tmp/core_provision/working_dir/orcid', + 'allowedsemrels' : 'isSupplementedBy;isSupplementTo', + 'saveGraph' : 'true' + } + + build-report + + + + + + + + mark results respecting some rules as belonging to communities + + executeOozieJob + IIS + + { + 'sourcePath' : 'orcidGraphPath', + 'outputPath': 'bulkTaggingGraphPath', + 'isLookUpUrl' : 'isLookUpUrl', + 'pathMap' : 'bulkTaggingPathMap' + } + + + { + 'oozie.wf.application.path' : '/lib/dnet/oa/enrichment/bulktag/oozie_app', + 'workingDir' : '/tmp/core_provision/working_dir/bulktag' + } + + build-report + + + + + + + + creates relashionships between results and organizations when the organizations are associated to institutional repositories + + executeOozieJob + IIS + + { + 'sourcePath' : 'bulkTaggingGraphPath', + 'outputPath': 'affiliationGraphPath' + } + + + { + 'oozie.wf.application.path' : '/lib/dnet/oa/enrichment/affiliation/oozie_app', + 'workingDir' : '/tmp/core_provision/working_dir/affiliation', + 'saveGraph' : 'true' + } + + build-report + + + + + + + + marks as belonging to communities the result collected from datasources related to the organizations specified in the organizationCommunityMap + + executeOozieJob + IIS + + { + 'sourcePath' : 'affiliationGraphPath', + 'outputPath': 'communityOrganizationGraphPath', + 'organizationtoresultcommunitymap': 'propagationOrganizationCommunityMap' + } + + + { + 'oozie.wf.application.path' : '/lib/dnet/oa/enrichment/community_organization/oozie_app', + 'workingDir' : '/tmp/core_provision/working_dir/community_organization', + 'saveGraph' : 'true' + } + + build-report + + + + + + + + created relation between projects and results linked to other results trough allowedsemrel semantic relations linked to projects + + executeOozieJob + IIS + + { + 'sourcePath' : 'communityOrganizationGraphPath', + 'outputPath': 'fundingGraphPath' + } + + + { + 'oozie.wf.application.path' : '/lib/dnet/oa/enrichment/funding/oozie_app', + 'workingDir' : '/tmp/core_provision/working_dir/funding', + 'allowedsemrels' : 'isSupplementedBy;isSupplementTo', + 'saveGraph' : 'true' + } + + build-report + + + + + + + + tag as belonging to communitites result in in allowedsemrels relation with other result already linked to communities + + executeOozieJob + IIS + + { + 'sourcePath' : 'fundingGraphPath', + 'outputPath': 'communitySemRelGraphPath', + 'isLookUpUrl' : 'isLookUpUrl' + } + + + { + 'oozie.wf.application.path' : '/lib/dnet/oa/enrichment/community_semrel/oozie_app', + 'workingDir' : '/tmp/core_provision/working_dir/community_semrel', + 'allowedsemrels' : 'isSupplementedBy;isSupplementTo', + 'saveGraph' : 'true' + } + + build-report + + + + + + + + associated to results colleced from allowedtypes and those in the whithelist the country of the organization(s) handling the datasource it is collected from + + executeOozieJob + IIS + + { + 'sourcePath' : 'communitySemRelGraphPath', + 'outputPath': 'countryGraphPath' + } + + + { + 'oozie.wf.application.path' : '/lib/dnet/oa/enrichment/country/oozie_app', + 'sparkExecutorCores' : '3', + 'sparkExecutorMemory' : '10G', + 'workingDir' : '/tmp/core_provision/working_dir/country', + 'allowedtypes' : 'pubsrepository::institutional', + 'whitelist' : '10|opendoar____::300891a62162b960cf02ce3827bb363c', + 'saveGraph' : 'true' + } + + build-report + + + + + + + + clean the properties in the graph typed as Qualifier according to the vocabulary indicated in schemeid + + executeOozieJob + IIS + + { + 'graphInputPath' : 'countryGraphPath', + 'graphOutputPath': 'cleanedGraphPath', + 'isLookupUrl': 'isLookUpUrl' + } + + + { + 'oozie.wf.application.path' : '/lib/dnet/oa/graph/clean/oozie_app', + 'workingPath' : '/tmp/core_provision/working_dir/clean' + } + + build-report + + + + + + + + removes blacklisted relations + + executeOozieJob + IIS + + { + 'sourcePath' : 'cleanedGraphPath', + 'outputPath': 'blacklistedGraphPath' + } + + + { + 'oozie.wf.application.path' : '/lib/dnet/oa/enrichment/blacklist/oozie_app', + 'workingDir' : '/tmp/core_provision/working_dir/blacklist', + 'postgresURL' : 'jdbc:postgresql://beta.services.openaire.eu:5432/dnet_openaireplus', + 'postgresUser' : 'dnet', + 'postgresPassword' : '' + } + + build-report + + + + + + + + + wf_20200615_163630_609 + 2020-06-15T17:08:00+00:00 + SUCCESS + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_construction.xml b/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_core_construction.xml similarity index 99% rename from dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_construction.xml rename to dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_core_construction.xml index 4d77883b4..3bce81c50 100644 --- a/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_construction.xml +++ b/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_core_construction.xml @@ -413,7 +413,7 @@ build-report
- +
From 853e8d79875ca0d713f4c64e8b589a18e8ca2fd5 Mon Sep 17 00:00:00 2001 From: Alessia Bardi Date: Wed, 8 Jul 2020 17:03:53 +0200 Subject: [PATCH 021/100] test for software merge --- .../java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java | 10 ++++++++++ .../eu/dnetlib/dhp/dedup/json/software_merge.json | 3 +++ 2 files changed, 13 insertions(+) create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/json/software_merge.json diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java index b8ccb038d..513e14f07 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java @@ -45,6 +45,16 @@ public class EntityMergerTest implements Serializable { } + @Test + public void softwareMergerTest() throws InstantiationException, IllegalAccessException { + List> softwares = readSample(testEntityBasePath + "/software_merge.json", Software.class); + + Software merged = DedupRecordFactory + .entityMerger(dedupId, softwares.iterator(), 0, dataInfo, Software.class); + + System.out.println(merged.getBestaccessright().getClassid()); + } + @Test public void publicationMergerTest() throws InstantiationException, IllegalAccessException { diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/json/software_merge.json b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/json/software_merge.json new file mode 100644 index 000000000..b146d6102 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/json/software_merge.json @@ -0,0 +1,3 @@ +{"context": [], "dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:datasetarchive", "classname": "sysimport:crosswalk:datasetarchive", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "dedup-similarity-result-levenstein", "invisible": false, "trust": "0.95"}, "resourcetype": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "pid": [], "contributor": [], "resulttype": {"classid": "software", "classname": "software", "schemename": "dnet:result_typologies", "schemeid": "dnet:result_typologies"}, "relevantdate": [], "collectedfrom": [{"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": "Journal.fi", "key": "10|openaire____::6eef8049d0feedc089ee009abca55e35"}], "id": "50|a89337edbe55::4930db9e954866d70916cbfba9f81f97", "subject": [], "instance": [{"refereed": null, "hostedby": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": "Journal.fi", "key": "10|openaire____::6eef8049d0feedc089ee009abca55e35"}, "processingchargeamount": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": ""}, "license": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": ""}, "url": [], "distributionlocation": "", "processingchargecurrency": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": ""}, "dateofacceptance": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": "2016-01-01"}, "collectedfrom": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": "Journal.fi", "key": "10|openaire____::6eef8049d0feedc089ee009abca55e35"}, "accessright": {"classid": "OPEN", "classname": "Open Access", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "instancetype": {"classid": "0001", "classname": "Article", "schemename": "dnet:dataCite_resource", "schemeid": "dnet:dataCite_resource"}}], "embargoenddate": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": ""}, "lastupdatetimestamp": 0, "author": [{"surname": "Go\\u0308tz", "name": "Julia", "pid": [], "rank": 5, "affiliation": [], "fullname": "G\\u00f6tz, Julia"}, {"surname": "Wolff", "name": "Stephan", "pid": [], "rank": 6, "affiliation": [], "fullname": "Wolff, Stephan"}, {"surname": "Jansen", "name": "Olav", "pid": [], "rank": 7, "affiliation": [], "fullname": "Jansen, Olav"}, {"surname": "Dressler", "name": "Dirk", "pid": [{"qualifier": {"classid": "ORCID", "classname": "ORCID"}, "value": "0000-0000-0656-9999"},{"qualifier": {"classid": "id", "classname": "id"}, "value": "987654321"}], "rank": 8, "affiliation": [], "fullname": "Dressler, Dirk"}, {"surname": "Schneider", "name": "Susanne A.", "pid": [], "rank": 9, "affiliation": [], "fullname": "Schneider, Susanne A."}], "source": [], "dateofcollection": "2019-11-05T14:49:22.351Z", "fulltext": [], "dateoftransformation": "2019-11-05T16:10:58.988Z", "description": [], "format": [], "coverage": [], "publisher": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": ""}, "language": {"classid": "eng", "classname": "English", "schemename": "dnet:languages", "schemeid": "dnet:languages"}, "bestaccessright": {"classid": "OPEN SOURCE", "classname": "Open Source", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "country": [], "extraInfo": [], "originalId": [], "dateofacceptance": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": "2018-09-30"}, "title": [{"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "qualifier": {"classid": "main title", "classname": "main title", "schemename": "dnet:dataCite_title", "schemeid": "dnet:dataCite_title"}, "value": "Altered brain activation in a reversal learning task unmasks adaptive changes in cognitive control in writer's cramp"}]} +{"context": [], "dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:repository", "classname": "sysimport:crosswalk:repository", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "dedup-similarity-result-levenstein", "invisible": false, "trust": "0.9"}, "resourcetype": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "pid": [{"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "qualifier": {"classid": "doi", "classname": "doi", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "10.1016/j.nicl.2015.11.006"}], "contributor": [], "resulttype": {"classid": "software", "classname": "software", "schemename": "dnet:result_typologies", "schemeid": "dnet:result_typologies"}, "relevantdate": [], "collectedfrom": [{"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": "BASE (Open Access Aggregator)", "key": "10|openaire____::df45502607927471ecf8a6ae83683ff5"}], "id": "50|base_oa_____::0968af610a356656706657e4f234b340", "subject": [], "instance": [{"refereed": null, "hostedby": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": "NeuroImage: Clinical", "key": "10|doajarticles::0c0e74daa5d95504eade9c81ebbd5b8a"}, "processingchargeamount": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": ""}, "license": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": "http://creativecommons.org/licenses/by-nc-nd/4.0/"}, "url": ["http://dx.doi.org/10.1016/j.nicl.2015.11.006"], "distributionlocation": "", "processingchargecurrency": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": ""}, "dateofacceptance": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": "2016-01-01"}, "collectedfrom": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": "BASE (Open Access Aggregator)", "key": "10|openaire____::df45502607927471ecf8a6ae83683ff5"}, "accessright": {"classid": "OPEN", "classname": "Open Access", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "instancetype": {"classid": "0001", "classname": "Article", "schemename": "dnet:publication_resource", "schemeid": "dnet:publication_resource"}}], "embargoenddate": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": ""}, "lastupdatetimestamp": 0, "author": [{"surname": "Zeuner", "name": "Kirsten E.", "pid": [], "rank": 1, "affiliation": [], "fullname": "Zeuner, Kirsten E."}, {"surname": "Knutzen", "name": "Arne", "pid": [], "rank": 2, "affiliation": [], "fullname": "Knutzen, Arne"}, {"surname": "Granert", "name": "Oliver", "pid": [{"qualifier": {"classid": "ORCID", "classname": "ORCID"}, "value": "0000-0002-0656-1023"}], "rank": 3, "affiliation": [], "fullname": "Granert, Oliver"}, {"surname": "Sablowsky", "name": "Simone", "pid": [], "rank": 4, "affiliation": [], "fullname": "Sablowsky, Simone"}, {"surname": "Go\\u0308tz", "name": "Julia", "pid": [], "rank": 5, "affiliation": [], "fullname": "G\\u00f6tz, Julia"}, {"surname": "Wolff", "name": "Stephan", "pid": [], "rank": 6, "affiliation": [], "fullname": "Wolff, Stephan"}, {"surname": "Jansen", "name": "Olav", "pid": [], "rank": 7, "affiliation": [], "fullname": "Jansen, Olav"}, {"surname": "Dressler", "name": "Dirk", "pid": [], "rank": 8, "affiliation": [], "fullname": "Dressler, Dirk"}, {"surname": "Schneider", "name": "Susanne A.", "pid": [], "rank": 9, "affiliation": [], "fullname": "Schneider, Susanne A."}, {"surname": "Klein", "name": "Christine", "pid": [], "rank": 10, "affiliation": [], "fullname": "Klein, Christine"}, {"surname": "Deuschl", "name": "Gu\\u0308nther", "pid": [], "rank": 11, "affiliation": [], "fullname": "Deuschl, G\\u00fcnther"}, {"surname": "Eimeren", "name": "Thilo", "pid": [], "rank": 12, "affiliation": [], "fullname": "van Eimeren, Thilo"}, {"surname": "Witt", "name": "Karsten", "pid": [], "rank": 13, "affiliation": [], "fullname": "Witt, Karsten"}], "source": [], "dateofcollection": "2017-07-27T19:04:09.131Z", "fulltext": [], "dateoftransformation": "2019-01-23T10:15:19.582Z", "description": [], "format": [], "coverage": [], "publisher": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": "Elsevier BV"}, "language": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "bestaccessright": {"classid": "OPEN SOURCE", "classname": "Open Source", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "country": [{"classid": "IT", "classname": "Italy", "schemeid": "dnet:countries", "schemename": "dnet:countries"}], "extraInfo": [], "originalId": ["10.1016/j.nicl.2015.11.006"], "dateofacceptance": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": "2016-01-01"}, "title": [{"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "qualifier": {"classid": "main title", "classname": "main title", "schemename": "dnet:dataCite_title", "schemeid": "dnet:dataCite_title"}, "value": "Altered brain activation in a reversal learning task unmasks adaptive changes in cognitive control in writer's cramp"}]} +{"context": [], "dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:datasetarchive", "classname": "sysimport:crosswalk:datasetarchive", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": true, "inferenceprovenance": "dedup-similarity-result-levenstein", "invisible": false, "trust": "0.9"}, "resourcetype": {"classid": "0004", "classname": "Conference object", "schemename": "dnet:dataCite_resource", "schemeid": "dnet:dataCite_resource"}, "pid": [], "contributor": [], "resulttype": {"classid": "software", "classname": "software", "schemename": "dnet:result_typologies", "schemeid": "dnet:result_typologies"}, "relevantdate": [], "collectedfrom": [{"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": "CRIS UNS (Current Research Information System University of Novi Sad)", "key": "10|CRIS_UNS____::f66f1bd369679b5b077dcdf006089556"}], "id": "50|CrisUnsNoviS::9f9d014eea45dab432cab636c4c9cf39", "subject": [], "instance": [{"refereed": null, "hostedby": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": "CRIS UNS (Current Research Information System University of Novi Sad)", "key": "10|CRIS_UNS____::f66f1bd369679b5b077dcdf006089556"}, "processingchargeamount": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": ""}, "license": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": ""}, "url": ["https://www.cris.uns.ac.rs/record.jsf?recordId=113444&source=OpenAIRE&language=en"], "distributionlocation": "", "processingchargecurrency": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": ""}, "dateofacceptance": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": "2019-01-01"}, "collectedfrom": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": "CRIS UNS (Current Research Information System University of Novi Sad)", "key": "10|CRIS_UNS____::f66f1bd369679b5b077dcdf006089556"}, "accessright": {"classid": "UNKNOWN", "classname": "UNKNOWN", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "instancetype": {"classid": "0004", "classname": "Conference object", "schemename": "dnet:dataCite_resource", "schemeid": "dnet:dataCite_resource"}}], "embargoenddate": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": ""}, "lastupdatetimestamp": 0, "author": [{"surname": "Zeuner", "name": "Kirsten E.", "pid": [], "rank": 1, "affiliation": [], "fullname": "Zeuner, Kirsten E."}, {"surname": "Knutzen", "name": "Arne", "pid": [], "rank": 2, "affiliation": [], "fullname": "Knutzen, Arne"}, {"surname": "Granert", "name": "Oliver", "pid": [{"qualifier": {"classid": "ORCID", "classname": "ORCID"}, "value": "0000-0002-0656-1023"}, {"qualifier": {"classid": "pubmed", "classname": "pubmed"}, "value": "pubmed.it"}], "rank": 3, "affiliation": [], "fullname": "Granert, Oliver"}, {"surname": "Sablowsky", "name": "Simone", "pid": [{"qualifier": {"classid": "id", "classname": "id"}, "value": "12345678"}], "rank": 4, "affiliation": [], "fullname": "Sablowsky, Simone"}, {"surname": "Go\\u0308tz", "name": "Julia", "pid": [], "rank": 5, "affiliation": [], "fullname": "G\\u00f6tz, Julia"}, {"surname": "Wolff", "name": "Stephan", "pid": [], "rank": 6, "affiliation": [], "fullname": "Wolff, Stephan"}, {"surname": "Jansen", "name": "Olav", "pid": [{"qualifier": {"classid": "ORCID", "classname": "ORCID"}, "value": "0000-0000-0656-1023"},{"qualifier": {"classid": "id", "classname": "id"}, "value": "987654321"}], "rank": 7, "affiliation": [], "fullname": "Jansen, Olav"}, {"surname": "Dressler", "name": "Dirk", "pid": [], "rank": 8, "affiliation": [], "fullname": "Dressler, Dirk"}, {"surname": "Schneider", "name": "Susanne A.", "pid": [], "rank": 9, "affiliation": [], "fullname": "Schneider, Susanne A."}], "source": [], "dateofcollection": "2020-03-10T15:05:38.685Z", "fulltext": [], "dateoftransformation": "2020-03-11T20:11:13.15Z", "description": [], "format": [], "coverage": [], "publisher": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": ""}, "language": {"classid": "en", "classname": "en", "schemename": "dnet:languages", "schemeid": "dnet:languages"}, "bestaccessright": {"classid": "UNKNOWN", "classname": "unknown", "schemename": "dnet:access_modes", "schemeid": "dnet:access_modes"}, "country": [{"classid": "FI", "classname": "Finland", "schemeid": "dnet:countries", "schemename": "dnet:countries"}], "extraInfo": [], "originalId": ["(BISIS)113444", "https://www.cris.uns.ac.rs/record.jsf?recordId=113444&source=OpenAIRE&language=en"], "dateofacceptance": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "value": "2016-01-01"}, "title": [{"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "", "classname": "", "schemename": "", "schemeid": ""}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": ""}, "qualifier": {"classid": "test title", "classname": "test title", "schemename": "dnet:dataCite_title", "schemeid": "dnet:dataCite_title"}, "value": "Antichains of copies of ultrahomogeneous structures"}]} \ No newline at end of file From 67e1d222b6768a2f2168f94621a482e4029bb05b Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 8 Jul 2020 17:53:35 +0200 Subject: [PATCH 022/100] bulk cleaning when found null or empty, sets bestaccessrights evaluating the result instances --- .../dhp/oa/graph/clean/CleanGraphSparkJob.java | 17 +++++++++++------ .../graph/raw/AbstractMdRecordToOafMapper.java | 4 ++++ .../oa/graph/clean/CleaningFunctionTest.java | 5 +++++ .../eu/dnetlib/dhp/oa/graph/clean/result.json | 7 +------ 4 files changed, 21 insertions(+), 12 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleanGraphSparkJob.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleanGraphSparkJob.java index e1c4b53b5..fd707e949 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleanGraphSparkJob.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleanGraphSparkJob.java @@ -8,6 +8,7 @@ import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; +import eu.dnetlib.dhp.oa.graph.raw.AbstractMdRecordToOafMapper; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.spark.SparkConf; @@ -97,7 +98,7 @@ public class CleanGraphSparkJob { .json(outputPath); } - private static T fixDefaults(T value) { + protected static T fixDefaults(T value) { if (value instanceof Datasource) { // nothing to clean here } else if (value instanceof Project) { @@ -134,11 +135,6 @@ public class CleanGraphSparkJob { .setResourcetype( qualifier("UNKNOWN", "Unknown", ModelConstants.DNET_DATA_CITE_RESOURCE)); } - if (Objects.isNull(r.getBestaccessright()) || StringUtils.isBlank(r.getBestaccessright().getClassid())) { - r - .setBestaccessright( - qualifier("UNKNOWN", "not available", ModelConstants.DNET_ACCESS_MODES)); - } if (Objects.nonNull(r.getInstance())) { for (Instance i : r.getInstance()) { if (Objects.isNull(i.getAccessright()) || StringUtils.isBlank(i.getAccessright().getClassid())) { @@ -152,6 +148,15 @@ public class CleanGraphSparkJob { } } } + if (Objects.isNull(r.getBestaccessright()) || StringUtils.isBlank(r.getBestaccessright().getClassid())) { + Qualifier bestaccessrights = AbstractMdRecordToOafMapper.createBestAccessRights(r.getInstance()); + if (Objects.isNull(bestaccessrights)) { + r.setBestaccessright( + qualifier("UNKNOWN", "not available", ModelConstants.DNET_ACCESS_MODES)); + } else { + r.setBestaccessright(bestaccessrights); + } + } if (Objects.nonNull(r.getAuthor())) { boolean nullRank = r .getAuthor() 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 fc77950d0..c43ee29fe 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 @@ -378,6 +378,10 @@ public abstract class AbstractMdRecordToOafMapper { protected abstract Field prepareDatasetStorageDate(Document doc, DataInfo info); + public static Qualifier createBestAccessRights(final List instanceList) { + return getBestAccessRights(instanceList); + } + protected static Qualifier getBestAccessRights(final List instanceList) { if (instanceList != null) { final Optional min = instanceList diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/clean/CleaningFunctionTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/clean/CleaningFunctionTest.java index 4783aa81f..559a30b1e 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/clean/CleaningFunctionTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/clean/CleaningFunctionTest.java @@ -57,6 +57,8 @@ public class CleaningFunctionTest { String json = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/result.json")); Publication p_in = MAPPER.readValue(json, Publication.class); + assertNull(p_in.getBestaccessright()); + assertTrue(p_in instanceof Result); assertTrue(p_in instanceof Publication); @@ -84,6 +86,9 @@ public class CleaningFunctionTest { .map(p -> p.getQualifier()) .allMatch(q -> pidTerms.contains(q.getClassid()))); + Publication p_defaults = CleanGraphSparkJob.fixDefaults(p_out); + assertEquals("CLOSED", p_defaults.getBestaccessright().getClassid()); + // TODO add more assertions to verity the cleaned values System.out.println(MAPPER.writeValueAsString(p_out)); diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/result.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/result.json index 2c1d5017d..5d0c0d1ed 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/result.json +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/result.json @@ -185,12 +185,7 @@ "surname": "" } ], - "bestaccessright": { - "classid": "CLOSED", - "classname": "Closed Access", - "schemeid": "dnet:access_modes", - "schemename": "dnet:access_modes" - }, + "bestaccessright": null, "collectedfrom": [ { "key": "10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747", From 4365cf41d70cd1a16b359340a4b2da2537237543 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 8 Jul 2020 22:31:46 +0200 Subject: [PATCH 023/100] trying to overcome OOM errors during duplicate scan phase --- .../dhp/oa/dedup/SparkCreateMergeRels.java | 7 +++++-- .../dhp/oa/dedup/SparkCreateSimRels.java | 20 +++++++++++-------- 2 files changed, 17 insertions(+), 10 deletions(-) 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 c0503d991..19e60b520 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 @@ -10,6 +10,7 @@ import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.api.java.function.PairFunction; import org.apache.spark.graphx.Edge; import org.apache.spark.rdd.RDD; @@ -100,8 +101,10 @@ public class SparkCreateMergeRels extends AbstractSparkAction { final RDD> edgeRdd = spark .read() - .load(DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity)) - .as(Encoders.bean(Relation.class)) + .textFile(DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity)) + .map( + (MapFunction) r -> OBJECT_MAPPER.readValue(r, Relation.class), + Encoders.bean(Relation.class)) .javaRDD() .map(it -> new Edge<>(hash(it.getSource()), hash(it.getTarget()), it.getRelClass())) .rdd(); 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 2cfe2e080..7bc77fe2b 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 @@ -95,19 +95,23 @@ public class SparkCreateSimRels extends AbstractSparkAction { }); // create blocks for deduplication - JavaPairRDD blocks = Deduper.createSortedBlocks(mapDocuments, dedupConf); + JavaPairRDD blocks = Deduper + .createSortedBlocks(mapDocuments, dedupConf) + .repartition(10000); // create relations by comparing only elements in the same group - JavaRDD relations = Deduper + Deduper .computeRelations(sc, blocks, dedupConf) - .map(t -> createSimRel(t._1(), t._2(), entity)); + .map(t -> createSimRel(t._1(), t._2(), entity)) + .repartition(10000) + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(outputPath); // save the simrel in the workingdir - spark - .createDataset(relations.rdd(), Encoders.bean(Relation.class)) - .write() - .mode(SaveMode.Append) - .save(outputPath); + /* + * spark .createDataset(relations.rdd(), Encoders.bean(Relation.class)) .write() .mode(SaveMode.Append) + * .save(outputPath); + */ } } From 3c728aaa0c1c993b172bac1c792ff45a0bac75ea Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 8 Jul 2020 22:39:51 +0200 Subject: [PATCH 024/100] trying to overcome OOM errors during duplicate scan phase --- .../main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateSimRels.java | 1 + 1 file changed, 1 insertion(+) 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 7bc77fe2b..1be2b9e31 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 @@ -88,6 +88,7 @@ public class SparkCreateSimRels extends AbstractSparkAction { JavaPairRDD mapDocuments = sc .textFile(DedupUtility.createEntityPath(graphBasePath, subEntity)) + .repartition(10000) .mapToPair( (PairFunction) s -> { MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s); From 1c6a1716332e02c49f1b05639803e9c3f8c83d00 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Thu, 9 Jul 2020 11:02:09 +0200 Subject: [PATCH 025/100] updated pom --- dhp-workflows/dhp-broker-events/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-broker-events/pom.xml b/dhp-workflows/dhp-broker-events/pom.xml index 119031b06..1a219c5c9 100644 --- a/dhp-workflows/dhp-broker-events/pom.xml +++ b/dhp-workflows/dhp-broker-events/pom.xml @@ -57,9 +57,9 @@
- eu.dnetlib + eu.dnetlib.dhp dnet-openaire-broker-common - [3.0.4,4.0.0) + [3.0.0-SNAPSHOT,) From 2d742a84aeb699aa67c889f7ed7ffe99c137fdae Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Thu, 9 Jul 2020 12:53:46 +0200 Subject: [PATCH 026/100] DedupConfig as json file --- .../dhp/broker/oa/GenerateEventsJob.java | 36 +----- .../dhp/broker/oa/matchers/UpdateMatcher.java | 4 +- .../dhp/broker/oa/util/EventFinder.java | 4 +- .../dhp/broker/oa/util/TrustUtils.java | 54 ++++++++ .../dhp/broker/oa/util/UpdateInfo.java | 41 +----- .../broker/oa/dedupConfig/dedupConfig.json | 122 ++++++++++++++++++ .../oa/generate_all/oozie_app/workflow.xml | 10 -- .../dhp/broker/oa/generate_events.json | 12 -- .../broker/oa/partial/oozie_app/workflow.xml | 9 -- .../broker/oa/matchers/UpdateMatcherTest.java | 12 +- .../dhp/broker/oa/util/TrustUtilsTest.java | 21 +++ 11 files changed, 208 insertions(+), 117 deletions(-) create mode 100644 dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/dedupConfig/dedupConfig.json diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java index a2d92e149..cfee360c5 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateEventsJob.java @@ -18,8 +18,6 @@ import org.apache.spark.util.LongAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.fasterxml.jackson.databind.ObjectMapper; - import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.broker.model.Event; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; @@ -27,9 +25,6 @@ import eu.dnetlib.dhp.broker.oa.util.ClusterUtils; import eu.dnetlib.dhp.broker.oa.util.EventFinder; import eu.dnetlib.dhp.broker.oa.util.EventGroup; import eu.dnetlib.dhp.broker.oa.util.aggregators.simple.ResultGroup; -import eu.dnetlib.dhp.utils.ISLookupClientFactory; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -import eu.dnetlib.pace.config.DedupConfig; public class GenerateEventsJob { @@ -52,12 +47,6 @@ public class GenerateEventsJob { final String workingPath = parser.get("workingPath"); log.info("workingPath: {}", workingPath); - final String isLookupUrl = parser.get("isLookupUrl"); - log.info("isLookupUrl: {}", isLookupUrl); - - final String dedupConfigProfileId = parser.get("dedupConfProfile"); - log.info("dedupConfigProfileId: {}", dedupConfigProfileId); - final String eventsPath = workingPath + "/events"; log.info("eventsPath: {}", eventsPath); @@ -72,10 +61,6 @@ public class GenerateEventsJob { final SparkConf conf = new SparkConf(); - // TODO UNCOMMENT - // final DedupConfig dedupConfig = loadDedupConfig(isLookupUrl, dedupConfigProfileId); - final DedupConfig dedupConfig = null; - runWithSparkSession(conf, isSparkSessionManaged, spark -> { ClusterUtils.removeDir(spark, eventsPath); @@ -90,7 +75,7 @@ public class GenerateEventsJob { final Dataset dataset = groups .map( g -> EventFinder - .generateEvents(g, dsIdWhitelist, dsIdBlacklist, dsTypeWhitelist, dedupConfig, accumulators), + .generateEvents(g, dsIdWhitelist, dsIdBlacklist, dsTypeWhitelist, accumulators), Encoders .bean(EventGroup.class)) .flatMap(g -> g.getData().iterator(), Encoders.bean(Event.class)); @@ -112,23 +97,4 @@ public class GenerateEventsJob { } - private static DedupConfig loadDedupConfig(final String isLookupUrl, final String profId) throws Exception { - - final ISLookUpService isLookUpService = ISLookupClientFactory.getLookUpService(isLookupUrl); - - final String conf = isLookUpService - .getResourceProfileByQuery( - String - .format( - "for $x in /RESOURCE_PROFILE[.//RESOURCE_IDENTIFIER/@value = '%s'] return $x//DEDUPLICATION/text()", - profId)); - - final DedupConfig dedupConfig = new ObjectMapper().readValue(conf, DedupConfig.class); - dedupConfig.getPace().initModel(); - dedupConfig.getPace().initTranslationMap(); - // dedupConfig.getWf().setConfigurationId("???"); - - return dedupConfig; - } - } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcher.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcher.java index af6ab30a1..3d688fa1d 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcher.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcher.java @@ -17,7 +17,6 @@ import org.apache.spark.util.LongAccumulator; import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.oa.util.UpdateInfo; -import eu.dnetlib.pace.config.DedupConfig; public abstract class UpdateMatcher { @@ -37,7 +36,6 @@ public abstract class UpdateMatcher { public Collection> searchUpdatesForRecord(final OaBrokerMainEntity res, final Collection others, - final DedupConfig dedupConfig, final Map accumulators) { final Map> infoMap = new HashMap<>(); @@ -49,7 +47,7 @@ public abstract class UpdateMatcher { if (topic != null) { final UpdateInfo info = new UpdateInfo<>(topic, hl, source, res, getCompileHighlightFunction(), - getHighlightToStringFunction(), dedupConfig); + getHighlightToStringFunction()); final String s = DigestUtils.md5Hex(info.getHighlightValueAsString()); if (!infoMap.containsKey(s) || infoMap.get(s).getTrust() < info.getTrust()) { diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java index 593e66d43..b6328eb95 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java @@ -37,7 +37,6 @@ import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMoreOpenAccess; import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMorePid; import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMoreSubject; import eu.dnetlib.dhp.broker.oa.util.aggregators.simple.ResultGroup; -import eu.dnetlib.pace.config.DedupConfig; public class EventFinder { @@ -76,7 +75,6 @@ public class EventFinder { final Set dsIdWhitelist, final Set dsIdBlacklist, final Set dsTypeWhitelist, - final DedupConfig dedupConfig, final Map accumulators) { final List> list = new ArrayList<>(); @@ -84,7 +82,7 @@ public class EventFinder { for (final OaBrokerMainEntity target : results.getData()) { if (verifyTarget(target, dsIdWhitelist, dsIdBlacklist, dsTypeWhitelist)) { for (final UpdateMatcher matcher : matchers) { - list.addAll(matcher.searchUpdatesForRecord(target, results.getData(), dedupConfig, accumulators)); + list.addAll(matcher.searchUpdatesForRecord(target, results.getData(), accumulators)); } } } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/TrustUtils.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/TrustUtils.java index 5338d4f3d..72fe1b204 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/TrustUtils.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/TrustUtils.java @@ -1,8 +1,62 @@ package eu.dnetlib.dhp.broker.oa.util; +import java.io.IOException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.broker.objects.OaBrokerMainEntity; +import eu.dnetlib.pace.config.DedupConfig; +import eu.dnetlib.pace.model.MapDocument; +import eu.dnetlib.pace.tree.support.TreeProcessor; +import eu.dnetlib.pace.util.MapDocumentUtil; + public class TrustUtils { + private static final Logger log = LoggerFactory.getLogger(TrustUtils.class); + + private static DedupConfig dedupConfig; + + static { + final ObjectMapper mapper = new ObjectMapper(); + try { + dedupConfig = mapper + .readValue( + DedupConfig.class.getResourceAsStream("/eu/dnetlib/dhp/broker/oa/dedupConfig/dedupConfig.json"), + DedupConfig.class); + } catch (final IOException e) { + log.error("Error loading dedupConfig, e"); + } + + } + + protected static float calculateTrust(final OaBrokerMainEntity r1, final OaBrokerMainEntity r2) { + + if (dedupConfig == null) { + return BrokerConstants.MIN_TRUST; + } + + try { + final ObjectMapper objectMapper = new ObjectMapper(); + final MapDocument doc1 = MapDocumentUtil + .asMapDocumentWithJPath(dedupConfig, objectMapper.writeValueAsString(r1)); + final MapDocument doc2 = MapDocumentUtil + .asMapDocumentWithJPath(dedupConfig, objectMapper.writeValueAsString(r2)); + + final double score = new TreeProcessor(dedupConfig).computeScore(doc1, doc2); + + final double threshold = dedupConfig.getWf().getThreshold(); + + return TrustUtils.rescale(score, threshold); + } catch (final Exception e) { + log.error("Error computing score between results", e); + return BrokerConstants.MIN_TRUST; + } + } + public static float rescale(final double score, final double threshold) { if (score >= BrokerConstants.MAX_TRUST) { return BrokerConstants.MAX_TRUST; diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/UpdateInfo.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/UpdateInfo.java index 0586b681e..ef8fb240c 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/UpdateInfo.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/UpdateInfo.java @@ -4,20 +4,11 @@ package eu.dnetlib.dhp.broker.oa.util; import java.util.function.BiConsumer; import java.util.function.Function; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.fasterxml.jackson.databind.ObjectMapper; - import eu.dnetlib.broker.objects.OaBrokerEventPayload; import eu.dnetlib.broker.objects.OaBrokerInstance; import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.broker.objects.OaBrokerProvenance; import eu.dnetlib.dhp.broker.model.Topic; -import eu.dnetlib.pace.config.DedupConfig; -import eu.dnetlib.pace.model.MapDocument; -import eu.dnetlib.pace.tree.support.TreeProcessor; -import eu.dnetlib.pace.util.MapDocumentUtil; public final class UpdateInfo { @@ -35,20 +26,17 @@ public final class UpdateInfo { private final float trust; - private static final Logger log = LoggerFactory.getLogger(UpdateInfo.class); - public UpdateInfo(final Topic topic, final T highlightValue, final OaBrokerMainEntity source, final OaBrokerMainEntity target, final BiConsumer compileHighlight, - final Function highlightToString, - final DedupConfig dedupConfig) { + final Function highlightToString) { this.topic = topic; this.highlightValue = highlightValue; this.source = source; this.target = target; this.compileHighlight = compileHighlight; this.highlightToString = highlightToString; - this.trust = calculateTrust(dedupConfig, source, target); + this.trust = TrustUtils.calculateTrust(source, target); } public T getHighlightValue() { @@ -63,31 +51,6 @@ public final class UpdateInfo { return target; } - private float calculateTrust(final DedupConfig dedupConfig, - final OaBrokerMainEntity r1, - final OaBrokerMainEntity r2) { - - if (dedupConfig == null) { - return BrokerConstants.MIN_TRUST; - } - - try { - final ObjectMapper objectMapper = new ObjectMapper(); - final MapDocument doc1 = MapDocumentUtil - .asMapDocumentWithJPath(dedupConfig, objectMapper.writeValueAsString(r1)); - final MapDocument doc2 = MapDocumentUtil - .asMapDocumentWithJPath(dedupConfig, objectMapper.writeValueAsString(r2)); - - final double score = new TreeProcessor(dedupConfig).computeScore(doc1, doc2); - final double threshold = dedupConfig.getWf().getThreshold(); - - return TrustUtils.rescale(score, threshold); - } catch (final Exception e) { - log.error("Error computing score between results", e); - return BrokerConstants.MIN_TRUST; - } - } - protected Topic getTopic() { return topic; } diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/dedupConfig/dedupConfig.json b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/dedupConfig/dedupConfig.json new file mode 100644 index 000000000..d0319b441 --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/dedupConfig/dedupConfig.json @@ -0,0 +1,122 @@ +{ + "wf": { + + }, + "pace": { + "clustering": [ + { + "name": "wordssuffixprefix", + "fields": [ + "title" + ], + "params": { + "max": "2", + "len": "3" + } + }, + { + "name": "lowercase", + "fields": [ + "doi" + ], + "params": { + + } + } + ], + "decisionTree": { + "start": { + "fields": [ + { + "field": "doi", + "comparator": "exactMatch", + "weight": 1.0, + "countIfUndefined": "false", + "params": { + + } + } + ], + "threshold": 0.5, + "aggregation": "AVG", + "positive": "MATCH", + "negative": "layer1", + "undefined": "layer1", + "ignoreUndefined": "true" + }, + "layer1": { + "fields": [ + { + "field": "title", + "comparator": "titleVersionMatch", + "weight": 0.9, + "countIfUndefined": "false", + "params": { + + } + }, + { + "field": "authors", + "comparator": "sizeMatch", + "weight": 0.9, + "countIfUndefined": "false", + "params": { + + } + } + ], + "threshold": 0.5, + "aggregation": "AVG", + "positive": "MATCH", + "negative": "layer2", + "undefined": "layer2", + "ignoreUndefined": "true" + }, + "layer2": { + "fields": [ + { + "field": "title", + "comparator": "levensteinTitle", + "weight": 1.0, + "countIfUndefined": "true", + "params": { + + } + } + ], + "threshold": 0.99, + "aggregation": "AVG", + "positive": "MATCH", + "negative": "NO_MATCH", + "undefined": "NO_MATCH", + "ignoreUndefined": "true" + } + }, + "model": [ + { + "name": "doi", + "type": "String", + "path": "$.pids[?(@.type == 'doi')].value" + }, + { + "name": "title", + "type": "String", + "path": "$.titles", + "length": 250, + "size": 5 + }, + { + "name": "authors", + "type": "List", + "path": "$.creators[*].fullname", + "size": 200 + } + ], + "blacklists": { + + }, + "synonyms": { + + } + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml index b85c60fdf..2c728cd98 100644 --- a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml @@ -8,14 +8,6 @@ workingPath the path where the the generated data will be stored - - - isLookupUrl - the address of the lookUp service - - - dedupConfProfId - the id of a valid Dedup Configuration Profile datasourceIdWhitelist @@ -427,8 +419,6 @@ --conf spark.sql.shuffle.partitions=3840 --workingPath${workingPath} - --isLookupUrl${isLookupUrl} - --dedupConfProfile${dedupConfProfId} --datasourceIdWhitelist${datasourceIdWhitelist} --datasourceTypeWhitelist${datasourceTypeWhitelist} --datasourceIdBlacklist${datasourceIdBlacklist} diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_events.json b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_events.json index c545884f9..bab808193 100644 --- a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_events.json +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_events.json @@ -5,18 +5,6 @@ "paramDescription": "the path where the generated events will be stored", "paramRequired": true }, - { - "paramName": "lu", - "paramLongName": "isLookupUrl", - "paramDescription": "the address of the ISLookUpService", - "paramRequired": true - }, - { - "paramName": "d", - "paramLongName": "dedupConfProfile", - "paramDescription": "the id of a valid Dedup Configuration Profile", - "paramRequired": true - }, { "paramName": "datasourceIdWhitelist", "paramLongName": "datasourceIdWhitelist", diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml index 392271260..d19ad6c5a 100644 --- a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml @@ -9,15 +9,6 @@ workingPath the path where the the generated data will be stored - - isLookupUrl - the address of the lookUp service - - - dedupConfProfId - the id of a valid Dedup Configuration Profile - - sparkDriverMemory memory for driver process diff --git a/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcherTest.java b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcherTest.java index 93bc5617f..82374b335 100644 --- a/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcherTest.java +++ b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcherTest.java @@ -30,7 +30,7 @@ class UpdateMatcherTest { final OaBrokerMainEntity p4 = new OaBrokerMainEntity(); final Collection> list = matcher - .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null); + .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null); assertTrue(list.isEmpty()); } @@ -46,7 +46,7 @@ class UpdateMatcherTest { res.setPublicationdate("2018"); final Collection> list = matcher - .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null); + .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null); assertTrue(list.isEmpty()); } @@ -62,7 +62,7 @@ class UpdateMatcherTest { p2.setPublicationdate("2018"); final Collection> list = matcher - .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null); + .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null); assertTrue(list.size() == 1); } @@ -79,7 +79,7 @@ class UpdateMatcherTest { p2.setPublicationdate("2018"); final Collection> list = matcher - .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null); + .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null); assertTrue(list.isEmpty()); } @@ -98,7 +98,7 @@ class UpdateMatcherTest { p4.setPublicationdate("2018"); final Collection> list = matcher - .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null); + .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null); assertTrue(list.isEmpty()); } @@ -117,7 +117,7 @@ class UpdateMatcherTest { p4.setPublicationdate("2018"); final Collection> list = matcher - .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null, null); + .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null); assertTrue(list.size() == 1); } diff --git a/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/util/TrustUtilsTest.java b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/util/TrustUtilsTest.java index bb23d6085..974baa28b 100644 --- a/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/util/TrustUtilsTest.java +++ b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/util/TrustUtilsTest.java @@ -5,6 +5,10 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import org.junit.jupiter.api.Test; +import eu.dnetlib.broker.objects.OaBrokerAuthor; +import eu.dnetlib.broker.objects.OaBrokerMainEntity; +import eu.dnetlib.broker.objects.OaBrokerTypedValue; + public class TrustUtilsTest { private static final double THRESHOLD = 0.95; @@ -64,6 +68,23 @@ public class TrustUtilsTest { verifyValue(2.00, BrokerConstants.MAX_TRUST); } + @Test + public void test() throws Exception { + final OaBrokerMainEntity r1 = new OaBrokerMainEntity(); + r1.getTitles().add("D-NET Service Package: Data Import"); + r1.getPids().add(new OaBrokerTypedValue("doi", "123")); + r1.getCreators().add(new OaBrokerAuthor("Michele Artini", null)); + r1.getCreators().add(new OaBrokerAuthor("Claudio Atzori", null)); + + final OaBrokerMainEntity r2 = new OaBrokerMainEntity(); + r2.getTitles().add("D-NET Service Package: Data Import"); + // r2.getPids().add(new OaBrokerTypedValue("doi", "123")); + r2.getCreators().add(new OaBrokerAuthor("Michele Artini", null)); + // r2.getCreators().add(new OaBrokerAuthor("Claudio Atzori", null)); + + System.out.println("TRUST: " + TrustUtils.calculateTrust(r1, r2)); + } + private void verifyValue(final double originalScore, final float expectedTrust) { final float trust = TrustUtils.rescale(originalScore, THRESHOLD); System.out.println(trust); From ff4d6214f19292d90076d8ff6ec86a885861c1af Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 10 Jul 2020 10:06:41 +0200 Subject: [PATCH 027/100] experimenting with pruning of relations --- .../dhp/oa/provision/PrepareRelationsJob.java | 344 +++++++++--------- 1 file changed, 174 insertions(+), 170 deletions(-) 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 eb63d4423..e1f7386e9 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 @@ -59,200 +59,204 @@ import scala.Tuple2; */ 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 final int DEFAULT_NUM_PARTITIONS = 3000; + public static final int DEFAULT_NUM_PARTITIONS = 3000; - 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); - int relPartitions = Optional - .ofNullable(parser.get("relPartitions")) - .map(Integer::valueOf) - .orElse(DEFAULT_NUM_PARTITIONS); - log.info("relPartitions: {}", relPartitions); + int relPartitions = Optional + .ofNullable(parser.get("relPartitions")) + .map(Integer::valueOf) + .orElse(DEFAULT_NUM_PARTITIONS); + log.info("relPartitions: {}", relPartitions); - Set relationFilter = Optional - .ofNullable(parser.get("relationFilter")) - .map(s -> Sets.newHashSet(Splitter.on(",").split(s))) - .orElse(new HashSet<>()); - log.info("relationFilter: {}", relationFilter); + Set relationFilter = Optional + .ofNullable(parser.get("relationFilter")) + .map(s -> Sets.newHashSet(Splitter.on(",").split(s))) + .orElse(new HashSet<>()); + log.info("relationFilter: {}", relationFilter); - int maxRelations = Optional - .ofNullable(parser.get("maxRelations")) - .map(Integer::valueOf) - .orElse(MAX_RELS); - log.info("maxRelations: {}", maxRelations); + int maxRelations = Optional + .ofNullable(parser.get("maxRelations")) + .map(Integer::valueOf) + .orElse(MAX_RELS); + log.info("maxRelations: {}", maxRelations); - SparkConf conf = new SparkConf(); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.registerKryoClasses(ProvisionModelSupport.getModelClasses()); + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ProvisionModelSupport.getModelClasses()); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - removeOutputDir(spark, outputPath); - prepareRelationsRDD( - spark, inputRelationsPath, outputPath, relationFilter, maxRelations, relPartitions); - }); + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + prepareRelationsRDD( + spark, inputRelationsPath, outputPath, relationFilter, maxRelations, relPartitions); + }); + } + + /** + * RDD based implementation that prepares the graph relations by limiting the number of outgoing links and filtering + * the relation types according to the given criteria. Moreover, outgoing links kept within the given limit are + * prioritized according to the weights indicated in eu.dnetlib.dhp.oa.provision.model.SortableRelation. + * + * @param spark the spark session + * @param inputRelationsPath source path for the graph relations + * @param outputPath output path for the processed relations + * @param relationFilter set of relation filters applied to the `relClass` field + * @param maxRelations maximum number of allowed outgoing edges + * @param relPartitions number of partitions for the output RDD + */ + private static void prepareRelationsRDD(SparkSession spark, String inputRelationsPath, String outputPath, + Set relationFilter, int maxRelations, int relPartitions) { + + JavaRDD rels = readPathRelationRDD(spark, inputRelationsPath); + + JavaRDD pruned = pruneRels( + pruneRels(rels, relationFilter, maxRelations, relPartitions, (Function) r -> r.getSource()), + relationFilter, maxRelations, relPartitions, (Function) r -> r.getTarget()); + spark + .createDataset(pruned.rdd(), Encoders.bean(Relation.class)) + .repartition(relPartitions) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath); + } + + private static JavaRDD pruneRels(JavaRDD rels, Set relationFilter, int maxRelations, int relPartitions, Function idFn) { + return rels + .filter(rel -> rel.getDataInfo().getDeletedbyinference() == false) + .filter(rel -> relationFilter.contains(rel.getRelClass()) == false) + .mapToPair(r -> new Tuple2<>(SortableRelationKey.create(r, idFn.call(r)), r)) + .repartitionAndSortWithinPartitions(new RelationPartitioner(relPartitions)) + .groupBy(Tuple2::_1) + .map(Tuple2::_2) + .map(t -> Iterables.limit(t, maxRelations)) + .flatMap(Iterable::iterator).map(Tuple2::_2); } - /** - * RDD based implementation that prepares the graph relations by limiting the number of outgoing links and filtering - * the relation types according to the given criteria. Moreover, outgoing links kept within the given limit are - * prioritized according to the weights indicated in eu.dnetlib.dhp.oa.provision.model.SortableRelation. - * - * @param spark the spark session - * @param inputRelationsPath source path for the graph relations - * @param outputPath output path for the processed relations - * @param relationFilter set of relation filters applied to the `relClass` field - * @param maxRelations maximum number of allowed outgoing edges - * @param relPartitions number of partitions for the output RDD - */ - private static void prepareRelationsRDD(SparkSession spark, String inputRelationsPath, String outputPath, - Set relationFilter, int maxRelations, int relPartitions) { + // experimental + private static void prepareRelationsDataset( + SparkSession spark, String inputRelationsPath, String outputPath, Set relationFilter, int maxRelations, + int relPartitions) { + spark + .read() + .textFile(inputRelationsPath) + .repartition(relPartitions) + .map( + (MapFunction) s -> OBJECT_MAPPER.readValue(s, Relation.class), + Encoders.kryo(Relation.class)) + .filter((FilterFunction) rel -> rel.getDataInfo().getDeletedbyinference() == false) + .filter((FilterFunction) rel -> relationFilter.contains(rel.getRelClass()) == false) + .groupByKey( + (MapFunction) Relation::getSource, + Encoders.STRING()) + .agg(new RelationAggregator(maxRelations).toColumn()) + .flatMap( + (FlatMapFunction, Relation>) t -> Iterables + .limit(t._2().getRelations(), maxRelations) + .iterator(), + Encoders.bean(Relation.class)) + .repartition(relPartitions) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath); + } - // group by SOURCE and apply limit - RDD bySource = readPathRelationRDD(spark, inputRelationsPath) - .filter(rel -> rel.getDataInfo().getDeletedbyinference() == false) - .filter(rel -> relationFilter.contains(rel.getRelClass()) == false) - .mapToPair(r -> new Tuple2<>(SortableRelationKey.create(r, r.getSource()), r)) - .repartitionAndSortWithinPartitions(new RelationPartitioner(relPartitions)) - .groupBy(Tuple2::_1) - .map(Tuple2::_2) - .map(t -> Iterables.limit(t, maxRelations)) - .flatMap(Iterable::iterator) - .map(Tuple2::_2) - .rdd(); + public static class RelationAggregator + extends Aggregator { - spark - .createDataset(bySource, Encoders.bean(Relation.class)) - .repartition(relPartitions) - .write() - .mode(SaveMode.Overwrite) - .parquet(outputPath); - } + private int maxRelations; - // experimental - private static void prepareRelationsDataset( - SparkSession spark, String inputRelationsPath, String outputPath, Set relationFilter, int maxRelations, - int relPartitions) { - spark - .read() - .textFile(inputRelationsPath) - .repartition(relPartitions) - .map( - (MapFunction) s -> OBJECT_MAPPER.readValue(s, Relation.class), - Encoders.kryo(Relation.class)) - .filter((FilterFunction) rel -> rel.getDataInfo().getDeletedbyinference() == false) - .filter((FilterFunction) rel -> relationFilter.contains(rel.getRelClass()) == false) - .groupByKey( - (MapFunction) Relation::getSource, - Encoders.STRING()) - .agg(new RelationAggregator(maxRelations).toColumn()) - .flatMap( - (FlatMapFunction, Relation>) t -> Iterables - .limit(t._2().getRelations(), maxRelations) - .iterator(), - Encoders.bean(Relation.class)) - .repartition(relPartitions) - .write() - .mode(SaveMode.Overwrite) - .parquet(outputPath); - } + public RelationAggregator(int maxRelations) { + this.maxRelations = maxRelations; + } - public static class RelationAggregator - extends Aggregator { + @Override + public RelationList zero() { + return new RelationList(); + } - private int maxRelations; + @Override + public RelationList reduce(RelationList b, Relation a) { + b.getRelations().add(a); + return getSortableRelationList(b); + } - public RelationAggregator(int maxRelations) { - this.maxRelations = maxRelations; - } + @Override + public RelationList merge(RelationList b1, RelationList b2) { + b1.getRelations().addAll(b2.getRelations()); + return getSortableRelationList(b1); + } - @Override - public RelationList zero() { - return new RelationList(); - } + @Override + public RelationList finish(RelationList r) { + return getSortableRelationList(r); + } - @Override - public RelationList reduce(RelationList b, Relation a) { - b.getRelations().add(a); - return getSortableRelationList(b); - } + private RelationList getSortableRelationList(RelationList b1) { + RelationList sr = new RelationList(); + sr + .setRelations( + b1 + .getRelations() + .stream() + .limit(maxRelations) + .collect(Collectors.toCollection(() -> new PriorityQueue<>(new RelationComparator())))); + return sr; + } - @Override - public RelationList merge(RelationList b1, RelationList b2) { - b1.getRelations().addAll(b2.getRelations()); - return getSortableRelationList(b1); - } + @Override + public Encoder bufferEncoder() { + return Encoders.kryo(RelationList.class); + } - @Override - public RelationList finish(RelationList r) { - return getSortableRelationList(r); - } + @Override + public Encoder outputEncoder() { + return Encoders.kryo(RelationList.class); + } + } - private RelationList getSortableRelationList(RelationList b1) { - RelationList sr = new RelationList(); - sr - .setRelations( - b1 - .getRelations() - .stream() - .limit(maxRelations) - .collect(Collectors.toCollection(() -> new PriorityQueue<>(new RelationComparator())))); - return sr; - } + /** + * Reads a JavaRDD of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline delimited json text + * file, + * + * @param spark + * @param inputPath + * @return the JavaRDD containing all the relationships + */ + 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, Relation.class)); + } - @Override - public Encoder bufferEncoder() { - return Encoders.kryo(RelationList.class); - } - - @Override - public Encoder outputEncoder() { - return Encoders.kryo(RelationList.class); - } - } - - /** - * Reads a JavaRDD of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline delimited json text - * file, - * - * @param spark - * @param inputPath - * @return the JavaRDD containing all the relationships - */ - 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, Relation.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()); + } } From b21866a2da7be44106713c3f3982e3ee646d22d4 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 10 Jul 2020 13:59:48 +0200 Subject: [PATCH 028/100] allow to set different to relations cut points by source and by target; adjusted weight assigned to relationship types --- .../dhp/oa/provision/PrepareRelationsJob.java | 358 +++++++++--------- .../provision/model/SortableRelationKey.java | 22 +- .../input_params_prepare_relations.json | 13 +- 3 files changed, 206 insertions(+), 187 deletions(-) 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 e1f7386e9..da0a81021 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 @@ -59,204 +59,216 @@ import scala.Tuple2; */ 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 final int DEFAULT_NUM_PARTITIONS = 3000; + public static final int DEFAULT_NUM_PARTITIONS = 3000; - 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); - int relPartitions = Optional - .ofNullable(parser.get("relPartitions")) - .map(Integer::valueOf) - .orElse(DEFAULT_NUM_PARTITIONS); - log.info("relPartitions: {}", relPartitions); + int relPartitions = Optional + .ofNullable(parser.get("relPartitions")) + .map(Integer::valueOf) + .orElse(DEFAULT_NUM_PARTITIONS); + log.info("relPartitions: {}", relPartitions); - Set relationFilter = Optional - .ofNullable(parser.get("relationFilter")) - .map(s -> Sets.newHashSet(Splitter.on(",").split(s))) - .orElse(new HashSet<>()); - log.info("relationFilter: {}", relationFilter); + Set relationFilter = Optional + .ofNullable(parser.get("relationFilter")) + .map(s -> Sets.newHashSet(Splitter.on(",").split(s))) + .orElse(new HashSet<>()); + log.info("relationFilter: {}", relationFilter); - int maxRelations = Optional - .ofNullable(parser.get("maxRelations")) - .map(Integer::valueOf) - .orElse(MAX_RELS); - log.info("maxRelations: {}", maxRelations); + int sourceMaxRelations = Optional + .ofNullable(parser.get("sourceMaxRelations")) + .map(Integer::valueOf) + .orElse(MAX_RELS); + log.info("sourceMaxRelations: {}", sourceMaxRelations); - SparkConf conf = new SparkConf(); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.registerKryoClasses(ProvisionModelSupport.getModelClasses()); + int targetMaxRelations = Optional + .ofNullable(parser.get("targetMaxRelations")) + .map(Integer::valueOf) + .orElse(MAX_RELS); + log.info("targetMaxRelations: {}", targetMaxRelations); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - removeOutputDir(spark, outputPath); - prepareRelationsRDD( - spark, inputRelationsPath, outputPath, relationFilter, maxRelations, relPartitions); - }); - } + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ProvisionModelSupport.getModelClasses()); - /** - * RDD based implementation that prepares the graph relations by limiting the number of outgoing links and filtering - * the relation types according to the given criteria. Moreover, outgoing links kept within the given limit are - * prioritized according to the weights indicated in eu.dnetlib.dhp.oa.provision.model.SortableRelation. - * - * @param spark the spark session - * @param inputRelationsPath source path for the graph relations - * @param outputPath output path for the processed relations - * @param relationFilter set of relation filters applied to the `relClass` field - * @param maxRelations maximum number of allowed outgoing edges - * @param relPartitions number of partitions for the output RDD - */ - private static void prepareRelationsRDD(SparkSession spark, String inputRelationsPath, String outputPath, - Set relationFilter, int maxRelations, int relPartitions) { - - JavaRDD rels = readPathRelationRDD(spark, inputRelationsPath); - - JavaRDD pruned = pruneRels( - pruneRels(rels, relationFilter, maxRelations, relPartitions, (Function) r -> r.getSource()), - relationFilter, maxRelations, relPartitions, (Function) r -> r.getTarget()); - spark - .createDataset(pruned.rdd(), Encoders.bean(Relation.class)) - .repartition(relPartitions) - .write() - .mode(SaveMode.Overwrite) - .parquet(outputPath); - } - - private static JavaRDD pruneRels(JavaRDD rels, Set relationFilter, int maxRelations, int relPartitions, Function idFn) { - return rels - .filter(rel -> rel.getDataInfo().getDeletedbyinference() == false) - .filter(rel -> relationFilter.contains(rel.getRelClass()) == false) - .mapToPair(r -> new Tuple2<>(SortableRelationKey.create(r, idFn.call(r)), r)) - .repartitionAndSortWithinPartitions(new RelationPartitioner(relPartitions)) - .groupBy(Tuple2::_1) - .map(Tuple2::_2) - .map(t -> Iterables.limit(t, maxRelations)) - .flatMap(Iterable::iterator).map(Tuple2::_2); + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + prepareRelationsRDD( + spark, inputRelationsPath, outputPath, relationFilter, sourceMaxRelations, targetMaxRelations, + relPartitions); + }); } - // experimental - private static void prepareRelationsDataset( - SparkSession spark, String inputRelationsPath, String outputPath, Set relationFilter, int maxRelations, - int relPartitions) { - spark - .read() - .textFile(inputRelationsPath) - .repartition(relPartitions) - .map( - (MapFunction) s -> OBJECT_MAPPER.readValue(s, Relation.class), - Encoders.kryo(Relation.class)) - .filter((FilterFunction) rel -> rel.getDataInfo().getDeletedbyinference() == false) - .filter((FilterFunction) rel -> relationFilter.contains(rel.getRelClass()) == false) - .groupByKey( - (MapFunction) Relation::getSource, - Encoders.STRING()) - .agg(new RelationAggregator(maxRelations).toColumn()) - .flatMap( - (FlatMapFunction, Relation>) t -> Iterables - .limit(t._2().getRelations(), maxRelations) - .iterator(), - Encoders.bean(Relation.class)) - .repartition(relPartitions) - .write() - .mode(SaveMode.Overwrite) - .parquet(outputPath); - } + /** + * RDD based implementation that prepares the graph relations by limiting the number of outgoing links and filtering + * the relation types according to the given criteria. Moreover, outgoing links kept within the given limit are + * prioritized according to the weights indicated in eu.dnetlib.dhp.oa.provision.model.SortableRelation. + * + * @param spark the spark session + * @param inputRelationsPath source path for the graph relations + * @param outputPath output path for the processed relations + * @param relationFilter set of relation filters applied to the `relClass` field + * @param sourceMaxRelations maximum number of allowed outgoing edges grouping by relation.source + * @param targetMaxRelations maximum number of allowed outgoing edges grouping by relation.target + * @param relPartitions number of partitions for the output RDD + */ + private static void prepareRelationsRDD(SparkSession spark, String inputRelationsPath, String outputPath, + Set relationFilter, int sourceMaxRelations, int targetMaxRelations, int relPartitions) { - public static class RelationAggregator - extends Aggregator { + JavaRDD rels = readPathRelationRDD(spark, inputRelationsPath) + .filter(rel -> rel.getDataInfo().getDeletedbyinference() == false) + .filter(rel -> relationFilter.contains(rel.getRelClass()) == false); - private int maxRelations; + JavaRDD pruned = pruneRels( + pruneRels( + rels, + sourceMaxRelations, relPartitions, (Function) r -> r.getSource()), + targetMaxRelations, relPartitions, (Function) r -> r.getTarget()); + spark + .createDataset(pruned.rdd(), Encoders.bean(Relation.class)) + .repartition(relPartitions) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath); + } - public RelationAggregator(int maxRelations) { - this.maxRelations = maxRelations; - } + private static JavaRDD pruneRels(JavaRDD rels, int maxRelations, + int relPartitions, Function idFn) { + return rels + .mapToPair(r -> new Tuple2<>(SortableRelationKey.create(r, idFn.call(r)), r)) + .repartitionAndSortWithinPartitions(new RelationPartitioner(relPartitions)) + .groupBy(Tuple2::_1) + .map(Tuple2::_2) + .map(t -> Iterables.limit(t, maxRelations)) + .flatMap(Iterable::iterator) + .map(Tuple2::_2); + } - @Override - public RelationList zero() { - return new RelationList(); - } + // experimental + private static void prepareRelationsDataset( + SparkSession spark, String inputRelationsPath, String outputPath, Set relationFilter, int maxRelations, + int relPartitions) { + spark + .read() + .textFile(inputRelationsPath) + .repartition(relPartitions) + .map( + (MapFunction) s -> OBJECT_MAPPER.readValue(s, Relation.class), + Encoders.kryo(Relation.class)) + .filter((FilterFunction) rel -> rel.getDataInfo().getDeletedbyinference() == false) + .filter((FilterFunction) rel -> relationFilter.contains(rel.getRelClass()) == false) + .groupByKey( + (MapFunction) Relation::getSource, + Encoders.STRING()) + .agg(new RelationAggregator(maxRelations).toColumn()) + .flatMap( + (FlatMapFunction, Relation>) t -> Iterables + .limit(t._2().getRelations(), maxRelations) + .iterator(), + Encoders.bean(Relation.class)) + .repartition(relPartitions) + .write() + .mode(SaveMode.Overwrite) + .parquet(outputPath); + } - @Override - public RelationList reduce(RelationList b, Relation a) { - b.getRelations().add(a); - return getSortableRelationList(b); - } + public static class RelationAggregator + extends Aggregator { - @Override - public RelationList merge(RelationList b1, RelationList b2) { - b1.getRelations().addAll(b2.getRelations()); - return getSortableRelationList(b1); - } + private int maxRelations; - @Override - public RelationList finish(RelationList r) { - return getSortableRelationList(r); - } + public RelationAggregator(int maxRelations) { + this.maxRelations = maxRelations; + } - private RelationList getSortableRelationList(RelationList b1) { - RelationList sr = new RelationList(); - sr - .setRelations( - b1 - .getRelations() - .stream() - .limit(maxRelations) - .collect(Collectors.toCollection(() -> new PriorityQueue<>(new RelationComparator())))); - return sr; - } + @Override + public RelationList zero() { + return new RelationList(); + } - @Override - public Encoder bufferEncoder() { - return Encoders.kryo(RelationList.class); - } + @Override + public RelationList reduce(RelationList b, Relation a) { + b.getRelations().add(a); + return getSortableRelationList(b); + } - @Override - public Encoder outputEncoder() { - return Encoders.kryo(RelationList.class); - } - } + @Override + public RelationList merge(RelationList b1, RelationList b2) { + b1.getRelations().addAll(b2.getRelations()); + return getSortableRelationList(b1); + } - /** - * Reads a JavaRDD of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline delimited json text - * file, - * - * @param spark - * @param inputPath - * @return the JavaRDD containing all the relationships - */ - 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, Relation.class)); - } + @Override + public RelationList finish(RelationList r) { + return getSortableRelationList(r); + } - private static void removeOutputDir(SparkSession spark, String path) { - HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); - } + private RelationList getSortableRelationList(RelationList b1) { + RelationList sr = new RelationList(); + sr + .setRelations( + b1 + .getRelations() + .stream() + .limit(maxRelations) + .collect(Collectors.toCollection(() -> new PriorityQueue<>(new RelationComparator())))); + return sr; + } + + @Override + public Encoder bufferEncoder() { + return Encoders.kryo(RelationList.class); + } + + @Override + public Encoder outputEncoder() { + return Encoders.kryo(RelationList.class); + } + } + + /** + * Reads a JavaRDD of eu.dnetlib.dhp.oa.provision.model.SortableRelation objects from a newline delimited json text + * file, + * + * @param spark + * @param inputPath + * @return the JavaRDD containing all the relationships + */ + 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, Relation.class)); + } + + 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/model/SortableRelationKey.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/SortableRelationKey.java index bf7f9330d..bd7b4d78e 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/SortableRelationKey.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/model/SortableRelationKey.java @@ -16,18 +16,18 @@ public class SortableRelationKey implements Comparable, Ser private static final Map weights = Maps.newHashMap(); static { - weights.put("outcome", 0); - weights.put("supplement", 1); - weights.put("review", 2); - weights.put("citation", 3); - weights.put("affiliation", 4); - weights.put("relationship", 5); - weights.put("publicationDataset", 6); - weights.put("similarity", 7); + weights.put("participation", 0); - weights.put("provision", 8); - weights.put("participation", 9); - weights.put("dedup", 10); + weights.put("outcome", 1); + weights.put("affiliation", 2); + weights.put("dedup", 3); + weights.put("publicationDataset", 4); + weights.put("citation", 5); + weights.put("supplement", 6); + weights.put("review", 7); + weights.put("relationship", 8); + weights.put("provision", 9); + weights.put("similarity", 10); } private static final long serialVersionUID = 3232323; diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_prepare_relations.json b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_prepare_relations.json index 71b2becc4..33fa1dc8d 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_prepare_relations.json +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/input_params_prepare_relations.json @@ -30,9 +30,16 @@ "paramRequired": false }, { - "paramName": "mr", - "paramLongName": "maxRelations", - "paramDescription": "maximum number of relations allowed for a each entity", + "paramName": "smr", + "paramLongName": "sourceMaxRelations", + "paramDescription": "maximum number of relations allowed for a each entity grouping by source", + "paramRequired": false + }, + { + "paramName": "tmr", + "paramLongName": "targetMaxRelations", + "paramDescription": "maximum number of relations allowed for a each entity grouping by target", "paramRequired": false } + ] From a7d39774812ab3ede6570e65ebe3cb902ca11667 Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Fri, 10 Jul 2020 14:44:50 +0200 Subject: [PATCH 029/100] added generation of EBI Dataset --- dhp-schemas/pom.xml | 31 +++ .../dhp/schema/scholexplorer/OafUtils.scala | 90 +++++++++ dhp-workflows/dhp-graph-mapper/pom.xml | 38 ++++ .../dhp/sx/ebi/SparkAddLinkUpdates.scala | 138 +++++++++++++ .../sx/ebi/SparkCreateBaselineDataFrame.scala | 49 +++++ .../dhp/sx/ebi/SparkCreateEBIDataFrame.scala | 6 +- .../dnetlib/dhp/sx/ebi/model/PMArticle.java | 64 ++++++ .../eu/dnetlib/dhp/sx/ebi/model/PMAuthor.java | 31 +++ .../dnetlib/dhp/sx/ebi/model/PMJournal.java | 53 +++++ .../dnetlib/dhp/sx/ebi/model/PMParser.scala | 92 +++++++++ .../parser/AbstractScholexplorerParser.java | 6 +- .../parser/DatasetScholexplorerParser.java | 30 ++- .../PublicationScholexplorerParser.java | 6 +- .../dhp/sx/ebi/oozie_app/config-default.xml | 46 ++++- .../dnetlib/dhp/sx/ebi/oozie_app/workflow.xml | 52 ++++- .../java/eu/dnetlib/dhp/sx/ebi/TestEBI.scala | 20 ++ .../eu/dnetlib/dhp/sx/ebi/TestEBITODS.scala | 22 -- .../resources/eu/dnetlib/dhp/sx/ebi/rel1.json | 55 +++++ .../eu/dnetlib/dhp/sx/ebi/rel_multiple.json | 191 ++++++++++++++++++ .../java/eu/dnetlib/dhp/export/DLIToOAF.scala | 43 +--- .../oa/provision/PrepareRelationsJobTest.java | 29 ++- 21 files changed, 985 insertions(+), 107 deletions(-) create mode 100644 dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/OafUtils.scala create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/SparkAddLinkUpdates.scala create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/SparkCreateBaselineDataFrame.scala create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/model/PMArticle.java create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/model/PMAuthor.java create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/model/PMJournal.java create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/model/PMParser.scala create mode 100644 dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/ebi/TestEBI.scala delete mode 100644 dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/ebi/TestEBITODS.scala create mode 100644 dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/sx/ebi/rel1.json create mode 100644 dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/sx/ebi/rel_multiple.json diff --git a/dhp-schemas/pom.xml b/dhp-schemas/pom.xml index 2e5652b43..b04d62dd2 100644 --- a/dhp-schemas/pom.xml +++ b/dhp-schemas/pom.xml @@ -14,6 +14,37 @@ This module contains common schema classes meant to be used across the dnet-hadoop submodules + + + + net.alchim31.maven + scala-maven-plugin + 4.0.1 + + + scala-compile-first + initialize + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + ${scala.version} + + + + + + diff --git a/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/OafUtils.scala b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/OafUtils.scala new file mode 100644 index 000000000..27eec77fa --- /dev/null +++ b/dhp-schemas/src/main/java/eu/dnetlib/dhp/schema/scholexplorer/OafUtils.scala @@ -0,0 +1,90 @@ +package eu.dnetlib.dhp.schema.scholexplorer + +import eu.dnetlib.dhp.schema.oaf.{DataInfo, Field, KeyValue, Qualifier, StructuredProperty} + +object OafUtils { + + + + def generateKeyValue(key: String, value: String): KeyValue = { + val kv: KeyValue = new KeyValue() + kv.setKey(key) + kv.setValue(value) + kv.setDataInfo(generateDataInfo("0.9")) + kv + } + + + def generateDataInfo(trust: String = "0.9", invisibile: Boolean = false): DataInfo = { + val di = new DataInfo + di.setDeletedbyinference(false) + di.setInferred(false) + di.setInvisible(false) + di.setTrust(trust) + di.setProvenanceaction(createQualifier("sysimport:actionset", "dnet:provenanceActions")) + di + } + + def createQualifier(cls: String, sch: String): Qualifier = { + createQualifier(cls, cls, sch, sch) + } + + + def createQualifier(classId: String, className: String, schemeId: String, schemeName: String): Qualifier = { + val q: Qualifier = new Qualifier + q.setClassid(classId) + q.setClassname(className) + q.setSchemeid(schemeId) + q.setSchemename(schemeName) + q + } + + + def asField[T](value: T): Field[T] = { + val tmp = new Field[T] + tmp.setValue(value) + tmp + + + } + + def createSP(value: String, classId: String,className:String, schemeId: String, schemeName:String): StructuredProperty = { + val sp = new StructuredProperty + sp.setQualifier(createQualifier(classId,className, schemeId, schemeName)) + sp.setValue(value) + sp + + } + + + + def createSP(value: String, classId: String,className:String, schemeId: String, schemeName:String, dataInfo: DataInfo): StructuredProperty = { + val sp = new StructuredProperty + sp.setQualifier(createQualifier(classId,className, schemeId, schemeName)) + sp.setValue(value) + sp.setDataInfo(dataInfo) + sp + + } + + def createSP(value: String, classId: String, schemeId: String): StructuredProperty = { + val sp = new StructuredProperty + sp.setQualifier(createQualifier(classId, schemeId)) + sp.setValue(value) + sp + + } + + + + def createSP(value: String, classId: String, schemeId: String, dataInfo: DataInfo): StructuredProperty = { + val sp = new StructuredProperty + sp.setQualifier(createQualifier(classId, schemeId)) + sp.setValue(value) + sp.setDataInfo(dataInfo) + sp + + } + + +} diff --git a/dhp-workflows/dhp-graph-mapper/pom.xml b/dhp-workflows/dhp-graph-mapper/pom.xml index 0439c2ba3..5ddcda3fa 100644 --- a/dhp-workflows/dhp-graph-mapper/pom.xml +++ b/dhp-workflows/dhp-graph-mapper/pom.xml @@ -9,6 +9,37 @@ dhp-graph-mapper + + + + net.alchim31.maven + scala-maven-plugin + 4.0.1 + + + scala-compile-first + initialize + + add-source + compile + + + + scala-test-compile + process-test-resources + + testCompile + + + + + ${scala.version} + + + + + + @@ -61,6 +92,13 @@ org.postgresql postgresql + + org.json4s + json4s-jackson_2.11 + 3.5.3 + + + diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/SparkAddLinkUpdates.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/SparkAddLinkUpdates.scala new file mode 100644 index 000000000..897bbd540 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/SparkAddLinkUpdates.scala @@ -0,0 +1,138 @@ +package eu.dnetlib.dhp.sx.ebi +import eu.dnetlib.dhp.application.ArgumentApplicationParser +import eu.dnetlib.dhp.schema.oaf.{Instance, KeyValue, Oaf} +import eu.dnetlib.dhp.schema.scholexplorer.OafUtils.createQualifier +import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIRelation, OafUtils, ProvenaceInfo} +import eu.dnetlib.dhp.utils.DHPUtils +import eu.dnetlib.scholexplorer.relation.RelationMapper +import org.apache.commons.io.IOUtils +import org.apache.spark.SparkConf +import org.apache.spark.sql._ +import org.json4s +import org.json4s.DefaultFormats +import org.json4s.JsonAST.{JField, JObject, JString} +import org.json4s.jackson.JsonMethods.parse + +import scala.collection.JavaConverters._ + +object SparkAddLinkUpdates { + + val relationMapper = RelationMapper.load + + +case class EBILinks(relation:String, pubdate:String, tpid:String, tpidType:String, turl:String, title:String, publisher:String) {} + + + def generatePubmedDLICollectedFrom(): KeyValue = { + OafUtils.generateKeyValue("dli_________::europe_pmc__", "Europe PMC") + } + + + def ebiLinksToOaf(input:(String, String)):List[Oaf] = { + val pmid :String = input._1 + val input_json :String = input._2 + implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats + lazy val json: json4s.JValue = parse(input_json) + + + val targets:List[EBILinks] = for { + JObject(link) <- json \\ "Category" \\ "Link" + JField("PublicationDate", JString(pubdate)) <- link + JField("RelationshipType", JObject(relationshipType)) <- link + JField("Name", JString(relname)) <- relationshipType + JField("Target", JObject(target)) <- link + JField("Identifier", JObject(identifier)) <- target + JField("ID", JString(tpid)) <- identifier + JField("IDScheme", JString(tpidtype)) <- identifier + JField("IDURL", JString(turl)) <- identifier + JField("Title", JString(title)) <- target + JField("Publisher", JObject(pub)) <- target + JField("Name", JString(publisher)) <- pub + } yield EBILinks(relname, pubdate, tpid, tpidtype, turl,title, publisher) + + + + val dnetPublicationId = s"50|${DHPUtils.md5(s"$pmid::pmid")}" + + targets.flatMap(l => { + val relation = new DLIRelation + val inverseRelation = new DLIRelation + val targetDnetId = s"50|${DHPUtils.md5(s"${l.tpid.toLowerCase.trim}::${l.tpidType.toLowerCase.trim}")}" + val relInfo = relationMapper.get(l.relation.toLowerCase) + val relationSemantic = relInfo.getOriginal + val inverseRelationSemantic = relInfo.getInverse + + relation.setSource(dnetPublicationId) + relation.setTarget(targetDnetId) + relation.setRelClass("datacite") + relation.setRelType(relationSemantic) + relation.setCollectedfrom(List(generatePubmedDLICollectedFrom()).asJava) + + inverseRelation.setSource(targetDnetId) + inverseRelation.setTarget(dnetPublicationId) + inverseRelation.setRelClass("datacite") + inverseRelation.setRelType(inverseRelationSemantic) + inverseRelation.setCollectedfrom(List(generatePubmedDLICollectedFrom()).asJava) + + + + val d = new DLIDataset + d.setId(targetDnetId) + d.setDataInfo(OafUtils.generateDataInfo()) + d.setPid(List(OafUtils.createSP(l.tpid.toLowerCase.trim, l.tpidType.toLowerCase.trim, "dnet:pid_types")).asJava) + d.setCompletionStatus("complete") + val pi = new ProvenaceInfo + pi.setId("dli_________::europe_pmc__") + pi.setName( "Europe PMC") + pi.setCompletionStatus("complete") + pi.setCollectionMode("collected") + d.setDlicollectedfrom(List(pi).asJava) + d.setCollectedfrom(List(generatePubmedDLICollectedFrom()).asJava) + d.setPublisher(OafUtils.asField(l.publisher)) + d.setTitle(List(OafUtils.createSP(l.title, "main title", "dnet:dataCite_title")).asJava) + d.setDateofacceptance(OafUtils.asField(l.pubdate)) + val i = new Instance + i.setCollectedfrom(generatePubmedDLICollectedFrom()) + i.setDateofacceptance(d.getDateofacceptance) + i.setUrl(List(l.turl).asJava) + i.setInstancetype(createQualifier("0021", "Dataset", "dnet:publication_resource", "dnet:publication_resource")) + d.setInstance(List(i).asJava) + List(relation, inverseRelation, d) + }) + } + + + def main(args: Array[String]): Unit = { + val conf: SparkConf = new SparkConf() + val parser = new ArgumentApplicationParser(IOUtils.toString(SparkCreateEBIDataFrame.getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/ebi/ebi_to_df_params.json"))) + parser.parseArgument(args) + val spark: SparkSession = + SparkSession + .builder() + .config(conf) + .appName(SparkCreateEBIDataFrame.getClass.getSimpleName) + .master(parser.get("master")).getOrCreate() + + + val workingPath = parser.get("workingPath") + implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf]) + implicit val relEncoder: Encoder[DLIRelation] = Encoders.kryo(classOf[DLIRelation]) + implicit val datEncoder: Encoder[DLIDataset] = Encoders.kryo(classOf[DLIDataset]) + + val ds:Dataset[(String,String)] = spark.read.load(s"$workingPath/baseline_links_updates").as[(String,String)](Encoders.tuple(Encoders.STRING, Encoders.STRING)) + + ds.flatMap(l =>ebiLinksToOaf(l)).write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_links_updates_oaf") + + ds.filter(s => s.isInstanceOf) + + + + val oDataset:Dataset[Oaf] = spark.read.load(s"$workingPath/baseline_links_updates_oaf").as[Oaf] + + oDataset.filter(p =>p.isInstanceOf[DLIRelation]).map(p => p.asInstanceOf[DLIRelation]).write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_links_updates_relation") + oDataset.filter(p =>p.isInstanceOf[DLIDataset]).map(p => p.asInstanceOf[DLIDataset]).write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_links_updates_dataset") + + + + } +} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/SparkCreateBaselineDataFrame.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/SparkCreateBaselineDataFrame.scala new file mode 100644 index 000000000..77e03c9b3 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/SparkCreateBaselineDataFrame.scala @@ -0,0 +1,49 @@ +package eu.dnetlib.dhp.sx.ebi + +import eu.dnetlib.dhp.application.ArgumentApplicationParser +import org.apache.commons.io.IOUtils +import org.apache.spark.SparkConf +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession} +import eu.dnetlib.dhp.sx.ebi.model.{PMArticle, PMAuthor, PMJournal, PMParser} + + +import scala.io.Source +import scala.xml.pull.XMLEventReader + +object SparkCreateBaselineDataFrame { + + + def main(args: Array[String]): Unit = { + val conf: SparkConf = new SparkConf() + val parser = new ArgumentApplicationParser(IOUtils.toString(SparkCreateEBIDataFrame.getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/ebi/ebi_to_df_params.json"))) + parser.parseArgument(args) + val spark: SparkSession = + SparkSession + .builder() + .config(conf) + .appName(SparkCreateEBIDataFrame.getClass.getSimpleName) + .master(parser.get("master")).getOrCreate() + + val sc = spark.sparkContext + + val workingPath = parser.get("workingPath") + + implicit val PMEncoder: Encoder[PMArticle] = Encoders.kryo(classOf[PMArticle]) + implicit val PMJEncoder: Encoder[PMJournal] = Encoders.kryo(classOf[PMJournal]) + implicit val PMAEncoder: Encoder[PMAuthor] = Encoders.kryo(classOf[PMAuthor]) + val k: RDD[(String, String)] = sc.wholeTextFiles(s"$workingPath/baseline",2000) + + val ds:Dataset[PMArticle] = spark.createDataset(k.filter(i => i._1.endsWith(".gz")).flatMap(i =>{ + val xml = new XMLEventReader(Source.fromBytes(i._2.getBytes())) + new PMParser(xml) + + } )) + + ds.write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_dataset") + + + + + } +} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/SparkCreateEBIDataFrame.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/SparkCreateEBIDataFrame.scala index c7cc3ed04..60857f0fc 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/SparkCreateEBIDataFrame.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/SparkCreateEBIDataFrame.scala @@ -51,7 +51,7 @@ object SparkCreateEBIDataFrame { spark.createDataset(oafPubsRDD).write.mode(SaveMode.Overwrite).save(s"$workingPath/oaf") logger.info("Extract Publication and relation from dataset_xml") - val oafDatsRDD:RDD[Oaf] = sc.textFile(s"$workingPath/_dataset_xml").map(s => + val oafDatsRDD:RDD[Oaf] = sc.textFile(s"$workingPath/dataset_xml").map(s => { new ObjectMapper().readValue(s, classOf[String]) }).flatMap(s => { @@ -79,5 +79,9 @@ object SparkCreateEBIDataFrame { .agg(EBIAggregator.getRelationAggregator().toColumn) .map(p => p._2) .write.mode(SaveMode.Overwrite).save(s"$workingPath/relation") + + + + relations.map(r => (r.getSource, r.getTarget))(Encoders.tuple(Encoders.STRING,Encoders.STRING)) } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/model/PMArticle.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/model/PMArticle.java new file mode 100644 index 000000000..75d4628e6 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/model/PMArticle.java @@ -0,0 +1,64 @@ + +package eu.dnetlib.dhp.sx.ebi.model; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +public class PMArticle implements Serializable { + + private String pmid; + private String date; + private PMJournal journal; + private String title; + private String description; + private List authors = new ArrayList<>(); + + public String getPmid() { + return pmid; + } + + public void setPmid(String pmid) { + this.pmid = pmid; + } + + public String getDate() { + return date; + } + + public void setDate(String date) { + this.date = date; + } + + public PMJournal getJournal() { + return journal; + } + + public void setJournal(PMJournal journal) { + this.journal = journal; + } + + public String getTitle() { + return title; + } + + public void setTitle(String title) { + this.title = title; + } + + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + public List getAuthors() { + return authors; + } + + public void setAuthors(List authors) { + this.authors = authors; + } +} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/model/PMAuthor.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/model/PMAuthor.java new file mode 100644 index 000000000..4a2198542 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/model/PMAuthor.java @@ -0,0 +1,31 @@ + +package eu.dnetlib.dhp.sx.ebi.model; + +import java.io.Serializable; + +public class PMAuthor implements Serializable { + + private String lastName; + private String foreName; + + public String getLastName() { + return lastName; + } + + public void setLastName(String lastName) { + this.lastName = lastName; + } + + public String getForeName() { + return foreName; + } + + public void setForeName(String foreName) { + this.foreName = foreName; + } + + public String getFullName() { + return String.format("%s, %s", this.foreName, this.lastName); + } + +} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/model/PMJournal.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/model/PMJournal.java new file mode 100644 index 000000000..d4ff5a158 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/model/PMJournal.java @@ -0,0 +1,53 @@ + +package eu.dnetlib.dhp.sx.ebi.model; + +import java.io.Serializable; + +public class PMJournal implements Serializable { + + private String issn; + private String volume; + private String issue; + private String date; + private String title; + + public String getIssn() { + return issn; + } + + public void setIssn(String issn) { + this.issn = issn; + } + + public String getVolume() { + return volume; + } + + public void setVolume(String volume) { + this.volume = volume; + } + + public String getIssue() { + return issue; + } + + public void setIssue(String issue) { + this.issue = issue; + } + + public String getDate() { + return date; + } + + public void setDate(String date) { + this.date = date; + } + + public String getTitle() { + return title; + } + + public void setTitle(String title) { + this.title = title; + } +} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/model/PMParser.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/model/PMParser.scala new file mode 100644 index 000000000..903eba134 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/model/PMParser.scala @@ -0,0 +1,92 @@ +package eu.dnetlib.dhp.sx.ebi.model +import scala.xml.pull.{EvElemEnd, EvElemStart, EvText, XMLEventReader} +class PMParser(xml:XMLEventReader) extends Iterator[PMArticle] { + + var currentArticle:PMArticle = generateNextArticle() + + override def hasNext: Boolean = currentArticle!= null + + override def next(): PMArticle = { + val tmp = currentArticle + currentArticle = generateNextArticle() + tmp + } + + + def generateNextArticle():PMArticle = { + + var currentAuthor: PMAuthor = null + var currentJournal: PMJournal = null + var currNode: String = null + var currentYear = "0" + var currentMonth = "01" + var currentDay = "01" + + while (xml.hasNext) { + xml.next match { + case EvElemStart(_, label, _, _) => + currNode = label + label match { + case "PubmedArticle" => currentArticle = new PMArticle + case "Author" => currentAuthor = new PMAuthor + case "Journal" => currentJournal = new PMJournal + case _ => + } + case EvElemEnd(_, label) => + label match { + case "PubmedArticle" => return currentArticle + case "Author" => currentArticle.getAuthors.add(currentAuthor) + case "Journal" => currentArticle.setJournal(currentJournal) + case "DateCompleted" => currentArticle.setDate(s"$currentYear-$currentMonth-$currentDay") + case "PubDate" => currentJournal.setDate(s"$currentYear-$currentMonth-$currentDay") + case _ => + } + case EvText(text) => + if (currNode!= null && text.trim.nonEmpty) + currNode match { + case "ArticleTitle" => { + if (currentArticle.getTitle==null) + currentArticle.setTitle(text.trim) + else + currentArticle.setTitle(currentArticle.getTitle + text.trim) + } + case "AbstractText" => { + if (currentArticle.getDescription==null) + currentArticle.setDescription(text.trim) + else + currentArticle.setDescription(currentArticle.getDescription + text.trim) + } + case "PMID" => currentArticle.setPmid(text.trim) + case "ISSN" => currentJournal.setIssn(text.trim) + case "Year" => currentYear = text.trim + case "Month" => currentMonth = text.trim + case "Day" => currentDay = text.trim + case "Volume" => currentJournal.setVolume( text.trim) + case "Issue" => currentJournal.setIssue (text.trim) + case "LastName" => { + if (currentAuthor != null) + currentAuthor.setLastName(text.trim) + + } + case "ForeName" => if (currentAuthor != null) + currentAuthor.setForeName(text.trim) + case "Title" => + if (currentJournal.getTitle==null) + currentJournal.setTitle(text.trim) + else + currentJournal.setTitle(currentJournal.getTitle + text.trim) + case _ => + + } + case _ => + } + + } + null + } +} + + + + + 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 f0c789323..75f28c129 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 @@ -150,8 +150,8 @@ public abstract class AbstractScholexplorerParser { return uk; } - - protected Qualifier generateQualifier(final String classId, final String className, final String schemeId, final String schemeName) { + protected Qualifier generateQualifier(final String classId, final String className, final String schemeId, + final String schemeName) { final Qualifier q = new Qualifier(); q.setClassid(classId); q.setClassid(className); @@ -159,8 +159,6 @@ public abstract class AbstractScholexplorerParser { q.setSchemename(schemeName); return q; - - } protected void generateRelations( 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 c5c6b1493..60371fa53 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 @@ -148,42 +148,37 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser { inferPid(currentPid); parsedObject.setPid(Collections.singletonList(currentPid)); - String resolvedURL = null; switch (currentPid.getQualifier().getClassname().toLowerCase()) { case "uniprot": - resolvedURL ="https://www.uniprot.org/uniprot/"+currentPid.getValue(); + resolvedURL = "https://www.uniprot.org/uniprot/" + currentPid.getValue(); break; case "ena": if (StringUtils.isNotBlank(currentPid.getValue()) && currentPid.getValue().length() > 7) - resolvedURL ="https://www.ebi.ac.uk/ena/data/view/"+currentPid.getValue().substring(0,8); + resolvedURL = "https://www.ebi.ac.uk/ena/data/view/" + currentPid.getValue().substring(0, 8); break; case "chembl": - resolvedURL ="https://www.ebi.ac.uk/chembl/compound_report_card/"+currentPid.getValue(); + resolvedURL = "https://www.ebi.ac.uk/chembl/compound_report_card/" + currentPid.getValue(); break; case "ncbi-n": - resolvedURL ="https://www.ncbi.nlm.nih.gov/nuccore/"+currentPid.getValue(); + resolvedURL = "https://www.ncbi.nlm.nih.gov/nuccore/" + currentPid.getValue(); break; case "ncbi-p": - resolvedURL ="https://www.ncbi.nlm.nih.gov/nuccore/"+currentPid.getValue(); + resolvedURL = "https://www.ncbi.nlm.nih.gov/nuccore/" + currentPid.getValue(); break; case "genbank": - resolvedURL ="https://www.ncbi.nlm.nih.gov/nuccore/"+currentPid.getValue(); + resolvedURL = "https://www.ncbi.nlm.nih.gov/nuccore/" + currentPid.getValue(); break; case "pdb": - resolvedURL ="https://www.ncbi.nlm.nih.gov/nuccore/"+currentPid.getValue(); + resolvedURL = "https://www.ncbi.nlm.nih.gov/nuccore/" + currentPid.getValue(); break; case "url": - resolvedURL =currentPid.getValue(); + resolvedURL = currentPid.getValue(); break; } - - - - final String sourceId = generateId( currentPid.getValue(), currentPid.getQualifier().getClassid(), "dataset"); parsedObject.setId(sourceId); @@ -286,7 +281,11 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser { t -> { final StructuredProperty st = new StructuredProperty(); st.setValue(t); - st.setQualifier(generateQualifier( "main title","main title", "dnet:dataCite_title","dnet:dataCite_title")); + st + .setQualifier( + generateQualifier( + "main title", "main title", "dnet:dataCite_title", + "dnet:dataCite_title")); return st; }) .collect(Collectors.toList())); @@ -318,8 +317,7 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser { .collect(Collectors.toList())); } - - if(StringUtils.isNotBlank(resolvedURL)) { + if (StringUtils.isNotBlank(resolvedURL)) { Instance i = new Instance(); i.setCollectedfrom(parsedObject.getCollectedfrom().get(0)); i.setUrl(Collections.singletonList(resolvedURL)); 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 b66bfd161..8d76004dc 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 @@ -202,7 +202,11 @@ public class PublicationScholexplorerParser extends AbstractScholexplorerParser t -> { final StructuredProperty st = new StructuredProperty(); st.setValue(t); - st.setQualifier(generateQualifier( "main title","main title", "dnet:dataCite_title","dnet:dataCite_title")); + st + .setQualifier( + generateQualifier( + "main title", "main title", "dnet:dataCite_title", + "dnet:dataCite_title")); return st; }) .collect(Collectors.toList())); diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/oozie_app/config-default.xml index cf617a84c..cac3cc2bb 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/oozie_app/config-default.xml @@ -1,4 +1,7 @@ + + + + + + + + jobTracker + yarn + + + nameNode + hdfs://hadoop-rm1.garr-pa1.d4science.org:8020 + + + hive_metastore_uris + thrift://hadoop-edge3.garr-pa1.d4science.org:9083 + + + spark2YarnHistoryServerAddress + http://hadoop-rm2.garr-pa1.d4science.org:19888 + + + + + oozie.launcher.mapreduce.user.classpath.first + true + + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + spark2EventLogDir /user/spark/spark2ApplicationHistory diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/oozie_app/workflow.xml index b81669674..a5035c56c 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/oozie_app/workflow.xml @@ -18,7 +18,7 @@
- + @@ -26,13 +26,59 @@ + + + yarn-cluster + cluster + Create Baselnie DataSet + + eu.dnetlib.dhp.sx.ebi.SparkCreateBaselineDataFrame + dhp-graph-mapper-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=1 + --driver-memory=${sparkDriverMemory} + --executor-cores=${sparkExecutorCores} + ${sparkExtraOPT} + + --workingPath${workingPath} + --masteryarn + + + + + + + + yarn-cluster + cluster + Create Baselnie DataSet + + eu.dnetlib.dhp.sx.ebi.SparkAddLinkUpdates + dhp-graph-mapper-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=1 + --driver-memory=${sparkDriverMemory} + --executor-cores=${sparkExecutorCores} + ${sparkExtraOPT} + + --workingPath${workingPath} + --masteryarn + + + + + + yarn-cluster cluster Create EBI DataSet + eu.dnetlib.dhp.sx.ebi.SparkCreateEBIDataFrame - dhp-doiboost-${projectVersion}.jar + dhp-graph-mapper-${projectVersion}.jar --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} @@ -41,7 +87,7 @@ ${sparkExtraOPT} --workingPath${workingPath} - --masteryarn-cluster + --masteryarn diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/ebi/TestEBI.scala b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/ebi/TestEBI.scala new file mode 100644 index 000000000..fa390a21b --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/ebi/TestEBI.scala @@ -0,0 +1,20 @@ +package eu.dnetlib.dhp.sx.ebi + +import org.junit.jupiter.api.Test + +class TestEBI { + + + + @Test + def testEBIData() = { + SparkAddLinkUpdates.main("-mt local[*] -w /home/sandro/Downloads".split(" ")) + + + + + + + } + +} diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/ebi/TestEBITODS.scala b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/ebi/TestEBITODS.scala deleted file mode 100644 index 979bf7e14..000000000 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/ebi/TestEBITODS.scala +++ /dev/null @@ -1,22 +0,0 @@ -package eu.dnetlib.dhp.sx.ebi - -import org.junit.jupiter.api.Test - -class TestEBITODS { - - - @Test - def testEBI():Unit = { - - - - - - - - - } - - - -} diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/sx/ebi/rel1.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/sx/ebi/rel1.json new file mode 100644 index 000000000..038b84a49 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/sx/ebi/rel1.json @@ -0,0 +1,55 @@ +{ + "Category": [ + { + "Section": [ + { + "Linklist": { + "Link": [ + { + "LinkProvider": { + "Name": "Europe PMC" + }, + "Target": { + "Publisher": { + "Name": "Altmetric" + }, + "ImageURL": "https://api.altmetric.com/v1/donut/58578459_64.png", + "Identifier": { + "ID": "https://www.altmetric.com/details/58578459", + "IDScheme": "URL", + "IDURL": "https://www.altmetric.com/details/58578459" + }, + "Type": { + "Name": "dataset" + }, + "Title": "Optical clumped isotope thermometry of carbon dioxide" + }, + "Source": { + "Identifier": { + "ID": "30886173", + "IDScheme": "PMID" + }, + "Type": { + "Name": "literature" + } + }, + "PublicationDate": "06-04-2019", + "RelationshipType": { + "Name": "IsReferencedBy" + }, + "ObtainedBy": "ext_links" + } + ] + }, + "ObtainedBy": "ext_links", + "SectionLinkCount": 1, + "Tags": [ + "altmetrics" + ] + } + ], + "CategoryLinkCount": 1, + "Name": "Altmetric" + } + ] +} diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/sx/ebi/rel_multiple.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/sx/ebi/rel_multiple.json new file mode 100644 index 000000000..2ad55861e --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/sx/ebi/rel_multiple.json @@ -0,0 +1,191 @@ +{ + "version": "6.3", + "hitCount": 4, + "request": { + "id": "28818901", + "source": "MED" + }, + "dataLinkList": { + "Category": [ + { + "Name": "Nucleotide Sequences", + "CategoryLinkCount": 3, + "Section": [ + { + "ObtainedBy": "tm_accession", + "Tags": [ + "supporting_data" + ], + "SectionLinkCount": 1, + "Linklist": { + "Link": [ + { + "ObtainedBy": "tm_accession", + "PublicationDate": "27-02-2020", + "LinkProvider": { + "Name": "Europe PMC" + }, + "RelationshipType": { + "Name": "References" + }, + "Source": { + "Type": { + "Name": "literature" + }, + "Identifier": { + "ID": "28818901", + "IDScheme": "MED" + } + }, + "Target": { + "Type": { + "Name": "dataset" + }, + "Identifier": { + "ID": "AP008937", + "IDScheme": "ENA", + "IDURL": "http://identifiers.org/ena.embl/AP008937" + }, + "Title": "AP008937", + "Publisher": { + "Name": "Europe PMC" + } + }, + "Frequency": 1 + } + ] + } + }, + { + "ObtainedBy": "submission", + "Tags": [ + "related_data" + ], + "SectionLinkCount": 2, + "CollectionURL": "http://www.ebi.ac.uk/ena/data/search?query=28818901", + "Linklist": { + "Link": [ + { + "ObtainedBy": "submission", + "PublicationDate": "25-06-2018", + "LinkProvider": { + "Name": "Europe PMC" + }, + "RelationshipType": { + "Name": "IsReferencedBy" + }, + "Source": { + "Type": { + "Name": "literature" + }, + "Identifier": { + "ID": "28818901", + "IDScheme": "PMID" + } + }, + "Target": { + "Type": { + "Name": "dataset" + }, + "Identifier": { + "ID": "NIWV01000000", + "IDScheme": "ENA", + "IDURL": "http://www.ebi.ac.uk/ena/data/view/NIWV01000000" + }, + "Title": "Nucleotide sequences", + "Publisher": { + "Name": "ENA" + } + } + }, + { + "ObtainedBy": "submission", + "PublicationDate": "25-06-2018", + "LinkProvider": { + "Name": "Europe PMC" + }, + "RelationshipType": { + "Name": "IsReferencedBy" + }, + "Source": { + "Type": { + "Name": "literature" + }, + "Identifier": { + "ID": "28818901", + "IDScheme": "PMID" + } + }, + "Target": { + "Type": { + "Name": "dataset" + }, + "Identifier": { + "ID": "PRJNA390617", + "IDScheme": "ENA", + "IDURL": "http://www.ebi.ac.uk/ena/data/view/PRJNA390617" + }, + "Title": "Lactobacillus fermentum strain:BFE 6620", + "Publisher": { + "Name": "ENA" + } + } + } + ] + } + } + ] + }, + { + "Name": "BioStudies: supplemental material and supporting data", + "CategoryLinkCount": 1, + "Section": [ + { + "ObtainedBy": "ext_links", + "Tags": [ + "supporting_data" + ], + "SectionLinkCount": 1, + "Linklist": { + "Link": [ + { + "ObtainedBy": "ext_links", + "PublicationDate": "24-07-2018", + "LinkProvider": { + "Name": "Europe PMC" + }, + "RelationshipType": { + "Name": "IsReferencedBy" + }, + "Source": { + "Type": { + "Name": "literature" + }, + "Identifier": { + "ID": "28818901", + "IDScheme": "PMID" + } + }, + "Target": { + "Type": { + "Name": "dataset" + }, + "Identifier": { + "ID": "http://www.ebi.ac.uk/biostudies/studies/S-EPMC5604774?xr=true", + "IDScheme": "URL", + "IDURL": "http://www.ebi.ac.uk/biostudies/studies/S-EPMC5604774?xr=true" + }, + "Title": "Draft Genome Sequence of Lactobacillus fermentum BFE 6620, a Potential Starter Culture for African Vegetable Foods, Isolated from Fermented Cassava.", + "Publisher": { + "Name": "BioStudies: supplemental material and supporting data" + } + } + } + ] + } + } + ] + } + ] + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/DLIToOAF.scala b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/DLIToOAF.scala index 967834d69..86b68fbd2 100644 --- a/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/DLIToOAF.scala +++ b/dhp-workflows/dhp-graph-provision-scholexplorer/src/main/java/eu/dnetlib/dhp/export/DLIToOAF.scala @@ -5,11 +5,12 @@ import java.time.format.DateTimeFormatter import eu.dnetlib.dhp.common.PacePerson import eu.dnetlib.dhp.schema.action.AtomicAction -import eu.dnetlib.dhp.schema.oaf.{Author, DataInfo, Dataset, ExternalReference, Field, Instance, KeyValue, Oaf, Publication, Qualifier, Relation, Result, StructuredProperty} +import eu.dnetlib.dhp.schema.oaf.{Author, Dataset, ExternalReference, Field, Instance, KeyValue, Oaf, Publication, Qualifier, Relation, Result, StructuredProperty} import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation} import eu.dnetlib.dhp.utils.DHPUtils import org.apache.commons.lang3.StringUtils import org.codehaus.jackson.map.ObjectMapper +import eu.dnetlib.dhp.schema.scholexplorer.OafUtils._ import scala.collection.JavaConverters._ @@ -426,46 +427,6 @@ object DLIToOAF { } - def generateKeyValue(key: String, value: String): KeyValue = { - val kv: KeyValue = new KeyValue() - kv.setKey(key) - kv.setValue(value) - kv.setDataInfo(generateDataInfo("0.9")) - kv - } - def generateDataInfo(trust: String = "0.9", invisibile: Boolean = false): DataInfo = { - val di = new DataInfo - di.setDeletedbyinference(false) - di.setInferred(false) - di.setInvisible(false) - di.setTrust(trust) - di.setProvenanceaction(createQualifier("sysimport:actionset", "dnet:provenanceActions")) - di - } - - def createQualifier(cls: String, sch: String): Qualifier = { - createQualifier(cls, cls, sch, sch) - } - - - def createQualifier(classId: String, className: String, schemeId: String, schemeName: String): Qualifier = { - val q: Qualifier = new Qualifier - q.setClassid(classId) - q.setClassname(className) - q.setSchemeid(schemeId) - q.setSchemename(schemeName) - q - } - - - def asField[T](value: T): Field[T] = { - val tmp = new Field[T] - tmp.setValue(value) - tmp - - - } - } diff --git a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJobTest.java b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJobTest.java index c16bbc6fb..528532edd 100644 --- a/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJobTest.java +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJobTest.java @@ -1,9 +1,10 @@ package eu.dnetlib.dhp.oa.provision; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport; -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.function.FilterFunction; @@ -19,9 +20,10 @@ import org.junit.jupiter.api.io.TempDir; 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 eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport; +import eu.dnetlib.dhp.schema.oaf.Relation; public class PrepareRelationsJobTest { @@ -74,14 +76,19 @@ public class PrepareRelationsJobTest { "-maxRelations", String.valueOf(maxRelations) }); - Dataset out = spark.read() - .parquet(testPath.toString()) - .as(Encoders.bean(Relation.class)) - .cache(); + Dataset out = spark + .read() + .parquet(testPath.toString()) + .as(Encoders.bean(Relation.class)) + .cache(); Assertions.assertEquals(10, out.count()); - Dataset freq = out.toDF().cube(SUBRELTYPE).count().filter((FilterFunction) value -> !value.isNullAt(0)); + Dataset freq = out + .toDF() + .cube(SUBRELTYPE) + .count() + .filter((FilterFunction) value -> !value.isNullAt(0)); long outcome = freq.filter(freq.col(SUBRELTYPE).equalTo(OUTCOME)).collectAsList().get(0).getAs("count"); long supplement = freq.filter(freq.col(SUBRELTYPE).equalTo(SUPPLEMENT)).collectAsList().get(0).getAs("count"); From 752d28f8eb955128eb59c5f71e3f0b652f003319 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 10 Jul 2020 15:09:50 +0200 Subject: [PATCH 030/100] make the relations produced by the dedup SparkPropagateRelation jon unique --- .../dhp/oa/dedup/SparkPropagateRelation.java | 45 ++++++++++++++++++- 1 file changed, 44 insertions(+), 1 deletion(-) 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 516808511..e65eb7ab5 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 @@ -7,6 +7,7 @@ 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.expressions.Aggregator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,7 +96,49 @@ public class SparkPropagateRelation extends AbstractSparkAction { FieldType.TARGET, getDeletedFn()); - save(newRels.union(updated).union(mergeRels), outputRelationPath, SaveMode.Overwrite); + save(distinctRelations(newRels.union(updated).union(mergeRels)), outputRelationPath, SaveMode.Overwrite); + } + + private Dataset distinctRelations(Dataset rels) { + return rels + .groupByKey((MapFunction) r -> ModelSupport.idFn().apply(r), Encoders.STRING()) + .agg(new RelationAggregator().toColumn()) + .map((MapFunction, Relation>) t -> t._2(), Encoders.bean(Relation.class)); + } + + class RelationAggregator extends Aggregator { + + @Override + public Relation zero() { + return new Relation(); + } + + @Override + public Relation reduce(Relation b, Relation a) { + b.mergeFrom(a); + return b; + } + + @Override + public Relation merge(Relation b, Relation a) { + b.mergeFrom(a); + return b; + } + + @Override + public Relation finish(Relation r) { + return r; + } + + @Override + public Encoder bufferEncoder() { + return Encoders.bean(Relation.class); + } + + @Override + public Encoder outputEncoder() { + return Encoders.bean(Relation.class); + } } private static Dataset processDataset( From e1ae964bc462e532cd6e61369899677941dc2a4a Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Fri, 10 Jul 2020 16:12:08 +0200 Subject: [PATCH 031/100] stats --- .../dhp/broker/oa/GenerateStatsJob.java | 63 +++++++++++++++++++ .../dnetlib/dhp/broker/oa/JoinStep2Job.java | 5 +- .../aggregators/stats/DatasourceStats.java | 61 ++++++++++++++++++ .../aggregators/stats/StatsAggregator.java | 59 +++++++++++++++++ .../broker/oa/partial/oozie_app/workflow.xml | 8 +-- .../dhp/oa/dedup/EntityMergerTest.java | 5 +- .../oa/graph/clean/CleanGraphSparkJob.java | 5 +- 7 files changed, 194 insertions(+), 12 deletions(-) create mode 100644 dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateStatsJob.java create mode 100644 dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/stats/DatasourceStats.java create mode 100644 dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/stats/StatsAggregator.java diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateStatsJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateStatsJob.java new file mode 100644 index 000000000..a51601cd7 --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/GenerateStatsJob.java @@ -0,0 +1,63 @@ + +package eu.dnetlib.dhp.broker.oa; + +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.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.TypedColumn; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.broker.model.Event; +import eu.dnetlib.dhp.broker.oa.util.ClusterUtils; +import eu.dnetlib.dhp.broker.oa.util.aggregators.stats.DatasourceStats; +import eu.dnetlib.dhp.broker.oa.util.aggregators.stats.StatsAggregator; + +public class GenerateStatsJob { + + private static final Logger log = LoggerFactory.getLogger(GenerateStatsJob.class); + + public static void main(final String[] args) throws Exception { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + IndexOnESJob.class + .getResourceAsStream("/eu/dnetlib/dhp/broker/oa/common_params.json"))); + parser.parseArgument(args); + + final Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + final SparkConf conf = new SparkConf(); + + final String eventsPath = parser.get("workingPath") + "/events"; + log.info("eventsPath: {}", eventsPath); + + final String statsPath = parser.get("workingPath") + "/stats"; + log.info("stats: {}", statsPath); + + final TypedColumn aggr = new StatsAggregator().toColumn(); + + runWithSparkSession(conf, isSparkSessionManaged, spark -> { + + final Dataset stats = ClusterUtils + .readPath(spark, eventsPath, Event.class) + .groupByKey(e -> e.getMap().getTargetDatasourceId(), Encoders.STRING()) + .agg(aggr) + .map(t -> t._2, Encoders.bean(DatasourceStats.class)); + + ClusterUtils.save(stats, statsPath, DatasourceStats.class, null); + }); + } + +} diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep2Job.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep2Job.java index cdcf0add4..55ab497f0 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep2Job.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep2Job.java @@ -7,7 +7,6 @@ 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.TypedColumn; @@ -65,9 +64,7 @@ public class JoinStep2Job { final Dataset dataset = sources .joinWith(typedRels, sources.col("openaireId").equalTo(typedRels.col("source")), "left_outer") - .groupByKey( - (MapFunction, String>) t -> t._1.getOpenaireId(), - Encoders.STRING()) + .groupByKey(t -> t._1.getOpenaireId(), Encoders.STRING()) .agg(aggr) .map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class)); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/stats/DatasourceStats.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/stats/DatasourceStats.java new file mode 100644 index 000000000..8b628809d --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/stats/DatasourceStats.java @@ -0,0 +1,61 @@ + +package eu.dnetlib.dhp.broker.oa.util.aggregators.stats; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +public class DatasourceStats implements Serializable { + + /** + * + */ + private static final long serialVersionUID = -282112564184047677L; + + private String id; + private String name; + private String type; + private Map topics = new HashMap<>(); + + public String getId() { + return id; + } + + public void setId(final String id) { + this.id = id; + } + + public String getName() { + return name; + } + + public void setName(final String name) { + this.name = name; + } + + public String getType() { + return type; + } + + public void setType(final String type) { + this.type = type; + } + + public Map getTopics() { + return topics; + } + + public void setTopics(final Map topics) { + this.topics = topics; + } + + public void incrementTopic(final String topic, final long inc) { + if (topics.containsKey(topic)) { + topics.put(topic, topics.get(topic) + inc); + } else { + topics.put(topic, inc); + } + + } + +} diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/stats/StatsAggregator.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/stats/StatsAggregator.java new file mode 100644 index 000000000..5aa6698e3 --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/stats/StatsAggregator.java @@ -0,0 +1,59 @@ + +package eu.dnetlib.dhp.broker.oa.util.aggregators.stats; + +import org.apache.commons.lang.StringUtils; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.expressions.Aggregator; + +import eu.dnetlib.dhp.broker.model.Event; + +public class StatsAggregator extends Aggregator { + + /** + * + */ + private static final long serialVersionUID = 6652105853037330529L; + + @Override + public DatasourceStats zero() { + return new DatasourceStats(); + } + + @Override + public DatasourceStats reduce(final DatasourceStats stats, final Event e) { + stats.setId(e.getMap().getTargetDatasourceId()); + stats.setName(e.getMap().getTargetDatasourceName()); + stats.setType(e.getMap().getTargetDatasourceType()); + stats.incrementTopic(e.getTopic(), 1l); + return stats; + } + + @Override + public DatasourceStats merge(final DatasourceStats stats0, final DatasourceStats stats1) { + if (StringUtils.isBlank(stats0.getId())) { + stats0.setId(stats1.getId()); + stats0.setName(stats1.getName()); + stats0.setType(stats1.getType()); + } + stats1.getTopics().entrySet().forEach(e -> stats0.incrementTopic(e.getKey(), e.getValue())); + return stats0; + } + + @Override + public Encoder bufferEncoder() { + return Encoders.bean(DatasourceStats.class); + + } + + @Override + public DatasourceStats finish(final DatasourceStats stats) { + return stats; + } + + @Override + public Encoder outputEncoder() { + return Encoders.bean(DatasourceStats.class); + + } +} diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml index d19ad6c5a..b4155f93f 100644 --- a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml @@ -64,19 +64,19 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + yarn cluster - Count - eu.dnetlib.dhp.broker.oa.CheckDuplictedIdsJob + GenerateStatsJob + eu.dnetlib.dhp.broker.oa.GenerateStatsJob dhp-broker-events-${projectVersion}.jar --executor-cores=${sparkExecutorCores} diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java index 513e14f07..3d45f666b 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java @@ -47,10 +47,11 @@ public class EntityMergerTest implements Serializable { @Test public void softwareMergerTest() throws InstantiationException, IllegalAccessException { - List> softwares = readSample(testEntityBasePath + "/software_merge.json", Software.class); + List> softwares = readSample( + testEntityBasePath + "/software_merge.json", Software.class); Software merged = DedupRecordFactory - .entityMerger(dedupId, softwares.iterator(), 0, dataInfo, Software.class); + .entityMerger(dedupId, softwares.iterator(), 0, dataInfo, Software.class); System.out.println(merged.getBestaccessright().getClassid()); } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleanGraphSparkJob.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleanGraphSparkJob.java index fd707e949..7091d9740 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleanGraphSparkJob.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleanGraphSparkJob.java @@ -8,7 +8,6 @@ import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; -import eu.dnetlib.dhp.oa.graph.raw.AbstractMdRecordToOafMapper; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.spark.SparkConf; @@ -24,6 +23,7 @@ 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.AbstractMdRecordToOafMapper; import eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils; import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyGroup; import eu.dnetlib.dhp.schema.common.ModelConstants; @@ -151,7 +151,8 @@ public class CleanGraphSparkJob { if (Objects.isNull(r.getBestaccessright()) || StringUtils.isBlank(r.getBestaccessright().getClassid())) { Qualifier bestaccessrights = AbstractMdRecordToOafMapper.createBestAccessRights(r.getInstance()); if (Objects.isNull(bestaccessrights)) { - r.setBestaccessright( + r + .setBestaccessright( qualifier("UNKNOWN", "not available", ModelConstants.DNET_ACCESS_MODES)); } else { r.setBestaccessright(bestaccessrights); From 4c3836f62e3358d173163aef8de11ead52fcc707 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 10 Jul 2020 19:00:44 +0200 Subject: [PATCH 032/100] materialize the related entities before joining them --- .../CreateRelatedEntitiesJob_phase1.java | 25 ++++++++++++++++--- .../CreateRelatedEntitiesJob_phase2.java | 17 +++++-------- .../dhp/oa/provision/ProvisionConstants.java | 14 +++++++++++ 3 files changed, 42 insertions(+), 14 deletions(-) create mode 100644 dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/ProvisionConstants.java 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 80b800017..57dca7bb1 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 @@ -9,6 +9,7 @@ import java.util.Optional; import java.util.stream.Collectors; 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; @@ -115,11 +116,21 @@ public class CreateRelatedEntitiesJob_phase1 { Encoders.tuple(Encoders.STRING(), Encoders.kryo(Relation.class))) .cache(); - Dataset> entities = readPathEntity(spark, inputEntityPath, clazz) + final String relatedEntityPath = outputPath + "_relatedEntity"; + readPathEntity(spark, inputEntityPath, clazz) .filter("dataInfo.invisible == false") .map( (MapFunction) value -> asRelatedEntity(value, clazz), Encoders.kryo(RelatedEntity.class)) + .repartition(5000) + .write() + .mode(SaveMode.Overwrite) + .parquet(relatedEntityPath); + + Dataset> entities = spark + .read() + .load(relatedEntityPath) + .as(Encoders.kryo(RelatedEntity.class)) .map( (MapFunction>) e -> new Tuple2<>(e.getId(), e), Encoders.tuple(Encoders.STRING(), Encoders.kryo(RelatedEntity.class))) @@ -165,13 +176,21 @@ public class CreateRelatedEntitiesJob_phase1 { Result result = (Result) entity; if (result.getTitle() != null && !result.getTitle().isEmpty()) { - re.setTitle(result.getTitle().stream().findFirst().get()); + final StructuredProperty title = result.getTitle().stream().findFirst().get(); + title.setValue(StringUtils.left(title.getValue(), ProvisionConstants.MAX_TITLE_LENGTH)); + re.setTitle(title); } re.setDateofacceptance(getValue(result.getDateofacceptance())); re.setPublisher(getValue(result.getPublisher())); re.setResulttype(result.getResulttype()); - re.setInstances(result.getInstance()); + re + .setInstances( + result + .getInstance() + .stream() + .limit(ProvisionConstants.MAX_INSTANCES) + .collect(Collectors.toList())); // TODO still to be mapped // re.setCodeRepositoryUrl(j.read("$.coderepositoryurl")); 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 bfcc648a3..7e175121e 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 @@ -61,12 +61,6 @@ public class CreateRelatedEntitiesJob_phase2 { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final int MAX_EXTERNAL_ENTITIES = 50; - private static final int MAX_AUTHORS = 200; - private static final int MAX_AUTHOR_FULLNAME_LENGTH = 1000; - private static final int MAX_TITLE_LENGTH = 5000; - private static final int MAX_ABSTRACT_LENGTH = 100000; - public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils @@ -246,15 +240,15 @@ public class CreateRelatedEntitiesJob_phase2 { List refs = r .getExternalReference() .stream() - .limit(MAX_EXTERNAL_ENTITIES) + .limit(ProvisionConstants.MAX_EXTERNAL_ENTITIES) .collect(Collectors.toList()); r.setExternalReference(refs); } if (r.getAuthor() != null) { List authors = Lists.newArrayList(); for (Author a : r.getAuthor()) { - a.setFullname(StringUtils.left(a.getFullname(), MAX_AUTHOR_FULLNAME_LENGTH)); - if (authors.size() < MAX_AUTHORS || hasORCID(a)) { + a.setFullname(StringUtils.left(a.getFullname(), ProvisionConstants.MAX_AUTHOR_FULLNAME_LENGTH)); + if (authors.size() < ProvisionConstants.MAX_AUTHORS || hasORCID(a)) { authors.add(a); } } @@ -266,7 +260,7 @@ public class CreateRelatedEntitiesJob_phase2 { .stream() .filter(Objects::nonNull) .map(d -> { - d.setValue(StringUtils.left(d.getValue(), MAX_ABSTRACT_LENGTH)); + d.setValue(StringUtils.left(d.getValue(), ProvisionConstants.MAX_ABSTRACT_LENGTH)); return d; }) .collect(Collectors.toList()); @@ -278,9 +272,10 @@ public class CreateRelatedEntitiesJob_phase2 { .stream() .filter(Objects::nonNull) .map(t -> { - t.setValue(StringUtils.left(t.getValue(), MAX_TITLE_LENGTH)); + t.setValue(StringUtils.left(t.getValue(), ProvisionConstants.MAX_TITLE_LENGTH)); return t; }) + .limit(ProvisionConstants.MAX_TITLES) .collect(Collectors.toList()); r.setTitle(titles); } diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/ProvisionConstants.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/ProvisionConstants.java new file mode 100644 index 000000000..9bc3706cd --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/ProvisionConstants.java @@ -0,0 +1,14 @@ + +package eu.dnetlib.dhp.oa.provision; + +public class ProvisionConstants { + + public static final int MAX_EXTERNAL_ENTITIES = 50; + public static final int MAX_AUTHORS = 200; + public static final int MAX_AUTHOR_FULLNAME_LENGTH = 1000; + public static final int MAX_TITLE_LENGTH = 5000; + public static final int MAX_TITLES = 10; + public static final int MAX_ABSTRACT_LENGTH = 100000; + public static final int MAX_INSTANCES = 10; + +} From cc77446dc4d092503bff388994a005e17e61dcfc Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 10 Jul 2020 19:01:50 +0200 Subject: [PATCH 033/100] added dbSchema parameter to the raw_db workflow --- .../eu/dnetlib/dhp/oa/graph/raw_db/oozie_app/workflow.xml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_db/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_db/oozie_app/workflow.xml index 575f9229e..eea8d0a5a 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_db/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_db/oozie_app/workflow.xml @@ -16,6 +16,11 @@ postgresPassword the password postgres
+ + dbSchema + beta + the database schema according to the D-Net infrastructure (beta or production) + isLookupUrl the address of the lookUp service @@ -93,6 +98,7 @@ --postgresUser${postgresUser} --postgresPassword${postgresPassword} --isLookupUrl${isLookupUrl} + --dbschema${dbSchema} @@ -109,6 +115,7 @@ --postgresUser${postgresUser} --postgresPassword${postgresPassword} --isLookupUrl${isLookupUrl} + --dbschema${dbSchema} --actionclaims From 06c1913062541062b4b5ed49044870acc1d3bd56 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 10 Jul 2020 19:03:33 +0200 Subject: [PATCH 034/100] added different limits for grouping by source and by target, incremented spark.sql.shuffle.partitions for the join operations --- .../dhp/oa/provision/oozie_app/workflow.xml | 40 ++++++++++--------- 1 file changed, 22 insertions(+), 18 deletions(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml index faa81ad64..32bf7ce83 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml @@ -18,8 +18,12 @@ filter applied reading relations (by relClass) - maxRelations - maximum number of relations allowed for a each entity + sourceMaxRelations + maximum number of relations allowed for a each entity grouping by source + + + targetMaxRelations + maximum number of relations allowed for a each entity grouping by target otherDsTypeId @@ -133,7 +137,8 @@ --inputRelationsPath${inputGraphRootPath}/relation --outputPath${workingDir}/relation - --maxRelations${maxRelations} + --sourceMaxRelations${sourceMaxRelations} + --targetMaxRelations${targetMaxRelations} --relationFilter${relationFilter} --relPartitions5000 @@ -166,7 +171,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=7680 + --conf spark.sql.shuffle.partitions=15000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputRelationsPath${workingDir}/relation @@ -193,7 +198,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 + --conf spark.sql.shuffle.partitions=15000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputRelationsPath${workingDir}/relation @@ -220,7 +225,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 + --conf spark.sql.shuffle.partitions=10000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputRelationsPath${workingDir}/relation @@ -247,7 +252,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 + --conf spark.sql.shuffle.partitions=5000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputRelationsPath${workingDir}/relation @@ -274,7 +279,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 + --conf spark.sql.shuffle.partitions=5000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputRelationsPath${workingDir}/relation @@ -301,7 +306,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 + --conf spark.sql.shuffle.partitions=5000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputRelationsPath${workingDir}/relation @@ -328,7 +333,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 + --conf spark.sql.shuffle.partitions=5000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputRelationsPath${workingDir}/relation @@ -367,7 +372,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=15360 + --conf spark.sql.shuffle.partitions=15000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputEntityPath${inputGraphRootPath}/publication @@ -395,7 +400,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=7680 + --conf spark.sql.shuffle.partitions=10000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputEntityPath${inputGraphRootPath}/dataset @@ -423,7 +428,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=7680 + --conf spark.sql.shuffle.partitions=10000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputEntityPath${inputGraphRootPath}/otherresearchproduct @@ -451,7 +456,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 + --conf spark.sql.shuffle.partitions=5000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputEntityPath${inputGraphRootPath}/software @@ -479,7 +484,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=7680 + --conf spark.sql.shuffle.partitions=8000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputEntityPath${inputGraphRootPath}/datasource @@ -507,7 +512,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=7680 + --conf spark.sql.shuffle.partitions=10000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputEntityPath${inputGraphRootPath}/organization @@ -535,7 +540,7 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 + --conf spark.sql.shuffle.partitions=5000 --conf spark.network.timeout=${sparkNetworkTimeout} --inputEntityPath${inputGraphRootPath}/project @@ -607,5 +612,4 @@ - \ No newline at end of file From 770adc26e9d509837710de1d15bdeab88ed675a0 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 10 Jul 2020 19:35:10 +0200 Subject: [PATCH 035/100] WIP aggregator to make relationships unique --- .../dnetlib/dhp/oa/dedup/SparkPropagateRelation.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) 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 e65eb7ab5..13d2e4cd7 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 @@ -96,14 +96,14 @@ public class SparkPropagateRelation extends AbstractSparkAction { FieldType.TARGET, getDeletedFn()); - save(distinctRelations(newRels.union(updated).union(mergeRels)), outputRelationPath, SaveMode.Overwrite); + save(newRels.union(updated).union(mergeRels).distinct(), outputRelationPath, SaveMode.Overwrite); } private Dataset distinctRelations(Dataset rels) { return rels - .groupByKey((MapFunction) r -> ModelSupport.idFn().apply(r), Encoders.STRING()) - .agg(new RelationAggregator().toColumn()) - .map((MapFunction, Relation>) t -> t._2(), Encoders.bean(Relation.class)); + .groupByKey((MapFunction) r -> ModelSupport.idFn().apply(r), Encoders.STRING()) + .agg(new RelationAggregator().toColumn()) + .map((MapFunction, Relation>) t -> t._2(), Encoders.bean(Relation.class)); } class RelationAggregator extends Aggregator { @@ -115,13 +115,11 @@ public class SparkPropagateRelation extends AbstractSparkAction { @Override public Relation reduce(Relation b, Relation a) { - b.mergeFrom(a); return b; } @Override public Relation merge(Relation b, Relation a) { - b.mergeFrom(a); return b; } From b7a39731a6385457e4defd7eeac08edf208266a8 Mon Sep 17 00:00:00 2001 From: Alessia Bardi Date: Sun, 12 Jul 2020 19:28:56 +0200 Subject: [PATCH 036/100] assert, not print --- .../src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java index 513e14f07..0b718ecdc 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java @@ -52,7 +52,7 @@ public class EntityMergerTest implements Serializable { Software merged = DedupRecordFactory .entityMerger(dedupId, softwares.iterator(), 0, dataInfo, Software.class); - System.out.println(merged.getBestaccessright().getClassid()); + assertEquals(merged.getBestaccessright().getClassid(), "OPEN SOURCE"); } @Test From 7a3fd9f54cc7cf803e799fc430b0eb898e33f3c6 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 13 Jul 2020 10:11:36 +0200 Subject: [PATCH 037/100] dedup relation aggregator moved into dedicated class --- .../dhp/oa/dedup/RelationAggregator.java | 46 +++++++++++++++++++ .../dhp/oa/dedup/SparkPropagateRelation.java | 36 +-------------- 2 files changed, 47 insertions(+), 35 deletions(-) create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/RelationAggregator.java diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/RelationAggregator.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/RelationAggregator.java new file mode 100644 index 000000000..0a29aa51b --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/RelationAggregator.java @@ -0,0 +1,46 @@ + +package eu.dnetlib.dhp.oa.dedup; + +import java.util.Objects; + +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.expressions.Aggregator; + +import eu.dnetlib.dhp.schema.oaf.Relation; + +public class RelationAggregator extends Aggregator { + + private static Relation ZERO = new Relation(); + + @Override + public Relation zero() { + return ZERO; + } + + @Override + public Relation reduce(Relation b, Relation a) { + return Objects.equals(a, ZERO) ? b : a; + } + + @Override + public Relation merge(Relation b, Relation a) { + b.mergeFrom(a); + return b; + } + + @Override + public Relation finish(Relation r) { + return r; + } + + @Override + public Encoder bufferEncoder() { + return Encoders.bean(Relation.class); + } + + @Override + public Encoder outputEncoder() { + return Encoders.bean(Relation.class); + } +} 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 13d2e4cd7..c19769749 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 @@ -7,7 +7,6 @@ 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.expressions.Aggregator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -96,7 +95,7 @@ public class SparkPropagateRelation extends AbstractSparkAction { FieldType.TARGET, getDeletedFn()); - save(newRels.union(updated).union(mergeRels).distinct(), outputRelationPath, SaveMode.Overwrite); + save(distinctRelations(newRels.union(updated).union(mergeRels)), outputRelationPath, SaveMode.Overwrite); } private Dataset distinctRelations(Dataset rels) { @@ -106,39 +105,6 @@ public class SparkPropagateRelation extends AbstractSparkAction { .map((MapFunction, Relation>) t -> t._2(), Encoders.bean(Relation.class)); } - class RelationAggregator extends Aggregator { - - @Override - public Relation zero() { - return new Relation(); - } - - @Override - public Relation reduce(Relation b, Relation a) { - return b; - } - - @Override - public Relation merge(Relation b, Relation a) { - return b; - } - - @Override - public Relation finish(Relation r) { - return r; - } - - @Override - public Encoder bufferEncoder() { - return Encoders.bean(Relation.class); - } - - @Override - public Encoder outputEncoder() { - return Encoders.bean(Relation.class); - } - } - private static Dataset processDataset( Dataset rels, Dataset> mergedIds, From ccbe5c565860f787d200226e4d6d44ef73fda0e5 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Mon, 13 Jul 2020 10:55:27 +0200 Subject: [PATCH 038/100] fixed import of eu.dnetlib.dhp:dnet-openaire-broker-common --- dhp-workflows/dhp-broker-events/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-broker-events/pom.xml b/dhp-workflows/dhp-broker-events/pom.xml index 1a219c5c9..6787226a8 100644 --- a/dhp-workflows/dhp-broker-events/pom.xml +++ b/dhp-workflows/dhp-broker-events/pom.xml @@ -59,7 +59,7 @@ eu.dnetlib.dhp dnet-openaire-broker-common - [3.0.0-SNAPSHOT,) + [3.0.0,) From d561b2dd210eece82b30311d19d7746adad8a497 Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Mon, 13 Jul 2020 14:18:42 +0200 Subject: [PATCH 039/100] implemented cut of connected component --- .../dhp/oa/dedup/SparkCreateMergeRels.java | 19 ++++++++++++++++++- .../oa/dedup/graph/ConnectedComponent.java | 9 ++++++++- .../dhp/oa/dedup/graph/GraphProcessor.scala | 8 ++++---- .../dhp/oa/dedup/createCC_parameters.json | 7 +++++++ 4 files changed, 37 insertions(+), 6 deletions(-) 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 19e60b520..0c31f5fa2 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 @@ -76,6 +76,19 @@ public class SparkCreateMergeRels extends AbstractSparkAction { final String workingPath = parser.get("workingPath"); final String isLookUpUrl = parser.get("isLookUpUrl"); final String actionSetId = parser.get("actionSetId"); + int cut = 0; + try { + cut = Integer.parseInt(parser.get("cutConnectedComponent")); + + + } catch (Throwable e) { + log.error("unable to parse "+parser.get(" cut-off threshold")); + } + + + + + log.info("graphBasePath: '{}'", graphBasePath); log.info("isLookUpUrl: '{}'", isLookUpUrl); @@ -112,7 +125,7 @@ public class SparkCreateMergeRels extends AbstractSparkAction { final Dataset mergeRels = spark .createDataset( GraphProcessor - .findCCs(vertexes.rdd(), edgeRdd, maxIterations) + .findCCs(vertexes.rdd(), edgeRdd, maxIterations, cut) .toJavaRDD() .filter(k -> k.getDocIds().size() > 1) .flatMap(cc -> ccToMergeRel(cc, dedupConf)) @@ -120,6 +133,10 @@ public class SparkCreateMergeRels extends AbstractSparkAction { Encoders.bean(Relation.class)); mergeRels.write().mode(SaveMode.Append).parquet(mergeRelPath); + + + + } } 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 bfd2c25e2..fd6e70916 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 @@ -4,6 +4,7 @@ package eu.dnetlib.dhp.oa.dedup.graph; import java.io.IOException; import java.io.Serializable; import java.util.Set; +import java.util.stream.Collectors; import org.apache.commons.lang.StringUtils; import org.codehaus.jackson.annotate.JsonIgnore; @@ -21,9 +22,14 @@ public class ConnectedComponent implements Serializable { public ConnectedComponent() { } - public ConnectedComponent(Set docIds) { + + public ConnectedComponent(Set docIds, final int cut) { this.docIds = docIds; createID(); + if (cut > 0 && docIds.size() > cut){ + docIds = docIds.stream().filter(s -> !ccId.equalsIgnoreCase(s)).limit(cut -1).collect(Collectors.toSet()); + docIds.add(ccId); + } } public String createID() { @@ -41,6 +47,7 @@ public class ConnectedComponent implements Serializable { public String getMin() { final StringBuilder min = new StringBuilder(); + docIds .forEach( i -> { diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/graph/GraphProcessor.scala b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/graph/GraphProcessor.scala index e19bb7ff5..f4dd85d75 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/graph/GraphProcessor.scala +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/graph/GraphProcessor.scala @@ -7,7 +7,7 @@ import scala.collection.JavaConversions; object GraphProcessor { - def findCCs(vertexes: RDD[(VertexId, String)], edges: RDD[Edge[String]], maxIterations: Int): RDD[ConnectedComponent] = { + def findCCs(vertexes: RDD[(VertexId, String)], edges: RDD[Edge[String]], maxIterations: Int, cut:Int): RDD[ConnectedComponent] = { val graph: Graph[String, String] = Graph(vertexes, edges).partitionBy(PartitionStrategy.RandomVertexCut) //TODO remember to remove partitionby val cc = graph.connectedComponents(maxIterations).vertices @@ -22,15 +22,15 @@ object GraphProcessor { } } val connectedComponents = joinResult.groupByKey() - .map[ConnectedComponent](cc => asConnectedComponent(cc)) + .map[ConnectedComponent](cc => asConnectedComponent(cc, cut)) connectedComponents } - def asConnectedComponent(group: (VertexId, Iterable[String])): ConnectedComponent = { + def asConnectedComponent(group: (VertexId, Iterable[String]), cut:Int): ConnectedComponent = { val docs = group._2.toSet[String] - val connectedComponent = new ConnectedComponent(JavaConversions.setAsJavaSet[String](docs)); + val connectedComponent = new ConnectedComponent(JavaConversions.setAsJavaSet[String](docs), cut); connectedComponent } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json index 6eedd5432..9350cf22b 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json @@ -17,6 +17,13 @@ "paramDescription": "the url for the lookup service", "paramRequired": true }, + { + "paramName": "cc", + "paramLongName": "cutConnectedComponent", + "paramDescription": "the number of maximum elements that belongs to a connected components", + "paramRequired": false + } +, { "paramName": "w", "paramLongName": "workingPath", From 9ef23850228ab7ece1d9e71c83fde76dc933c9cc Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Mon, 13 Jul 2020 15:28:17 +0200 Subject: [PATCH 040/100] implemented test for cut of connected component --- .../dhp/oa/dedup/SparkCreateMergeRels.java | 11 +- .../oa/dedup/graph/ConnectedComponent.java | 14 +-- .../dhp/oa/dedup/createCC_parameters.json | 3 +- .../dnetlib/dhp/oa/dedup/SparkDedupTest.java | 102 +++++++++++++++++- 4 files changed, 107 insertions(+), 23 deletions(-) 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 0c31f5fa2..0b44935d0 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 @@ -80,16 +80,10 @@ public class SparkCreateMergeRels extends AbstractSparkAction { try { cut = Integer.parseInt(parser.get("cutConnectedComponent")); - } catch (Throwable e) { - log.error("unable to parse "+parser.get(" cut-off threshold")); + log.error("unable to parse " + parser.get(" cut-off threshold")); } - - - - - log.info("graphBasePath: '{}'", graphBasePath); log.info("isLookUpUrl: '{}'", isLookUpUrl); log.info("actionSetId: '{}'", actionSetId); @@ -134,9 +128,6 @@ public class SparkCreateMergeRels extends AbstractSparkAction { mergeRels.write().mode(SaveMode.Append).parquet(mergeRelPath); - - - } } 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 fd6e70916..cd4f99f63 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 @@ -19,16 +19,16 @@ public class ConnectedComponent implements Serializable { private Set docIds; private String ccId; - public ConnectedComponent() { - } - - public ConnectedComponent(Set docIds, final int cut) { this.docIds = docIds; createID(); - if (cut > 0 && docIds.size() > cut){ - docIds = docIds.stream().filter(s -> !ccId.equalsIgnoreCase(s)).limit(cut -1).collect(Collectors.toSet()); - docIds.add(ccId); + if (cut > 0 && docIds.size() > cut) { + this.docIds = docIds + .stream() + .filter(s -> !ccId.equalsIgnoreCase(s)) + .limit(cut - 1) + .collect(Collectors.toSet()); + this.docIds.add(ccId); } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json index 9350cf22b..b1df08535 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json @@ -22,8 +22,7 @@ "paramLongName": "cutConnectedComponent", "paramDescription": "the number of maximum elements that belongs to a connected components", "paramRequired": false - } -, + }, { "paramName": "w", "paramLongName": "workingPath", 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 88d5f24f9..66f0af176 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 @@ -3,6 +3,8 @@ package eu.dnetlib.dhp.oa.dedup; import static java.nio.file.Files.createTempDirectory; +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.count; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.lenient; @@ -11,6 +13,9 @@ import java.io.IOException; import java.io.Serializable; import java.net.URISyntaxException; import java.nio.file.Paths; +import java.util.HashSet; +import java.util.Set; +import java.util.stream.Collectors; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; @@ -18,6 +23,7 @@ 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.FilterFunction; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.api.java.function.PairFunction; import org.apache.spark.sql.Dataset; @@ -190,6 +196,94 @@ public class SparkDedupTest implements Serializable { @Test @Order(2) + public void cutMergeRelsTest() 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, + "-cc", + "3" + }); + + new SparkCreateMergeRels(parser, spark).run(isLookUpService); + + long orgs_mergerel = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel") + .as(Encoders.bean(Relation.class)) + .filter((FilterFunction) r -> r.getRelClass().equalsIgnoreCase("merges")) + .groupBy("source") + .agg(count("target").alias("cnt")) + .select("source", "cnt") + .where("cnt > 3") + .count(); + + long pubs_mergerel = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/publication_mergerel") + .as(Encoders.bean(Relation.class)) + .filter((FilterFunction) r -> r.getRelClass().equalsIgnoreCase("merges")) + .groupBy("source") + .agg(count("target").alias("cnt")) + .select("source", "cnt") + .where("cnt > 3") + .count(); + long sw_mergerel = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/software_mergerel") + .as(Encoders.bean(Relation.class)) + .filter((FilterFunction) r -> r.getRelClass().equalsIgnoreCase("merges")) + .groupBy("source") + .agg(count("target").alias("cnt")) + .select("source", "cnt") + .where("cnt > 3") + .count(); + + long ds_mergerel = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/dataset_mergerel") + .as(Encoders.bean(Relation.class)) + .filter((FilterFunction) r -> r.getRelClass().equalsIgnoreCase("merges")) + .groupBy("source") + .agg(count("target").alias("cnt")) + .select("source", "cnt") + .where("cnt > 3") + .count(); + + long orp_mergerel = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_mergerel") + .as(Encoders.bean(Relation.class)) + .filter((FilterFunction) r -> r.getRelClass().equalsIgnoreCase("merges")) + .groupBy("source") + .agg(count("target").alias("cnt")) + .select("source", "cnt") + .where("cnt > 3") + .count(); + + assertEquals(0, orgs_mergerel); + assertEquals(0, pubs_mergerel); + assertEquals(0, sw_mergerel); + assertEquals(0, ds_mergerel); + assertEquals(0, orp_mergerel); + } + + @Test + @Order(3) public void createMergeRelsTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -241,7 +335,7 @@ public class SparkDedupTest implements Serializable { } @Test - @Order(3) + @Order(4) public void createDedupRecordTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -288,7 +382,7 @@ public class SparkDedupTest implements Serializable { } @Test - @Order(4) + @Order(5) public void updateEntityTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -404,7 +498,7 @@ public class SparkDedupTest implements Serializable { } @Test - @Order(5) + @Order(6) public void propagateRelationTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -454,7 +548,7 @@ public class SparkDedupTest implements Serializable { } @Test - @Order(6) + @Order(7) public void testRelations() throws Exception { testUniqueness("/eu/dnetlib/dhp/dedup/test/relation_1.json", 12, 10); testUniqueness("/eu/dnetlib/dhp/dedup/test/relation_2.json", 10, 2); From 8a612d861a89bb7dbe1da3992b9cc6730a4d5125 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 13 Jul 2020 15:30:57 +0200 Subject: [PATCH 041/100] WIP SparkCreateMergeRels distinct relations --- .../dhp/oa/dedup/SparkCreateMergeRels.java | 11 +------- .../dhp/oa/dedup/SparkCreateSimRels.java | 21 ++++---------- .../dhp/oa/dedup/SparkPropagateRelation.java | 18 +++++++++++- .../oa/dedup/graph/ConnectedComponent.java | 5 ++-- .../dnetlib/dhp/oa/dedup/SparkDedupTest.java | 28 +++++++++++++------ .../dnetlib/dhp/dedup/conf/ds.curr.conf.json | 7 +++-- .../dnetlib/dhp/dedup/conf/orp.curr.conf.json | 7 +++-- .../dnetlib/dhp/dedup/conf/pub.curr.conf.json | 7 +++-- .../dnetlib/dhp/dedup/conf/sw.curr.conf.json | 9 +++--- 9 files changed, 61 insertions(+), 52 deletions(-) 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 0c31f5fa2..0b44935d0 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 @@ -80,16 +80,10 @@ public class SparkCreateMergeRels extends AbstractSparkAction { try { cut = Integer.parseInt(parser.get("cutConnectedComponent")); - } catch (Throwable e) { - log.error("unable to parse "+parser.get(" cut-off threshold")); + log.error("unable to parse " + parser.get(" cut-off threshold")); } - - - - - log.info("graphBasePath: '{}'", graphBasePath); log.info("isLookUpUrl: '{}'", isLookUpUrl); log.info("actionSetId: '{}'", actionSetId); @@ -134,9 +128,6 @@ public class SparkCreateMergeRels extends AbstractSparkAction { mergeRels.write().mode(SaveMode.Append).parquet(mergeRelPath); - - - } } 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 1be2b9e31..2e96b3563 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 @@ -34,6 +34,8 @@ public class SparkCreateSimRels extends AbstractSparkAction { private static final Logger log = LoggerFactory.getLogger(SparkCreateSimRels.class); + public static final int NUM_PARTITIONS = 10000; + public SparkCreateSimRels(ArgumentApplicationParser parser, SparkSession spark) { super(parser, spark); } @@ -48,13 +50,6 @@ public class SparkCreateSimRels extends AbstractSparkAction { 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 - }); - new SparkCreateSimRels(parser, getSparkSession(conf)) .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); } @@ -88,7 +83,7 @@ public class SparkCreateSimRels extends AbstractSparkAction { JavaPairRDD mapDocuments = sc .textFile(DedupUtility.createEntityPath(graphBasePath, subEntity)) - .repartition(10000) + .repartition(NUM_PARTITIONS) .mapToPair( (PairFunction) s -> { MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s); @@ -98,21 +93,15 @@ public class SparkCreateSimRels extends AbstractSparkAction { // create blocks for deduplication JavaPairRDD blocks = Deduper .createSortedBlocks(mapDocuments, dedupConf) - .repartition(10000); + .repartition(NUM_PARTITIONS); // create relations by comparing only elements in the same group Deduper .computeRelations(sc, blocks, dedupConf) .map(t -> createSimRel(t._1(), t._2(), entity)) - .repartition(10000) + .repartition(NUM_PARTITIONS) .map(r -> OBJECT_MAPPER.writeValueAsString(r)) .saveAsTextFile(outputPath); - - // save the simrel in the workingdir - /* - * spark .createDataset(relations.rdd(), Encoders.bean(Relation.class)) .write() .mode(SaveMode.Append) - * .save(outputPath); - */ } } 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 c19769749..88fe5b26d 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 @@ -4,7 +4,9 @@ package eu.dnetlib.dhp.oa.dedup; import static org.apache.spark.sql.functions.col; import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.FilterFunction; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.*; import org.slf4j.Logger; @@ -95,11 +97,17 @@ public class SparkPropagateRelation extends AbstractSparkAction { FieldType.TARGET, getDeletedFn()); - save(distinctRelations(newRels.union(updated).union(mergeRels)), outputRelationPath, SaveMode.Overwrite); + save( + newRels + .union(updated) + .union(mergeRels) + .map((MapFunction) r -> r, Encoders.kryo(Relation.class)), + outputRelationPath, SaveMode.Overwrite); } private Dataset distinctRelations(Dataset rels) { return rels + .filter(getRelationFilterFunction()) .groupByKey((MapFunction) r -> ModelSupport.idFn().apply(r), Encoders.STRING()) .agg(new RelationAggregator().toColumn()) .map((MapFunction, Relation>) t -> t._2(), Encoders.bean(Relation.class)); @@ -119,6 +127,14 @@ public class SparkPropagateRelation extends AbstractSparkAction { .map(mapFn, Encoders.bean(Relation.class)); } + private FilterFunction getRelationFilterFunction() { + return (FilterFunction) r -> StringUtils.isNotBlank(r.getSource()) || + StringUtils.isNotBlank(r.getTarget()) || + StringUtils.isNotBlank(r.getRelClass()) || + StringUtils.isNotBlank(r.getSubRelType()) || + StringUtils.isNotBlank(r.getRelClass()); + } + private static MapFunction patchRelFn() { return value -> { final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class); 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 fd6e70916..c5f893668 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 @@ -22,12 +22,11 @@ public class ConnectedComponent implements Serializable { public ConnectedComponent() { } - public ConnectedComponent(Set docIds, final int cut) { this.docIds = docIds; createID(); - if (cut > 0 && docIds.size() > cut){ - docIds = docIds.stream().filter(s -> !ccId.equalsIgnoreCase(s)).limit(cut -1).collect(Collectors.toSet()); + if (cut > 0 && docIds.size() > cut) { + docIds = docIds.stream().filter(s -> !ccId.equalsIgnoreCase(s)).limit(cut - 1).collect(Collectors.toSet()); docIds.add(ccId); } } 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 88d5f24f9..8e1ef8f9c 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 @@ -166,23 +166,31 @@ public class SparkDedupTest implements Serializable { long orgs_simrel = spark .read() - .load(testOutputBasePath + "/" + testActionSetId + "/organization_simrel") + .textFile(testOutputBasePath + "/" + testActionSetId + "/organization_simrel") .count(); + long pubs_simrel = spark .read() - .load(testOutputBasePath + "/" + testActionSetId + "/publication_simrel") + .textFile(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 sw_simrel = spark + .read() + .textFile(testOutputBasePath + "/" + testActionSetId + "/software_simrel") + .count(); + + long ds_simrel = spark + .read() + .textFile(testOutputBasePath + "/" + testActionSetId + "/dataset_simrel") + .count(); long orp_simrel = spark .read() - .load(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_simrel") + .textFile(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_simrel") .count(); assertEquals(3432, orgs_simrel); - assertEquals(7054, pubs_simrel); + assertEquals(7152, pubs_simrel); assertEquals(344, sw_simrel); assertEquals(458, ds_simrel); assertEquals(6750, orp_simrel); @@ -225,8 +233,10 @@ public class SparkDedupTest implements Serializable { .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() @@ -234,7 +244,7 @@ public class SparkDedupTest implements Serializable { .count(); assertEquals(1276, orgs_mergerel); - assertEquals(1440, pubs_mergerel); + assertEquals(1442, pubs_mergerel); assertEquals(288, sw_mergerel); assertEquals(472, ds_mergerel); assertEquals(718, orp_mergerel); diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/ds.curr.conf.json b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/ds.curr.conf.json index 2469b2cc0..fa889d63b 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/ds.curr.conf.json +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/ds.curr.conf.json @@ -6,10 +6,10 @@ "subEntityType" : "resulttype", "subEntityValue" : "dataset", "orderField" : "title", - "queueMaxSize" : "800", + "queueMaxSize" : "100", "groupMaxSize" : "100", "maxChildren" : "100", - "slidingWindowSize" : "80", + "slidingWindowSize" : "100", "rootBuilder" : ["result", "resultProject_outcome_isProducedBy", "resultResult_publicationDataset_isRelatedTo", "resultResult_similarity_isAmongTopNSimilarDocuments", "resultResult_similarity_hasAmongTopNSimilarDocuments", "resultOrganization_affiliation_hasAuthorInstitution", "resultResult_part_hasPart", "resultResult_part_isPartOf", "resultResult_supplement_isSupplementTo", "resultResult_supplement_isSupplementedBy", "resultResult_version_isVersionOf" ], "includeChildren" : "true", "idPath" : "$.id", @@ -17,7 +17,8 @@ }, "pace" : { "clustering" : [ - { "name" : "wordssuffixprefix", "fields" : [ "title" ], "params" : { "max" : "2", "len" : "3" } }, + { "name" : "ngrampairs", "fields" : [ "title" ], "params" : { "max" : "1", "ngramLen" : "3"} }, + { "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } }, { "name" : "lowercase", "fields" : [ "doi" ], "params" : { } } ], "decisionTree" : { diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/orp.curr.conf.json b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/orp.curr.conf.json index 4adcc0439..b45b6ae83 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/orp.curr.conf.json +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/orp.curr.conf.json @@ -6,10 +6,10 @@ "subEntityType" : "resulttype", "subEntityValue" : "otherresearchproduct", "orderField" : "title", - "queueMaxSize" : "800", + "queueMaxSize" : "100", "groupMaxSize" : "100", "maxChildren" : "100", - "slidingWindowSize" : "80", + "slidingWindowSize" : "100", "rootBuilder" : [ "result", "resultProject_outcome_isProducedBy", "resultResult_publicationDataset_isRelatedTo", "resultResult_similarity_isAmongTopNSimilarDocuments", "resultResult_similarity_hasAmongTopNSimilarDocuments", "resultOrganization_affiliation_hasAuthorInstitution", "resultResult_part_hasPart", "resultResult_part_isPartOf", "resultResult_supplement_isSupplementTo", "resultResult_supplement_isSupplementedBy", "resultResult_version_isVersionOf" ], "includeChildren" : "true", "idPath" : "$.id", @@ -17,7 +17,8 @@ }, "pace" : { "clustering" : [ - { "name" : "wordssuffixprefix", "fields" : [ "title" ], "params" : { "max" : "2", "len" : "3" } }, + { "name" : "ngrampairs", "fields" : [ "title" ], "params" : { "max" : "1", "ngramLen" : "3"} }, + { "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } }, { "name" : "lowercase", "fields" : [ "doi" ], "params" : { } } ], "decisionTree" : { diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json index ef0b26af4..15ebc7a6a 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json @@ -6,10 +6,10 @@ "subEntityType": "resulttype", "subEntityValue": "publication", "orderField": "title", - "queueMaxSize": "800", + "queueMaxSize": "100", "groupMaxSize": "100", "maxChildren": "100", - "slidingWindowSize": "80", + "slidingWindowSize": "100", "rootBuilder": [ "result", "resultProject_outcome_isProducedBy", @@ -29,7 +29,8 @@ }, "pace": { "clustering" : [ - { "name" : "wordssuffixprefix", "fields" : [ "title" ], "params" : { "max" : "2", "len" : "3" } }, + { "name" : "ngrampairs", "fields" : [ "title" ], "params" : { "max" : "1", "ngramLen" : "3"} }, + { "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } }, { "name" : "lowercase", "fields" : [ "doi" ], "params" : { } } ], "decisionTree": { diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/sw.curr.conf.json b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/sw.curr.conf.json index 623abbf9f..f53ff385f 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/sw.curr.conf.json +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/conf/sw.curr.conf.json @@ -6,10 +6,10 @@ "subEntityType" : "resulttype", "subEntityValue" : "software", "orderField" : "title", - "queueMaxSize" : "800", + "queueMaxSize" : "100", "groupMaxSize" : "100", "maxChildren" : "100", - "slidingWindowSize" : "80", + "slidingWindowSize" : "100", "rootBuilder" : [ "result", "resultProject_outcome_isProducedBy", "resultResult_publicationDataset_isRelatedTo", "resultResult_similarity_isAmongTopNSimilarDocuments", "resultResult_similarity_hasAmongTopNSimilarDocuments", "resultOrganization_affiliation_hasAuthorInstitution", "resultResult_part_hasPart", "resultResult_part_isPartOf", "resultResult_supplement_isSupplementTo", "resultResult_supplement_isSupplementedBy", "resultResult_version_isVersionOf" ], "includeChildren" : "true", "idPath" : "$.id", @@ -17,8 +17,9 @@ }, "pace" : { "clustering" : [ - { "name" : "wordssuffixprefix", "fields" : [ "title" ], "params" : { "max" : "2", "len" : "3" } }, - { "name" : "lowercase", "fields" : [ "doi", "url" ], "params" : { } } + { "name" : "ngrampairs", "fields" : [ "title" ], "params" : { "max" : "1", "ngramLen" : "3"} }, + { "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } }, + { "name" : "lowercase", "fields" : [ "doi" ], "params" : { } } ], "decisionTree": { "start": { From 7dd91edf4335e7f23ac87661b86008e835134aa6 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 13 Jul 2020 15:40:41 +0200 Subject: [PATCH 042/100] parsing of optional parameter --- .../dnetlib/dhp/oa/dedup/SparkCreateMergeRels.java | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) 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 0b44935d0..70457fb4a 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 @@ -5,6 +5,7 @@ import java.io.IOException; 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; @@ -76,14 +77,11 @@ public class SparkCreateMergeRels extends AbstractSparkAction { final String workingPath = parser.get("workingPath"); final String isLookUpUrl = parser.get("isLookUpUrl"); final String actionSetId = parser.get("actionSetId"); - int cut = 0; - try { - cut = Integer.parseInt(parser.get("cutConnectedComponent")); - - } catch (Throwable e) { - log.error("unable to parse " + parser.get(" cut-off threshold")); - } - + int cut = Optional + .ofNullable(parser.get("cutConnectedComponent")) + .map(Integer::valueOf) + .orElse(0); + log.info("connected component cut: '{}'", cut); log.info("graphBasePath: '{}'", graphBasePath); log.info("isLookUpUrl: '{}'", isLookUpUrl); log.info("actionSetId: '{}'", actionSetId); From 3635d05061b45a1daa944193814f64f81947ad51 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Mon, 13 Jul 2020 15:52:23 +0200 Subject: [PATCH 043/100] poms --- dhp-workflows/dhp-broker-events/pom.xml | 1 - pom.xml | 7 +++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-broker-events/pom.xml b/dhp-workflows/dhp-broker-events/pom.xml index 6787226a8..f98708c64 100644 --- a/dhp-workflows/dhp-broker-events/pom.xml +++ b/dhp-workflows/dhp-broker-events/pom.xml @@ -59,7 +59,6 @@ eu.dnetlib.dhp dnet-openaire-broker-common - [3.0.0,) diff --git a/pom.xml b/pom.xml index 4619f3174..411ef9521 100644 --- a/pom.xml +++ b/pom.xml @@ -323,6 +323,12 @@ [2.0.0,3.0.0)
+ + eu.dnetlib.dhp + dnet-openaire-broker-common + ${dnet.openaire.broker.common} + + org.apache.cxf cxf-rt-transports-http @@ -618,5 +624,6 @@ 3.3.3 3.4.2 [2.12,3.0) + 3.0.0
From 1d133b7fe6590dbd706ef77e40cfd657debbc00c Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Mon, 13 Jul 2020 15:52:41 +0200 Subject: [PATCH 044/100] update test --- .../test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java | 6 ++++++ 1 file changed, 6 insertions(+) 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 e10655126..294b19ecd 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 @@ -288,6 +288,12 @@ public class SparkDedupTest implements Serializable { assertEquals(0, sw_mergerel); assertEquals(0, ds_mergerel); assertEquals(0, orp_mergerel); + + FileUtils.deleteDirectory(new File(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel")); + FileUtils.deleteDirectory(new File(testOutputBasePath + "/" + testActionSetId + "/publication_mergerel")); + FileUtils.deleteDirectory(new File(testOutputBasePath + "/" + testActionSetId + "/software_mergerel")); + FileUtils.deleteDirectory(new File(testOutputBasePath + "/" + testActionSetId + "/dataset_mergerel")); + FileUtils.deleteDirectory(new File(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_mergerel")); } @Test From c8284bab06c13f47d135cbd7d8a8fe009f090b6b Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 13 Jul 2020 15:54:51 +0200 Subject: [PATCH 045/100] WIP SparkCreateMergeRels distinct relations --- .../main/java/eu/dnetlib/dhp/oa/dedup/RelationAggregator.java | 4 ++-- .../java/eu/dnetlib/dhp/oa/dedup/SparkPropagateRelation.java | 4 ++-- .../src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java | 4 +++- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/RelationAggregator.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/RelationAggregator.java index 0a29aa51b..7935fe1ca 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/RelationAggregator.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/RelationAggregator.java @@ -36,11 +36,11 @@ public class RelationAggregator extends Aggregator @Override public Encoder bufferEncoder() { - return Encoders.bean(Relation.class); + return Encoders.kryo(Relation.class); } @Override public Encoder outputEncoder() { - return Encoders.bean(Relation.class); + return Encoders.kryo(Relation.class); } } 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 88fe5b26d..baba3bc87 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 @@ -98,10 +98,10 @@ public class SparkPropagateRelation extends AbstractSparkAction { getDeletedFn()); save( - newRels + distinctRelations(newRels .union(updated) .union(mergeRels) - .map((MapFunction) r -> r, Encoders.kryo(Relation.class)), + .map((MapFunction) r -> r, Encoders.kryo(Relation.class))), outputRelationPath, SaveMode.Overwrite); } 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 e10655126..82c2d82b7 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 @@ -77,11 +77,13 @@ public class SparkDedupTest implements Serializable { FileUtils.deleteDirectory(new File(testOutputBasePath)); FileUtils.deleteDirectory(new File(testDedupGraphBasePath)); + final SparkConf conf = new SparkConf(); + conf.set("spark.sql.shuffle.partitions", "200"); spark = SparkSession .builder() .appName(SparkDedupTest.class.getSimpleName()) .master("local[*]") - .config(new SparkConf()) + .config(conf) .getOrCreate(); jsc = JavaSparkContext.fromSparkContext(spark.sparkContext()); From 8c67938ad0a9d7713530d2a1ede0cf59f161fbe5 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 13 Jul 2020 16:07:07 +0200 Subject: [PATCH 046/100] configurable number of partitions used in the SparkCreateSimRels phase --- .../dhp/oa/dedup/SparkCreateSimRels.java | 14 +++++--- .../oa/dedup/createSimRels_parameters.json | 6 ++++ .../dhp/oa/dedup/scan/oozie_app/workflow.xml | 36 +++++++++++-------- .../dhp/oa/dedup/updateEntity_parameters.json | 26 +++++++------- 4 files changed, 50 insertions(+), 32 deletions(-) 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 2e96b3563..a66ab431c 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 @@ -2,6 +2,7 @@ package eu.dnetlib.dhp.oa.dedup; import java.io.IOException; +import java.util.Optional; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; @@ -34,7 +35,7 @@ public class SparkCreateSimRels extends AbstractSparkAction { private static final Logger log = LoggerFactory.getLogger(SparkCreateSimRels.class); - public static final int NUM_PARTITIONS = 10000; + public static final int NUM_PARTITIONS = 1000; public SparkCreateSimRels(ArgumentApplicationParser parser, SparkSession spark) { super(parser, spark); @@ -63,7 +64,12 @@ public class SparkCreateSimRels extends AbstractSparkAction { final String isLookUpUrl = parser.get("isLookUpUrl"); final String actionSetId = parser.get("actionSetId"); final String workingPath = parser.get("workingPath"); + final int numPartitions = Optional + .ofNullable(parser.get("numPartitions")) + .map(Integer::valueOf) + .orElse(NUM_PARTITIONS); + log.info("numPartitions: '{}'", numPartitions); log.info("graphBasePath: '{}'", graphBasePath); log.info("isLookUpUrl: '{}'", isLookUpUrl); log.info("actionSetId: '{}'", actionSetId); @@ -83,7 +89,7 @@ public class SparkCreateSimRels extends AbstractSparkAction { JavaPairRDD mapDocuments = sc .textFile(DedupUtility.createEntityPath(graphBasePath, subEntity)) - .repartition(NUM_PARTITIONS) + .repartition(numPartitions) .mapToPair( (PairFunction) s -> { MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s); @@ -93,13 +99,13 @@ public class SparkCreateSimRels extends AbstractSparkAction { // create blocks for deduplication JavaPairRDD blocks = Deduper .createSortedBlocks(mapDocuments, dedupConf) - .repartition(NUM_PARTITIONS); + .repartition(numPartitions); // create relations by comparing only elements in the same group Deduper .computeRelations(sc, blocks, dedupConf) .map(t -> createSimRel(t._1(), t._2(), entity)) - .repartition(NUM_PARTITIONS) + .repartition(numPartitions) .map(r -> OBJECT_MAPPER.writeValueAsString(r)) .saveAsTextFile(outputPath); } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json index ce38dc6f0..09f4365d3 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json @@ -22,5 +22,11 @@ "paramLongName": "workingPath", "paramDescription": "path of the working directory", "paramRequired": true + }, + { + "paramName": "np", + "paramLongName": "numPartitions", + "paramDescription": "number of partitions for the similarity relations intermediate phases", + "paramRequired": false } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml index 298a248e3..c42ce1263 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml @@ -20,6 +20,10 @@ dedupGraphPath path for the output graph + + cutConnectedComponent + max number of elements in a connected component + sparkDriverMemory memory for driver process @@ -106,10 +110,11 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.shuffle.partitions=3840 - --i${graphBasePath} - --la${isLookUpUrl} - --asi${actionSetId} - --w${workingPath} + --graphBasePath${graphBasePath} + --isLookUpUrl${isLookUpUrl} + --actionSetId${actionSetId} + --workingPath${workingPath} + --numPartitions8000 @@ -132,10 +137,11 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.shuffle.partitions=3840 - --i${graphBasePath} - --w${workingPath} - --la${isLookUpUrl} - --asi${actionSetId} + --graphBasePath${graphBasePath} + --workingPath${workingPath} + --isLookUpUrl${isLookUpUrl} + --actionSetId${actionSetId} + --cutConnectedComponent${cutConnectedComponent} @@ -158,10 +164,10 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.shuffle.partitions=3840 - --i${graphBasePath} - --w${workingPath} - --la${isLookUpUrl} - --asi${actionSetId} + --graphBasePath${graphBasePath} + --workingPath${workingPath} + --isLookUpUrl${isLookUpUrl} + --actionSetId${actionSetId} @@ -184,9 +190,9 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.shuffle.partitions=3840 - --i${graphBasePath} - --w${workingPath} - --o${dedupGraphPath} + --graphBasePath${graphBasePath} + --workingPath${workingPath} + --dedupGraphPath${dedupGraphPath} diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json index c91f3c04b..6a2a48746 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json @@ -1,17 +1,17 @@ [ -{ - "paramName": "i", - "paramLongName": "graphBasePath", - "paramDescription": "the base path of raw graph", - "paramRequired": true -}, -{ - "paramName": "w", - "paramLongName": "workingPath", - "paramDescription": "the working directory path", - "paramRequired": true -}, -{ + { + "paramName": "i", + "paramLongName": "graphBasePath", + "paramDescription": "the base path of raw graph", + "paramRequired": true + }, + { + "paramName": "w", + "paramLongName": "workingPath", + "paramDescription": "the working directory path", + "paramRequired": true + }, + { "paramName": "o", "paramLongName": "dedupGraphPath", "paramDescription": "the path of the dedup graph", From 1143f426aaf2eadfa8b55dcb62fabf8b52bcb503 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 13 Jul 2020 16:13:36 +0200 Subject: [PATCH 047/100] WIP SparkCreateMergeRels distinct relations --- .../dhp/oa/dedup/RelationAggregator.java | 17 ++++++++++++++--- .../dhp/oa/dedup/SparkPropagateRelation.java | 3 ++- 2 files changed, 16 insertions(+), 4 deletions(-) diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/RelationAggregator.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/RelationAggregator.java index 7935fe1ca..6fb7b844b 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/RelationAggregator.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/RelationAggregator.java @@ -20,13 +20,12 @@ public class RelationAggregator extends Aggregator @Override public Relation reduce(Relation b, Relation a) { - return Objects.equals(a, ZERO) ? b : a; + return mergeRel(b, a); } @Override public Relation merge(Relation b, Relation a) { - b.mergeFrom(a); - return b; + return mergeRel(b, a); } @Override @@ -34,6 +33,18 @@ public class RelationAggregator extends Aggregator return r; } + private Relation mergeRel(Relation b, Relation a) { + if (Objects.equals(b, ZERO)) { + return a; + } + if (Objects.equals(a, ZERO)) { + return b; + } + + b.mergeFrom(a); + return b; + } + @Override public Encoder bufferEncoder() { return Encoders.kryo(Relation.class); 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 baba3bc87..1073adbea 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 @@ -3,6 +3,7 @@ package eu.dnetlib.dhp.oa.dedup; import static org.apache.spark.sql.functions.col; +import com.google.common.base.Joiner; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.spark.SparkConf; @@ -108,7 +109,7 @@ public class SparkPropagateRelation extends AbstractSparkAction { private Dataset distinctRelations(Dataset rels) { return rels .filter(getRelationFilterFunction()) - .groupByKey((MapFunction) r -> ModelSupport.idFn().apply(r), Encoders.STRING()) + .groupByKey((MapFunction) r -> String.join(r.getSource(), r.getTarget(), r.getRelType(), r.getSubRelType(), r.getRelClass()), Encoders.STRING()) .agg(new RelationAggregator().toColumn()) .map((MapFunction, Relation>) t -> t._2(), Encoders.bean(Relation.class)); } From 344a90c2e6ab1aecf2f48bd5e949bfb8bbb467ae Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 13 Jul 2020 16:32:04 +0200 Subject: [PATCH 048/100] updated assertions in propagateRelationTest --- .../eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) 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 8ed05c18d..91e45447c 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 @@ -160,14 +160,11 @@ public class SparkDedupTest implements Serializable { parser .parseArgument( new String[] { - "-i", - testGraphBasePath, - "-asi", - testActionSetId, - "-la", - "lookupurl", - "-w", - testOutputBasePath + "-i", testGraphBasePath, + "-asi", testActionSetId, + "-la", "lookupurl", + "-w", testOutputBasePath, + "-np", "50" }); new SparkCreateSimRels(parser, spark).run(isLookUpService); @@ -535,7 +532,7 @@ public class SparkDedupTest implements Serializable { long relations = jsc.textFile(testDedupGraphBasePath + "/relation").count(); - assertEquals(4971, relations); + assertEquals(4866, relations); // check deletedbyinference final Dataset mergeRels = spark From c6f6fb0f2853efcb1cce467fa3113242e88cf8f0 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 13 Jul 2020 16:46:13 +0200 Subject: [PATCH 049/100] code formatting --- .../dhp/oa/dedup/SparkCreateMergeRels.java | 6 +++--- .../dhp/oa/dedup/SparkCreateSimRels.java | 6 +++--- .../dhp/oa/dedup/SparkPropagateRelation.java | 17 +++++++++++------ .../eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java | 3 ++- 4 files changed, 19 insertions(+), 13 deletions(-) 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 70457fb4a..6d625cd11 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 @@ -78,9 +78,9 @@ public class SparkCreateMergeRels extends AbstractSparkAction { final String isLookUpUrl = parser.get("isLookUpUrl"); final String actionSetId = parser.get("actionSetId"); int cut = Optional - .ofNullable(parser.get("cutConnectedComponent")) - .map(Integer::valueOf) - .orElse(0); + .ofNullable(parser.get("cutConnectedComponent")) + .map(Integer::valueOf) + .orElse(0); log.info("connected component cut: '{}'", cut); log.info("graphBasePath: '{}'", graphBasePath); log.info("isLookUpUrl: '{}'", isLookUpUrl); 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 a66ab431c..3beb90e0b 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 @@ -65,9 +65,9 @@ public class SparkCreateSimRels extends AbstractSparkAction { final String actionSetId = parser.get("actionSetId"); final String workingPath = parser.get("workingPath"); final int numPartitions = Optional - .ofNullable(parser.get("numPartitions")) - .map(Integer::valueOf) - .orElse(NUM_PARTITIONS); + .ofNullable(parser.get("numPartitions")) + .map(Integer::valueOf) + .orElse(NUM_PARTITIONS); log.info("numPartitions: '{}'", numPartitions); log.info("graphBasePath: '{}'", graphBasePath); 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 1073adbea..03e6674e4 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 @@ -3,7 +3,6 @@ package eu.dnetlib.dhp.oa.dedup; import static org.apache.spark.sql.functions.col; -import com.google.common.base.Joiner; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.spark.SparkConf; @@ -13,6 +12,8 @@ import org.apache.spark.sql.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.base.Joiner; + import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.DataInfo; @@ -99,17 +100,21 @@ public class SparkPropagateRelation extends AbstractSparkAction { getDeletedFn()); save( - distinctRelations(newRels - .union(updated) - .union(mergeRels) - .map((MapFunction) r -> r, Encoders.kryo(Relation.class))), + distinctRelations( + newRels + .union(updated) + .union(mergeRels) + .map((MapFunction) r -> r, Encoders.kryo(Relation.class))), outputRelationPath, SaveMode.Overwrite); } private Dataset distinctRelations(Dataset rels) { return rels .filter(getRelationFilterFunction()) - .groupByKey((MapFunction) r -> String.join(r.getSource(), r.getTarget(), r.getRelType(), r.getSubRelType(), r.getRelClass()), Encoders.STRING()) + .groupByKey( + (MapFunction) r -> String + .join(r.getSource(), r.getTarget(), r.getRelType(), r.getSubRelType(), r.getRelClass()), + Encoders.STRING()) .agg(new RelationAggregator().toColumn()) .map((MapFunction, Relation>) t -> t._2(), Encoders.bean(Relation.class)); } 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 91e45447c..fb5ebc099 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 @@ -292,7 +292,8 @@ public class SparkDedupTest implements Serializable { FileUtils.deleteDirectory(new File(testOutputBasePath + "/" + testActionSetId + "/publication_mergerel")); FileUtils.deleteDirectory(new File(testOutputBasePath + "/" + testActionSetId + "/software_mergerel")); FileUtils.deleteDirectory(new File(testOutputBasePath + "/" + testActionSetId + "/dataset_mergerel")); - FileUtils.deleteDirectory(new File(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_mergerel")); + FileUtils + .deleteDirectory(new File(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_mergerel")); } @Test From 9258e4f095ab6e181d2c06161b1b3c69a4397669 Mon Sep 17 00:00:00 2001 From: miconis Date: Mon, 13 Jul 2020 18:22:34 +0200 Subject: [PATCH 050/100] implementation of a new workflow to compute statistics on the blocks --- .../eu/dnetlib/dhp/oa/dedup/DedupUtility.java | 5 + .../dnetlib/dhp/oa/dedup/SparkBlockStats.java | 121 ++++++++++++++++++ .../dhp/oa/dedup/model/BlockStats.java | 42 ++++++ .../oa/dedup/createBlockStats_parameters.json | 26 ++++ .../statistics/oozie_app/config-default.xml | 18 +++ .../dedup/statistics/oozie_app/workflow.xml | 111 ++++++++++++++++ 6 files changed, 323 insertions(+) create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/BlockStats.java create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createBlockStats_parameters.json create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/oozie_app/workflow.xml 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 222794d64..01065510a 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 @@ -100,6 +100,11 @@ public class DedupUtility { return String.format("%s/%s/%s_mergerel", basePath, actionSetId, entityType); } + public static String createBlockStatsPath( + final String basePath, final String actionSetId, final String entityType) { + return String.format("%s/%s/%s_blockstats", basePath, actionSetId, entityType); + } + public static List getConfigurations(String isLookUpUrl, String orchestrator) throws ISLookUpException, DocumentException { final ISLookUpService isLookUpService = ISLookupClientFactory.getLookUpService(isLookUpUrl); diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java new file mode 100644 index 000000000..bfd98e78e --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java @@ -0,0 +1,121 @@ +package eu.dnetlib.dhp.oa.dedup; + +import java.io.IOException; + +import eu.dnetlib.dhp.oa.dedup.model.BlockStats; +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.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.model.FieldListImpl; +import eu.dnetlib.pace.model.FieldValueImpl; +import eu.dnetlib.pace.model.MapDocument; +import eu.dnetlib.pace.util.MapDocumentUtil; +import scala.Tuple2; + +public class SparkBlockStats extends AbstractSparkAction { + + private static final Logger log = LoggerFactory.getLogger(SparkCreateSimRels.class); + + public SparkBlockStats(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/createBlockStats_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 + }); + + new SparkCreateSimRels(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } + + @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"); + + 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)) { + + final String subEntity = dedupConf.getWf().getSubEntityValue(); + log.info("Creating blockstats for: '{}'", subEntity); + + final String outputPath = DedupUtility.createBlockStatsPath(workingPath, actionSetId, subEntity); + removeOutputDir(spark, outputPath); + + 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); + }); + + // create blocks for deduplication + JavaPairRDD blocks = Deduper.createSortedBlocks(mapDocuments, dedupConf); + + JavaRDD blockStats = blocks.map(b -> + new BlockStats( + b._1(), + (long) b._2().getDocuments().size(), + computeComparisons( + (long) b._2().getDocuments().size(), (long) dedupConf.getWf().getSlidingWindowSize())) + ); + + // save the blockstats in the workingdir + spark + .createDataset(blockStats.rdd(), Encoders.bean(BlockStats.class)) + .write() + .mode(SaveMode.Overwrite) + .save(outputPath); + } + } + + public Long computeComparisons(Long blockSize, Long slidingWindowSize){ + + if (slidingWindowSize >= blockSize) + return (slidingWindowSize * (slidingWindowSize - 1)) / 2; + else { + return (blockSize - slidingWindowSize + 1) * (slidingWindowSize * (slidingWindowSize - 1)) / 2; + } + } +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/BlockStats.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/BlockStats.java new file mode 100644 index 000000000..ef8505e8f --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/BlockStats.java @@ -0,0 +1,42 @@ +package eu.dnetlib.dhp.oa.dedup.model; + +public class BlockStats { + + private String key; //key of the block + private Long size; //number of elements in the block + private Long comparisons; //number of comparisons in the block + + public BlockStats() { + } + + public BlockStats(String key, Long size, Long comparisons) { + this.key = key; + this.size = size; + this.comparisons = comparisons; + } + + public String getKey() { + return key; + } + + public void setKey(String key) { + this.key = key; + } + + public Long getSize() { + return size; + } + + public void setSize(Long size) { + this.size = size; + } + + public Long getComparisons() { + return comparisons; + } + + public void setComparisons(Long comparisons) { + this.comparisons = comparisons; + } + +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createBlockStats_parameters.json b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createBlockStats_parameters.json new file mode 100644 index 000000000..ce38dc6f0 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createBlockStats_parameters.json @@ -0,0 +1,26 @@ +[ + { + "paramName": "la", + "paramLongName": "isLookUpUrl", + "paramDescription": "address for the LookUp", + "paramRequired": true + }, + { + "paramName": "asi", + "paramLongName": "actionSetId", + "paramDescription": "action set identifier (name of the orchestrator)", + "paramRequired": true + }, + { + "paramName": "i", + "paramLongName": "graphBasePath", + "paramDescription": "the base path of the raw graph", + "paramRequired": true + }, + { + "paramName": "w", + "paramLongName": "workingPath", + "paramDescription": "path of the working directory", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/oozie_app/config-default.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/oozie_app/config-default.xml new file mode 100644 index 000000000..2e0ed9aee --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/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-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/oozie_app/workflow.xml new file mode 100644 index 000000000..e1019cd01 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/oozie_app/workflow.xml @@ -0,0 +1,111 @@ + + + + graphBasePath + the raw graph base path + + + isLookUpUrl + the address of the lookUp service + + + actionSetId + id of the actionSet + + + workingPath + path for the working directory + + + dedupGraphPath + path for the output graph + + + sparkDriverMemory + memory for driver process + + + sparkExecutorMemory + memory for individual executor + + + sparkExecutorCores + number of cores used by single executor + + + oozieActionShareLibForSpark2 + oozie action sharelib for spark 2.* + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + spark 2.* extra listeners classname + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + spark 2.* sql query execution listeners classname + + + spark2YarnHistoryServerAddress + spark 2.* yarn history server address + + + spark2EventLogDir + spark 2.* event log dir location + + + + + ${jobTracker} + ${nameNode} + + + mapreduce.job.queuename + ${queueName} + + + oozie.launcher.mapred.job.queue.name + ${oozieLauncherQueueName} + + + oozie.action.sharelib.for.spark + ${oozieActionShareLibForSpark2} + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + yarn + cluster + Create Similarity Relations + eu.dnetlib.dhp.oa.dedup.SparkBlockStats + dhp-dedup-openaire-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.shuffle.partitions=3840 + + --i${graphBasePath} + --la${isLookUpUrl} + --asi${actionSetId} + --w${workingPath} + + + + + + + \ No newline at end of file From 03ecfa5ebd617eeac6d35006608fc742b96d92ac Mon Sep 17 00:00:00 2001 From: miconis Date: Mon, 13 Jul 2020 18:48:23 +0200 Subject: [PATCH 051/100] implementation of the test class for the new block stats spark action --- .../dhp/oa/dedup/model/BlockStats.java | 4 +- .../dnetlib/dhp/oa/dedup/SparkStatsTest.java | 175 ++++++++++++++++++ 2 files changed, 178 insertions(+), 1 deletion(-) create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkStatsTest.java diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/BlockStats.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/BlockStats.java index ef8505e8f..be4ec51a0 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/BlockStats.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/BlockStats.java @@ -1,6 +1,8 @@ package eu.dnetlib.dhp.oa.dedup.model; -public class BlockStats { +import java.io.Serializable; + +public class BlockStats implements Serializable { private String key; //key of the block private Long size; //number of elements in the block diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkStatsTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkStatsTest.java new file mode 100644 index 000000000..8fac56ef3 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkStatsTest.java @@ -0,0 +1,175 @@ +package eu.dnetlib.dhp.oa.dedup; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.*; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.io.File; +import java.io.IOException; +import java.io.Serializable; +import java.net.URISyntaxException; +import java.nio.file.Paths; + +import static java.nio.file.Files.createTempDirectory; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.lenient; + +@ExtendWith(MockitoExtension.class) +public class SparkStatsTest implements Serializable { + + @Mock(serializable = true) + ISLookUpService isLookUpService; + + private static SparkSession spark; + private static JavaSparkContext jsc; + + private static String testGraphBasePath; + private static String testOutputBasePath; + private static final String testActionSetId = "test-orchestrator"; + + @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(); + + FileUtils.deleteDirectory(new File(testOutputBasePath)); + + final SparkConf conf = new SparkConf(); + conf.set("spark.sql.shuffle.partitions", "200"); + spark = SparkSession + .builder() + .appName(SparkDedupTest.class.getSimpleName()) + .master("local[*]") + .config(conf) + .getOrCreate(); + + jsc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + } + + @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("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("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("otherresearchproduct"))) + .thenReturn( + IOUtils + .toString( + SparkDedupTest.class + .getResourceAsStream( + "/eu/dnetlib/dhp/dedup/conf/orp.curr.conf.json"))); + } + + @Test + public void createBlockStatsTest() throws Exception { + + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/createBlockStats_parameters.json"))); + parser + .parseArgument( + new String[] { + "-i", testGraphBasePath, + "-asi", testActionSetId, + "-la", "lookupurl", + "-w", testOutputBasePath + }); + + new SparkBlockStats(parser, spark).run(isLookUpService); + + long orgs_blocks = spark + .read() + .textFile(testOutputBasePath + "/" + testActionSetId + "/organization_blockstats") + .count(); + + long pubs_blocks = spark + .read() + .textFile(testOutputBasePath + "/" + testActionSetId + "/publication_blockstats") + .count(); + + long sw_blocks = spark + .read() + .textFile(testOutputBasePath + "/" + testActionSetId + "/software_blockstats") + .count(); + + long ds_blocks = spark + .read() + .textFile(testOutputBasePath + "/" + testActionSetId + "/dataset_blockstats") + .count(); + + long orp_blocks = spark + .read() + .textFile(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_blockstats") + .count(); + + assertEquals(121, orgs_blocks); + assertEquals(110, pubs_blocks); + assertEquals(21, sw_blocks); + assertEquals(67, ds_blocks); + assertEquals(55, orp_blocks); + } +} + From 66f9f6d3239545d842798a869a1152f5204616ff Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 13 Jul 2020 19:26:46 +0200 Subject: [PATCH 052/100] adjusted parameters for the dedup stats workflow --- .../dnetlib/dhp/oa/dedup/SparkBlockStats.java | 142 +++++---- .../dhp/oa/dedup/model/BlockStats.java | 57 ++-- .../dedup/statistics/oozie_app/workflow.xml | 14 +- .../dnetlib/dhp/oa/dedup/SparkStatsTest.java | 270 +++++++++--------- 4 files changed, 237 insertions(+), 246 deletions(-) diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java index bfd98e78e..7fd359a0c 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java @@ -1,8 +1,8 @@ + package eu.dnetlib.dhp.oa.dedup; import java.io.IOException; -import eu.dnetlib.dhp.oa.dedup.model.BlockStats; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; @@ -15,8 +15,10 @@ 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.oa.dedup.model.BlockStats; import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; @@ -29,93 +31,87 @@ import scala.Tuple2; public class SparkBlockStats extends AbstractSparkAction { - private static final Logger log = LoggerFactory.getLogger(SparkCreateSimRels.class); + private static final Logger log = LoggerFactory.getLogger(SparkCreateSimRels.class); - public SparkBlockStats(ArgumentApplicationParser parser, SparkSession spark) { - super(parser, spark); - } + public SparkBlockStats(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/createBlockStats_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/createBlockStats_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(); - 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 subEntity = dedupConf.getWf().getSubEntityValue(); - log.info("Creating blockstats for: '{}'", subEntity); + final String subEntity = dedupConf.getWf().getSubEntityValue(); + log.info("Creating blockstats for: '{}'", subEntity); - final String outputPath = DedupUtility.createBlockStatsPath(workingPath, actionSetId, subEntity); - removeOutputDir(spark, outputPath); + final String outputPath = DedupUtility.createBlockStatsPath(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); - JavaRDD blockStats = blocks.map(b -> - new BlockStats( - b._1(), - (long) b._2().getDocuments().size(), - computeComparisons( - (long) b._2().getDocuments().size(), (long) dedupConf.getWf().getSlidingWindowSize())) - ); + JavaRDD blockStats = blocks + .map( + b -> new BlockStats( + b._1(), + (long) b._2().getDocuments().size(), + computeComparisons( + (long) b._2().getDocuments().size(), (long) dedupConf.getWf().getSlidingWindowSize()))); - // save the blockstats in the workingdir - spark - .createDataset(blockStats.rdd(), Encoders.bean(BlockStats.class)) - .write() - .mode(SaveMode.Overwrite) - .save(outputPath); - } - } + // save the blockstats in the workingdir + spark + .createDataset(blockStats.rdd(), Encoders.bean(BlockStats.class)) + .write() + .mode(SaveMode.Overwrite) + .save(outputPath); + } + } - public Long computeComparisons(Long blockSize, Long slidingWindowSize){ + public Long computeComparisons(Long blockSize, Long slidingWindowSize) { - if (slidingWindowSize >= blockSize) - return (slidingWindowSize * (slidingWindowSize - 1)) / 2; - else { - return (blockSize - slidingWindowSize + 1) * (slidingWindowSize * (slidingWindowSize - 1)) / 2; - } - } + if (slidingWindowSize >= blockSize) + return (slidingWindowSize * (slidingWindowSize - 1)) / 2; + else { + return (blockSize - slidingWindowSize + 1) * (slidingWindowSize * (slidingWindowSize - 1)) / 2; + } + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/BlockStats.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/BlockStats.java index be4ec51a0..0ec8c2699 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/BlockStats.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/BlockStats.java @@ -1,44 +1,45 @@ + package eu.dnetlib.dhp.oa.dedup.model; import java.io.Serializable; public class BlockStats implements Serializable { - private String key; //key of the block - private Long size; //number of elements in the block - private Long comparisons; //number of comparisons in the block + private String key; // key of the block + private Long size; // number of elements in the block + private Long comparisons; // number of comparisons in the block - public BlockStats() { - } + public BlockStats() { + } - public BlockStats(String key, Long size, Long comparisons) { - this.key = key; - this.size = size; - this.comparisons = comparisons; - } + public BlockStats(String key, Long size, Long comparisons) { + this.key = key; + this.size = size; + this.comparisons = comparisons; + } - 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 Long getSize() { - return size; - } + public Long getSize() { + return size; + } - public void setSize(Long size) { - this.size = size; - } + public void setSize(Long size) { + this.size = size; + } - public Long getComparisons() { - return comparisons; - } + public Long getComparisons() { + return comparisons; + } - public void setComparisons(Long comparisons) { - this.comparisons = comparisons; - } + public void setComparisons(Long comparisons) { + this.comparisons = comparisons; + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/oozie_app/workflow.xml index e1019cd01..8b75d16b3 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/oozie_app/workflow.xml @@ -1,4 +1,4 @@ - + graphBasePath @@ -12,14 +12,6 @@ actionSetId id of the actionSet - - workingPath - path for the working directory - - - dedupGraphPath - path for the output graph - sparkDriverMemory memory for driver process @@ -85,7 +77,7 @@ yarn cluster - Create Similarity Relations + Create deduplication blocks eu.dnetlib.dhp.oa.dedup.SparkBlockStats dhp-dedup-openaire-${projectVersion}.jar @@ -101,7 +93,7 @@ --i${graphBasePath} --la${isLookUpUrl} --asi${actionSetId} - --w${workingPath} + --w${workingDir} diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkStatsTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkStatsTest.java index 8fac56ef3..7e76c284b 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkStatsTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkStatsTest.java @@ -1,8 +1,17 @@ + package eu.dnetlib.dhp.oa.dedup; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import static java.nio.file.Files.createTempDirectory; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.lenient; + +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; @@ -14,162 +23,155 @@ import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.junit.jupiter.MockitoExtension; -import java.io.File; -import java.io.IOException; -import java.io.Serializable; -import java.net.URISyntaxException; -import java.nio.file.Paths; - -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.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; @ExtendWith(MockitoExtension.class) public class SparkStatsTest 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 final String testActionSetId = "test-orchestrator"; + private static String testGraphBasePath; + private static String testOutputBasePath; + 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(); + testGraphBasePath = Paths + .get(SparkDedupTest.class.getResource("/eu/dnetlib/dhp/dedup/entities").toURI()) + .toFile() + .getAbsolutePath(); + testOutputBasePath = createTempDirectory(SparkDedupTest.class.getSimpleName() + "-") + .toAbsolutePath() + .toString(); - FileUtils.deleteDirectory(new File(testOutputBasePath)); + FileUtils.deleteDirectory(new File(testOutputBasePath)); - final SparkConf conf = new SparkConf(); - conf.set("spark.sql.shuffle.partitions", "200"); - spark = SparkSession - .builder() - .appName(SparkDedupTest.class.getSimpleName()) - .master("local[*]") - .config(conf) - .getOrCreate(); + final SparkConf conf = new SparkConf(); + conf.set("spark.sql.shuffle.partitions", "200"); + spark = SparkSession + .builder() + .appName(SparkDedupTest.class.getSimpleName()) + .master("local[*]") + .config(conf) + .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 - public void createBlockStatsTest() throws Exception { + @Test + public void createBlockStatsTest() throws Exception { - ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkCreateSimRels.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/createBlockStats_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/createBlockStats_parameters.json"))); + parser + .parseArgument( + new String[] { + "-i", testGraphBasePath, + "-asi", testActionSetId, + "-la", "lookupurl", + "-w", testOutputBasePath + }); - new SparkBlockStats(parser, spark).run(isLookUpService); + new SparkBlockStats(parser, spark).run(isLookUpService); - long orgs_blocks = spark - .read() - .textFile(testOutputBasePath + "/" + testActionSetId + "/organization_blockstats") - .count(); + long orgs_blocks = spark + .read() + .textFile(testOutputBasePath + "/" + testActionSetId + "/organization_blockstats") + .count(); - long pubs_blocks = spark - .read() - .textFile(testOutputBasePath + "/" + testActionSetId + "/publication_blockstats") - .count(); + long pubs_blocks = spark + .read() + .textFile(testOutputBasePath + "/" + testActionSetId + "/publication_blockstats") + .count(); - long sw_blocks = spark - .read() - .textFile(testOutputBasePath + "/" + testActionSetId + "/software_blockstats") - .count(); + long sw_blocks = spark + .read() + .textFile(testOutputBasePath + "/" + testActionSetId + "/software_blockstats") + .count(); - long ds_blocks = spark - .read() - .textFile(testOutputBasePath + "/" + testActionSetId + "/dataset_blockstats") - .count(); + long ds_blocks = spark + .read() + .textFile(testOutputBasePath + "/" + testActionSetId + "/dataset_blockstats") + .count(); - long orp_blocks = spark - .read() - .textFile(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_blockstats") - .count(); + long orp_blocks = spark + .read() + .textFile(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_blockstats") + .count(); - assertEquals(121, orgs_blocks); - assertEquals(110, pubs_blocks); - assertEquals(21, sw_blocks); - assertEquals(67, ds_blocks); - assertEquals(55, orp_blocks); - } + assertEquals(121, orgs_blocks); + assertEquals(110, pubs_blocks); + assertEquals(21, sw_blocks); + assertEquals(67, ds_blocks); + assertEquals(55, orp_blocks); + } } - From b8a45041fdbf0142a2abdfb36b9d5a1bb21551a7 Mon Sep 17 00:00:00 2001 From: miconis Date: Mon, 13 Jul 2020 19:53:18 +0200 Subject: [PATCH 053/100] minor changes --- .../main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java index bfd98e78e..31aa1ae81 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java @@ -29,7 +29,7 @@ import scala.Tuple2; public class SparkBlockStats extends AbstractSparkAction { - private static final Logger log = LoggerFactory.getLogger(SparkCreateSimRels.class); + private static final Logger log = LoggerFactory.getLogger(SparkBlockStats.class); public SparkBlockStats(ArgumentApplicationParser parser, SparkSession spark) { super(parser, spark); @@ -39,7 +39,7 @@ public class SparkBlockStats extends AbstractSparkAction { ArgumentApplicationParser parser = new ArgumentApplicationParser( IOUtils .toString( - SparkCreateSimRels.class + SparkBlockStats.class .getResourceAsStream( "/eu/dnetlib/dhp/oa/dedup/createBlockStats_parameters.json"))); parser.parseArgument(args); @@ -52,7 +52,7 @@ public class SparkBlockStats extends AbstractSparkAction { MapDocument.class, FieldListImpl.class, FieldValueImpl.class, Block.class }); - new SparkCreateSimRels(parser, getSparkSession(conf)) + new SparkBlockStats(parser, getSparkSession(conf)) .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); } From 06def0c0cb7ac5f2bef1a56b9a4fad3bc9ea3415 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 13 Jul 2020 20:09:06 +0200 Subject: [PATCH 054/100] SparkBlockStats allows to repartition the input rdd via the numPartitions workflow parameter --- .../dhp/oa/dedup/AbstractSparkAction.java | 2 + .../dnetlib/dhp/oa/dedup/SparkBlockStats.java | 61 +++++++++---------- .../dhp/oa/dedup/SparkCreateSimRels.java | 2 - .../oa/dedup/createBlockStats_parameters.json | 6 ++ .../dedup/statistics/oozie_app/workflow.xml | 13 ++-- 5 files changed, 47 insertions(+), 37 deletions(-) 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 2120da080..74cecb7b6 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 @@ -28,6 +28,8 @@ import eu.dnetlib.pace.config.DedupConfig; abstract class AbstractSparkAction implements Serializable { + protected static final int NUM_PARTITIONS = 1000; + protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper() .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java index 49f8123e8..d5de30967 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java @@ -2,6 +2,7 @@ package eu.dnetlib.dhp.oa.dedup; import java.io.IOException; +import java.util.Optional; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; @@ -23,49 +24,41 @@ 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.model.FieldListImpl; -import eu.dnetlib.pace.model.FieldValueImpl; import eu.dnetlib.pace.model.MapDocument; import eu.dnetlib.pace.util.MapDocumentUtil; import scala.Tuple2; public class SparkBlockStats extends AbstractSparkAction { - private static final Logger log = LoggerFactory.getLogger(SparkBlockStats.class); + private static final Logger log = LoggerFactory.getLogger(SparkBlockStats.class); - public SparkBlockStats(ArgumentApplicationParser parser, SparkSession spark) { - super(parser, spark); - } + public SparkBlockStats(ArgumentApplicationParser parser, SparkSession spark) { + super(parser, spark); + } - public static void main(String[] args) throws Exception { - ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkBlockStats.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/createBlockStats_parameters.json"))); - parser.parseArgument(args); + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkBlockStats.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/createBlockStats_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(); - new SparkBlockStats(parser, getSparkSession(conf)) - .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); - } + new SparkBlockStats(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } - public Long computeComparisons(Long blockSize, Long slidingWindowSize){ + public Long computeComparisons(Long blockSize, Long slidingWindowSize) { - if (slidingWindowSize >= blockSize) - return (slidingWindowSize * (slidingWindowSize - 1)) / 2; - else { - return (blockSize - slidingWindowSize + 1) * (slidingWindowSize * (slidingWindowSize - 1)) / 2; - } - } + if (slidingWindowSize >= blockSize) + return (slidingWindowSize * (slidingWindowSize - 1)) / 2; + else { + return (blockSize - slidingWindowSize + 1) * (slidingWindowSize * (slidingWindowSize - 1)) / 2; + } + } @Override public void run(ISLookUpService isLookUpService) @@ -76,6 +69,10 @@ public class SparkBlockStats extends AbstractSparkAction { final String isLookUpUrl = parser.get("isLookUpUrl"); final String actionSetId = parser.get("actionSetId"); final String workingPath = parser.get("workingPath"); + final int numPartitions = Optional + .ofNullable(parser.get("numPartitions")) + .map(Integer::valueOf) + .orElse(NUM_PARTITIONS); log.info("graphBasePath: '{}'", graphBasePath); log.info("isLookUpUrl: '{}'", isLookUpUrl); @@ -95,6 +92,7 @@ public class SparkBlockStats extends AbstractSparkAction { JavaPairRDD mapDocuments = sc .textFile(DedupUtility.createEntityPath(graphBasePath, subEntity)) + .repartition(numPartitions) .mapToPair( (PairFunction) s -> { MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s); @@ -105,6 +103,7 @@ public class SparkBlockStats extends AbstractSparkAction { JavaPairRDD blocks = Deduper.createSortedBlocks(mapDocuments, dedupConf); JavaRDD blockStats = blocks + .repartition(numPartitions) .map( b -> new BlockStats( b._1(), 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 3beb90e0b..b3ee47bfc 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 @@ -35,8 +35,6 @@ public class SparkCreateSimRels extends AbstractSparkAction { private static final Logger log = LoggerFactory.getLogger(SparkCreateSimRels.class); - public static final int NUM_PARTITIONS = 1000; - public SparkCreateSimRels(ArgumentApplicationParser parser, SparkSession spark) { super(parser, spark); } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createBlockStats_parameters.json b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createBlockStats_parameters.json index ce38dc6f0..09f4365d3 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createBlockStats_parameters.json +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createBlockStats_parameters.json @@ -22,5 +22,11 @@ "paramLongName": "workingPath", "paramDescription": "path of the working directory", "paramRequired": true + }, + { + "paramName": "np", + "paramLongName": "numPartitions", + "paramDescription": "number of partitions for the similarity relations intermediate phases", + "paramRequired": false } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/oozie_app/workflow.xml index 8b75d16b3..c0080b028 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/oozie_app/workflow.xml @@ -12,6 +12,10 @@ actionSetId id of the actionSet + + numPartitions + number of partitions for the similarity relations intermediate phases + sparkDriverMemory memory for driver process @@ -90,10 +94,11 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.shuffle.partitions=3840 - --i${graphBasePath} - --la${isLookUpUrl} - --asi${actionSetId} - --w${workingDir} + --graphBasePath${graphBasePath} + --isLookUpUrl${isLookUpUrl} + --actionSetId${actionSetId} + --workingPath${workingDir} + --numPartitions${numPartitions} From 8e97598eb4b56cc1ca05260e59b81e6bfae4aff4 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 13 Jul 2020 20:46:14 +0200 Subject: [PATCH 055/100] avoid to NPE in case of null instances --- .../CreateRelatedEntitiesJob_phase1.java | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) 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 57dca7bb1..f3188007a 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 @@ -184,13 +184,16 @@ public class CreateRelatedEntitiesJob_phase1 { re.setDateofacceptance(getValue(result.getDateofacceptance())); re.setPublisher(getValue(result.getPublisher())); re.setResulttype(result.getResulttype()); - re - .setInstances( - result - .getInstance() - .stream() - .limit(ProvisionConstants.MAX_INSTANCES) - .collect(Collectors.toList())); + if (Objects.nonNull(result.getInstance())) { + re + .setInstances( + result + .getInstance() + .stream() + .filter(Objects::nonNull) + .limit(ProvisionConstants.MAX_INSTANCES) + .collect(Collectors.toList())); + } // TODO still to be mapped // re.setCodeRepositoryUrl(j.read("$.coderepositoryurl")); From 7d6e269b40d0b54b33fe450c6008f973b77ed708 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 13 Jul 2020 22:54:04 +0200 Subject: [PATCH 056/100] reverted CreateRelatedEntitiesJob_phase1 to its previous state --- .../CreateRelatedEntitiesJob_phase1.java | 17 ++--------------- 1 file changed, 2 insertions(+), 15 deletions(-) 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 f3188007a..b08e593f7 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 @@ -116,23 +116,10 @@ public class CreateRelatedEntitiesJob_phase1 { Encoders.tuple(Encoders.STRING(), Encoders.kryo(Relation.class))) .cache(); - final String relatedEntityPath = outputPath + "_relatedEntity"; - readPathEntity(spark, inputEntityPath, clazz) + Dataset> entities = readPathEntity(spark, inputEntityPath, clazz) .filter("dataInfo.invisible == false") .map( - (MapFunction) value -> asRelatedEntity(value, clazz), - Encoders.kryo(RelatedEntity.class)) - .repartition(5000) - .write() - .mode(SaveMode.Overwrite) - .parquet(relatedEntityPath); - - Dataset> entities = spark - .read() - .load(relatedEntityPath) - .as(Encoders.kryo(RelatedEntity.class)) - .map( - (MapFunction>) e -> new Tuple2<>(e.getId(), e), + (MapFunction>) e -> new Tuple2<>(e.getId(), asRelatedEntity(e, clazz)), Encoders.tuple(Encoders.STRING(), Encoders.kryo(RelatedEntity.class))) .cache(); From 262c29463e19d696899d3e6a730cb9bbf7a19d26 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Wed, 15 Jul 2020 09:18:40 +0200 Subject: [PATCH 057/100] relations with multiple datasources --- .../dhp/broker/model/EventFactory.java | 24 ++- .../dnetlib/dhp/broker/oa/JoinStep0Job.java | 20 +- .../oa/PrepareRelatedDatasourcesJob.java | 55 ++++- .../dhp/broker/oa/matchers/UpdateMatcher.java | 10 +- .../dhp/broker/oa/util/BrokerConstants.java | 4 + .../dhp/broker/oa/util/ConversionUtils.java | 16 +- .../util/DatasourceRelationsAccumulator.java | 68 ++++++ .../dhp/broker/oa/util/EventFinder.java | 17 +- .../dhp/broker/oa/util/UpdateInfo.java | 25 ++- .../withRels/RelatedDatasource.java | 42 ++++ ....java => RelatedDatasourceAggregator.java} | 24 ++- .../withRels/SimpleDatasourceInfo.java | 40 ---- .../oa/generate_all/oozie_app/workflow.xml | 24 +++ .../broker/oa/partial/oozie_app/workflow.xml | 199 +++++++++++++++++- .../broker/oa/matchers/UpdateMatcherTest.java | 20 +- pom.xml | 2 +- 16 files changed, 495 insertions(+), 95 deletions(-) create mode 100644 dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/DatasourceRelationsAccumulator.java create mode 100644 dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/RelatedDatasource.java rename dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/{AddDatasourceTypeAggregator.java => RelatedDatasourceAggregator.java} (55%) delete mode 100644 dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/SimpleDatasourceInfo.java diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/model/EventFactory.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/model/EventFactory.java index 49e750698..4a58cfd36 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/model/EventFactory.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/model/EventFactory.java @@ -11,6 +11,8 @@ import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.time.DateUtils; import eu.dnetlib.broker.objects.OaBrokerMainEntity; +import eu.dnetlib.broker.objects.OaBrokerRelatedDatasource; +import eu.dnetlib.dhp.broker.oa.util.BrokerConstants; import eu.dnetlib.dhp.broker.oa.util.UpdateInfo; public class EventFactory { @@ -52,9 +54,11 @@ public class EventFactory { final OaBrokerMainEntity source = updateInfo.getSource(); final OaBrokerMainEntity target = updateInfo.getTarget(); - map.setTargetDatasourceId(target.getCollectedFromId()); - map.setTargetDatasourceName(target.getCollectedFromName()); - map.setTargetDatasourceType(target.getCollectedFromType()); + final OaBrokerRelatedDatasource targetDs = updateInfo.getTargetDs(); + + map.setTargetDatasourceId(targetDs.getOpenaireId()); + map.setTargetDatasourceName(targetDs.getName()); + map.setTargetDatasourceType(targetDs.getType()); map.setTargetResultId(target.getOpenaireId()); @@ -73,11 +77,19 @@ public class EventFactory { // PROVENANCE INFO map.setTrust(updateInfo.getTrust()); - map.setProvenanceDatasourceId(source.getCollectedFromId()); - map.setProvenanceDatasourceName(source.getCollectedFromName()); - map.setProvenanceDatasourceType(source.getCollectedFromType()); map.setProvenanceResultId(source.getOpenaireId()); + source + .getDatasources() + .stream() + .filter(ds -> ds.getRelType().equals(BrokerConstants.COLLECTED_FROM_REL)) + .findFirst() + .ifPresent(ds -> { + map.setProvenanceDatasourceId(ds.getOpenaireId()); + map.setProvenanceDatasourceName(ds.getName()); + map.setProvenanceDatasourceType(ds.getType()); + }); + return map; } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep0Job.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep0Job.java index eb1825fa5..39fa76e43 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep0Job.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep0Job.java @@ -17,8 +17,8 @@ import org.slf4j.LoggerFactory; import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.broker.oa.util.ClusterUtils; -import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.AddDatasourceTypeAggregator; -import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.SimpleDatasourceInfo; +import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedDatasource; +import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedDatasourceAggregator; import scala.Tuple2; public class JoinStep0Job { @@ -45,33 +45,33 @@ public class JoinStep0Job { final String workingPath = parser.get("workingPath"); log.info("workingPath: {}", workingPath); - final String outputPath = workingPath + "/joinedEntities_step0"; - log.info("outputPath: {}", outputPath); + final String joinedEntitiesPath = workingPath + "/joinedEntities_step0"; + log.info("joinedEntitiesPath: {}", joinedEntitiesPath); final SparkConf conf = new SparkConf(); runWithSparkSession(conf, isSparkSessionManaged, spark -> { - ClusterUtils.removeDir(spark, outputPath); + ClusterUtils.removeDir(spark, joinedEntitiesPath); final LongAccumulator total = spark.sparkContext().longAccumulator("total_entities"); final Dataset sources = ClusterUtils .readPath(spark, workingPath + "/simpleEntities", OaBrokerMainEntity.class); - final Dataset datasources = ClusterUtils - .readPath(spark, workingPath + "/datasources", SimpleDatasourceInfo.class); + final Dataset typedRels = ClusterUtils + .readPath(spark, workingPath + "/relatedDatasources", RelatedDatasource.class); - final TypedColumn, OaBrokerMainEntity> aggr = new AddDatasourceTypeAggregator() + final TypedColumn, OaBrokerMainEntity> aggr = new RelatedDatasourceAggregator() .toColumn(); final Dataset dataset = sources - .joinWith(datasources, sources.col("collectedFromId").equalTo(datasources.col("id")), "inner") + .joinWith(typedRels, sources.col("openaireId").equalTo(typedRels.col("source")), "left_outer") .groupByKey(t -> t._1.getOpenaireId(), Encoders.STRING()) .agg(aggr) .map(t -> t._2, Encoders.bean(OaBrokerMainEntity.class)); - ClusterUtils.save(dataset, outputPath, OaBrokerMainEntity.class, total); + ClusterUtils.save(dataset, joinedEntitiesPath, OaBrokerMainEntity.class, total); }); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedDatasourcesJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedDatasourcesJob.java index 30f5ddac3..166372a7f 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedDatasourcesJob.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedDatasourcesJob.java @@ -9,14 +9,23 @@ import org.apache.commons.io.IOUtils; 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.apache.spark.util.LongAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import eu.dnetlib.broker.objects.OaBrokerRelatedDatasource; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.broker.oa.util.ClusterUtils; -import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.SimpleDatasourceInfo; +import eu.dnetlib.dhp.broker.oa.util.ConversionUtils; +import eu.dnetlib.dhp.broker.oa.util.DatasourceRelationsAccumulator; +import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedDatasource; import eu.dnetlib.dhp.schema.oaf.Datasource; +import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct; +import eu.dnetlib.dhp.schema.oaf.Publication; +import eu.dnetlib.dhp.schema.oaf.Result; +import eu.dnetlib.dhp.schema.oaf.Software; +import scala.Tuple3; public class PrepareRelatedDatasourcesJob { @@ -42,7 +51,7 @@ public class PrepareRelatedDatasourcesJob { final String workingPath = parser.get("workingPath"); log.info("workingPath: {}", workingPath); - final String relsPath = workingPath + "/datasources"; + final String relsPath = workingPath + "/relatedDatasources"; log.info("relsPath: {}", relsPath); final SparkConf conf = new SparkConf(); @@ -53,16 +62,46 @@ public class PrepareRelatedDatasourcesJob { final LongAccumulator total = spark.sparkContext().longAccumulator("total_datasources"); - final Dataset dataset = ClusterUtils - .readPath(spark, graphPath + "/datasource", Datasource.class) - .map( - ds -> new SimpleDatasourceInfo(ds.getId(), ds.getDatasourcetype().getClassid()), - Encoders.bean(SimpleDatasourceInfo.class)); + final Dataset> rels = prepareResultTuples( + spark, graphPath, Publication.class) + .union(prepareResultTuples(spark, graphPath, eu.dnetlib.dhp.schema.oaf.Dataset.class)) + .union(prepareResultTuples(spark, graphPath, Software.class)) + .union(prepareResultTuples(spark, graphPath, OtherResearchProduct.class)); - ClusterUtils.save(dataset, relsPath, SimpleDatasourceInfo.class, total); + final Dataset datasources = ClusterUtils + .readPath(spark, graphPath + "/datasource", Datasource.class) + .map(ConversionUtils::oafDatasourceToBrokerDatasource, Encoders.bean(OaBrokerRelatedDatasource.class)); + + final Dataset dataset = rels + .joinWith(datasources, datasources.col("openaireId").equalTo(rels.col("_2")), "inner") + .map(t -> { + final RelatedDatasource r = new RelatedDatasource(); + r.setSource(t._1._1()); + r.setRelDatasource(t._2); + r.getRelDatasource().setRelType(t._1._3()); + return r; + }, Encoders.bean(RelatedDatasource.class)); + + ClusterUtils.save(dataset, relsPath, RelatedDatasource.class, total); }); } + private static final Dataset> prepareResultTuples(final SparkSession spark, + final String graphPath, + final Class sourceClass) { + + return ClusterUtils + .readPath(spark, graphPath + "/" + sourceClass.getSimpleName().toLowerCase(), sourceClass) + .filter(r -> !ClusterUtils.isDedupRoot(r.getId())) + .filter(r -> r.getDataInfo().getDeletedbyinference()) + .map( + r -> DatasourceRelationsAccumulator.calculateTuples(r), + Encoders.bean(DatasourceRelationsAccumulator.class)) + .flatMap( + acc -> acc.getRels().iterator(), + Encoders.tuple(Encoders.STRING(), Encoders.STRING(), Encoders.STRING())); + } + } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcher.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcher.java index 3d688fa1d..fba82aa8c 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcher.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcher.java @@ -15,6 +15,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.spark.util.LongAccumulator; import eu.dnetlib.broker.objects.OaBrokerMainEntity; +import eu.dnetlib.broker.objects.OaBrokerRelatedDatasource; import eu.dnetlib.dhp.broker.model.Topic; import eu.dnetlib.dhp.broker.oa.util.UpdateInfo; @@ -34,18 +35,19 @@ public abstract class UpdateMatcher { this.highlightToStringFunction = highlightToStringFunction; } - public Collection> searchUpdatesForRecord(final OaBrokerMainEntity res, + public Collection> searchUpdatesForRecord(final OaBrokerMainEntity target, + final OaBrokerRelatedDatasource targetDs, final Collection others, final Map accumulators) { final Map> infoMap = new HashMap<>(); for (final OaBrokerMainEntity source : others) { - if (source != res) { - for (final T hl : findDifferences(source, res)) { + if (source != target) { + for (final T hl : findDifferences(source, target)) { final Topic topic = getTopicFunction().apply(hl); if (topic != null) { - final UpdateInfo info = new UpdateInfo<>(topic, hl, source, res, + final UpdateInfo info = new UpdateInfo<>(topic, hl, source, target, targetDs, getCompileHighlightFunction(), getHighlightToStringFunction()); diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/BrokerConstants.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/BrokerConstants.java index 5308b9dff..7a09862d8 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/BrokerConstants.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/BrokerConstants.java @@ -14,6 +14,10 @@ public class BrokerConstants { public static final String OPEN_ACCESS = "OPEN"; public static final String IS_MERGED_IN_CLASS = "isMergedIn"; + public static final String COLLECTED_FROM_REL = "collectedFrom"; + + public static final String HOSTED_BY_REL = "hostedBy"; + public static final float MIN_TRUST = 0.25f; public static final float MAX_TRUST = 1.00f; diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java index d00c5b817..053627a5f 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ConversionUtils.java @@ -22,11 +22,13 @@ import eu.dnetlib.broker.objects.OaBrokerJournal; import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.broker.objects.OaBrokerProject; import eu.dnetlib.broker.objects.OaBrokerRelatedDataset; +import eu.dnetlib.broker.objects.OaBrokerRelatedDatasource; import eu.dnetlib.broker.objects.OaBrokerRelatedPublication; import eu.dnetlib.broker.objects.OaBrokerRelatedSoftware; import eu.dnetlib.broker.objects.OaBrokerTypedValue; import eu.dnetlib.dhp.schema.oaf.Author; import eu.dnetlib.dhp.schema.oaf.Dataset; +import eu.dnetlib.dhp.schema.oaf.Datasource; import eu.dnetlib.dhp.schema.oaf.ExternalReference; import eu.dnetlib.dhp.schema.oaf.Field; import eu.dnetlib.dhp.schema.oaf.Instance; @@ -119,8 +121,6 @@ public class ConversionUtils { res .setJournal( result instanceof Publication ? oafJournalToBrokerJournal(((Publication) result).getJournal()) : null); - res.setCollectedFromId(mappedFirst(result.getCollectedfrom(), KeyValue::getKey)); - res.setCollectedFromName(mappedFirst(result.getCollectedfrom(), KeyValue::getValue)); res.setPids(mappedList(result.getPid(), ConversionUtils::oafPidToBrokerPid)); res.setInstances(flatMappedList(result.getInstance(), ConversionUtils::oafInstanceToBrokerInstances)); res @@ -223,6 +223,18 @@ public class ConversionUtils { return res; } + public static final OaBrokerRelatedDatasource oafDatasourceToBrokerDatasource(final Datasource ds) { + if (ds == null) { + return null; + } + + final OaBrokerRelatedDatasource res = new OaBrokerRelatedDatasource(); + res.setName(StringUtils.defaultIfBlank(fieldValue(ds.getOfficialname()), fieldValue(ds.getEnglishname()))); + res.setOpenaireId(ds.getId()); + res.setType(classId(ds.getDatasourcetype())); + return res; + } + private static String first(final List list) { return list != null && list.size() > 0 ? list.get(0) : null; } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/DatasourceRelationsAccumulator.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/DatasourceRelationsAccumulator.java new file mode 100644 index 000000000..75c4625ce --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/DatasourceRelationsAccumulator.java @@ -0,0 +1,68 @@ + +package eu.dnetlib.dhp.broker.oa.util; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.commons.lang3.StringUtils; + +import eu.dnetlib.dhp.schema.oaf.Result; +import scala.Tuple3; + +public class DatasourceRelationsAccumulator implements Serializable { + + /** + * + */ + private static final long serialVersionUID = 3256220670651218957L; + + private List> rels = new ArrayList<>(); + + public List> getRels() { + return rels; + } + + public void setRels(final List> rels) { + this.rels = rels; + } + + protected void addTuple(final Tuple3 t) { + rels.add(t); + } + + public static final DatasourceRelationsAccumulator calculateTuples(final Result r) { + + final Set collectedFromSet = r + .getCollectedfrom() + .stream() + .map(kv -> kv.getKey()) + .filter(StringUtils::isNotBlank) + .distinct() + .collect(Collectors.toSet()); + + final Set hostedBySet = r + .getInstance() + .stream() + .map(i -> i.getHostedby()) + .filter(Objects::nonNull) + .filter(kv -> !StringUtils.equalsIgnoreCase(kv.getValue(), "Unknown Repository")) + .map(kv -> kv.getKey()) + .filter(StringUtils::isNotBlank) + .distinct() + .filter(id -> !collectedFromSet.contains(id)) + .collect(Collectors.toSet()); + + final DatasourceRelationsAccumulator res = new DatasourceRelationsAccumulator(); + collectedFromSet + .stream() + .map(s -> new Tuple3<>(r.getId(), s, BrokerConstants.COLLECTED_FROM_REL)) + .forEach(res::addTuple); + hostedBySet.stream().map(s -> new Tuple3<>(r.getId(), s, BrokerConstants.HOSTED_BY_REL)).forEach(res::addTuple); + return res; + } + +} diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java index b6328eb95..1ab56cc34 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/EventFinder.java @@ -11,6 +11,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import eu.dnetlib.broker.objects.OaBrokerMainEntity; +import eu.dnetlib.broker.objects.OaBrokerRelatedDatasource; import eu.dnetlib.dhp.broker.model.EventFactory; import eu.dnetlib.dhp.broker.oa.matchers.UpdateMatcher; import eu.dnetlib.dhp.broker.oa.matchers.relatedDatasets.EnrichMissingDatasetIsReferencedBy; @@ -80,9 +81,11 @@ public class EventFinder { final List> list = new ArrayList<>(); for (final OaBrokerMainEntity target : results.getData()) { - if (verifyTarget(target, dsIdWhitelist, dsIdBlacklist, dsTypeWhitelist)) { - for (final UpdateMatcher matcher : matchers) { - list.addAll(matcher.searchUpdatesForRecord(target, results.getData(), accumulators)); + for (final OaBrokerRelatedDatasource targetDs : target.getDatasources()) { + if (verifyTarget(targetDs, dsIdWhitelist, dsIdBlacklist, dsTypeWhitelist)) { + for (final UpdateMatcher matcher : matchers) { + list.addAll(matcher.searchUpdatesForRecord(target, targetDs, results.getData(), accumulators)); + } } } } @@ -90,17 +93,17 @@ public class EventFinder { return asEventGroup(list); } - private static boolean verifyTarget(final OaBrokerMainEntity target, + private static boolean verifyTarget(final OaBrokerRelatedDatasource target, final Set dsIdWhitelist, final Set dsIdBlacklist, final Set dsTypeWhitelist) { - if (dsIdWhitelist.contains(target.getCollectedFromId())) { + if (dsIdWhitelist.contains(target.getOpenaireId())) { return true; - } else if (dsIdBlacklist.contains(target.getCollectedFromId())) { + } else if (dsIdBlacklist.contains(target.getOpenaireId())) { return false; } else { - return dsTypeWhitelist.contains(target.getCollectedFromType()); + return dsTypeWhitelist.contains(target.getType()); } } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/UpdateInfo.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/UpdateInfo.java index ef8fb240c..fca954247 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/UpdateInfo.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/UpdateInfo.java @@ -8,6 +8,7 @@ import eu.dnetlib.broker.objects.OaBrokerEventPayload; import eu.dnetlib.broker.objects.OaBrokerInstance; import eu.dnetlib.broker.objects.OaBrokerMainEntity; import eu.dnetlib.broker.objects.OaBrokerProvenance; +import eu.dnetlib.broker.objects.OaBrokerRelatedDatasource; import eu.dnetlib.dhp.broker.model.Topic; public final class UpdateInfo { @@ -20,6 +21,8 @@ public final class UpdateInfo { private final OaBrokerMainEntity target; + private final OaBrokerRelatedDatasource targetDs; + private final BiConsumer compileHighlight; private final Function highlightToString; @@ -28,12 +31,14 @@ public final class UpdateInfo { public UpdateInfo(final Topic topic, final T highlightValue, final OaBrokerMainEntity source, final OaBrokerMainEntity target, + final OaBrokerRelatedDatasource targetDs, final BiConsumer compileHighlight, final Function highlightToString) { this.topic = topic; this.highlightValue = highlightValue; this.source = source; this.target = target; + this.targetDs = targetDs; this.compileHighlight = compileHighlight; this.highlightToString = highlightToString; this.trust = TrustUtils.calculateTrust(source, target); @@ -51,6 +56,10 @@ public final class UpdateInfo { return target; } + public OaBrokerRelatedDatasource getTargetDs() { + return targetDs; + } + protected Topic getTopic() { return topic; } @@ -75,8 +84,20 @@ public final class UpdateInfo { compileHighlight.accept(hl, getHighlightValue()); final String provId = getSource().getOpenaireId(); - final String provRepo = getSource().getCollectedFromName(); - final String provType = getSource().getCollectedFromType(); + final String provRepo = getSource() + .getDatasources() + .stream() + .filter(ds -> ds.getRelType().equals(BrokerConstants.COLLECTED_FROM_REL)) + .map(ds -> ds.getName()) + .findFirst() + .orElse(""); + final String provType = getSource() + .getDatasources() + .stream() + .filter(ds -> ds.getRelType().equals(BrokerConstants.COLLECTED_FROM_REL)) + .map(ds -> ds.getType()) + .findFirst() + .orElse(""); final String provUrl = getSource() .getInstances() diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/RelatedDatasource.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/RelatedDatasource.java new file mode 100644 index 000000000..a27df502b --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/RelatedDatasource.java @@ -0,0 +1,42 @@ + +package eu.dnetlib.dhp.broker.oa.util.aggregators.withRels; + +import java.io.Serializable; + +import eu.dnetlib.broker.objects.OaBrokerRelatedDatasource; + +public class RelatedDatasource implements Serializable { + + /** + * + */ + private static final long serialVersionUID = 3015550240920424010L; + + private String source; + private OaBrokerRelatedDatasource relDatasource; + + public RelatedDatasource() { + } + + public RelatedDatasource(final String source, final OaBrokerRelatedDatasource relDatasource) { + this.source = source; + this.relDatasource = relDatasource; + } + + public String getSource() { + return source; + } + + public void setSource(final String source) { + this.source = source; + } + + public OaBrokerRelatedDatasource getRelDatasource() { + return relDatasource; + } + + public void setRelDatasource(final OaBrokerRelatedDatasource relDatasource) { + this.relDatasource = relDatasource; + } + +} diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/AddDatasourceTypeAggregator.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/RelatedDatasourceAggregator.java similarity index 55% rename from dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/AddDatasourceTypeAggregator.java rename to dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/RelatedDatasourceAggregator.java index ccd15c8c6..2c0c7917d 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/AddDatasourceTypeAggregator.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/RelatedDatasourceAggregator.java @@ -7,15 +7,16 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.expressions.Aggregator; import eu.dnetlib.broker.objects.OaBrokerMainEntity; +import eu.dnetlib.dhp.broker.oa.util.BrokerConstants; import scala.Tuple2; -public class AddDatasourceTypeAggregator - extends Aggregator, OaBrokerMainEntity, OaBrokerMainEntity> { +public class RelatedDatasourceAggregator + extends Aggregator, OaBrokerMainEntity, OaBrokerMainEntity> { /** * */ - private static final long serialVersionUID = 8788588975496014728L; + private static final long serialVersionUID = -7212121913834713672L; @Override public OaBrokerMainEntity zero() { @@ -29,10 +30,10 @@ public class AddDatasourceTypeAggregator @Override public OaBrokerMainEntity reduce(final OaBrokerMainEntity g, - final Tuple2 t) { + final Tuple2 t) { final OaBrokerMainEntity res = StringUtils.isNotBlank(g.getOpenaireId()) ? g : t._1; - if (t._2 != null && StringUtils.isNotBlank(t._2.getType())) { - res.setCollectedFromType(t._2.getType()); + if (t._2 != null && res.getDatasources().size() < BrokerConstants.MAX_NUMBER_OF_RELS) { + res.getDatasources().add(t._2.getRelDatasource()); } return res; @@ -40,7 +41,15 @@ public class AddDatasourceTypeAggregator @Override public OaBrokerMainEntity merge(final OaBrokerMainEntity g1, final OaBrokerMainEntity g2) { - if (StringUtils.isNotBlank(g1.getOpenaireId()) && StringUtils.isNotBlank(g1.getCollectedFromType())) { + if (StringUtils.isNotBlank(g1.getOpenaireId())) { + final int availables = BrokerConstants.MAX_NUMBER_OF_RELS - g1.getDatasources().size(); + if (availables > 0) { + if (g2.getDatasources().size() <= availables) { + g1.getDatasources().addAll(g2.getDatasources()); + } else { + g1.getDatasources().addAll(g2.getDatasources().subList(0, availables)); + } + } return g1; } else { return g2; @@ -56,4 +65,5 @@ public class AddDatasourceTypeAggregator public Encoder outputEncoder() { return Encoders.bean(OaBrokerMainEntity.class); } + } diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/SimpleDatasourceInfo.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/SimpleDatasourceInfo.java deleted file mode 100644 index 966f63fa0..000000000 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/SimpleDatasourceInfo.java +++ /dev/null @@ -1,40 +0,0 @@ - -package eu.dnetlib.dhp.broker.oa.util.aggregators.withRels; - -import java.io.Serializable; - -public class SimpleDatasourceInfo implements Serializable { - - /** - * - */ - private static final long serialVersionUID = 2996609859416024734L; - - private String id; - private String type; - - public SimpleDatasourceInfo() { - } - - public SimpleDatasourceInfo(final String id, final String type) { - this.id = id; - this.type = type; - } - - public String getId() { - return id; - } - - public void setId(final String id) { - this.id = id; - } - - public String getType() { - return type; - } - - public void setType(final String type) { - this.type = type; - } - -} diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml index 2c728cd98..568d5dc5a 100644 --- a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_all/oozie_app/workflow.xml @@ -448,6 +448,30 @@ --index${esIndexName} --esHost${esIndexHost} + + + + + + + yarn + cluster + GenerateStatsJob + eu.dnetlib.dhp.broker.oa.GenerateStatsJob + dhp-broker-events-${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.sql.shuffle.partitions=3840 + + --graphPath${graphInputPath} + --workingPath${workingPath} + diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml index b4155f93f..2271a9e0e 100644 --- a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml @@ -64,14 +64,209 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + yarn + cluster + JoinStep0 + eu.dnetlib.dhp.broker.oa.JoinStep0Job + dhp-broker-events-${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.sql.shuffle.partitions=3840 + + --graphPath${graphInputPath} + --workingPath${workingPath} + + + + + + + + yarn + cluster + JoinStep1 + eu.dnetlib.dhp.broker.oa.JoinStep1Job + dhp-broker-events-${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.sql.shuffle.partitions=3840 + + --graphPath${graphInputPath} + --workingPath${workingPath} + + + + + + + + yarn + cluster + JoinStep2 + eu.dnetlib.dhp.broker.oa.JoinStep2Job + dhp-broker-events-${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.sql.shuffle.partitions=3840 + + --graphPath${graphInputPath} + --workingPath${workingPath} + + + + + + + + yarn + cluster + JoinStep3 + eu.dnetlib.dhp.broker.oa.JoinStep3Job + dhp-broker-events-${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.sql.shuffle.partitions=3840 + + --graphPath${graphInputPath} + --workingPath${workingPath} + + + + + + + + yarn + cluster + JoinStep4 + eu.dnetlib.dhp.broker.oa.JoinStep4Job + dhp-broker-events-${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.sql.shuffle.partitions=3840 + + --graphPath${graphInputPath} + --workingPath${workingPath} + + + + - + + + yarn + cluster + PrepareGroupsJob + eu.dnetlib.dhp.broker.oa.PrepareGroupsJob + dhp-broker-events-${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.sql.shuffle.partitions=3840 + + --graphPath${graphInputPath} + --workingPath${workingPath} + + + + + + + + yarn + cluster + GenerateEventsJob + eu.dnetlib.dhp.broker.oa.GenerateEventsJob + dhp-broker-events-${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.sql.shuffle.partitions=3840 + + --workingPath${workingPath} + --datasourceIdWhitelist${datasourceIdWhitelist} + --datasourceTypeWhitelist${datasourceTypeWhitelist} + --datasourceIdBlacklist${datasourceIdBlacklist} + + + + + + + + yarn + cluster + IndexOnESJob + eu.dnetlib.dhp.broker.oa.IndexOnESJob + dhp-broker-events-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.dynamicAllocation.maxExecutors="8" + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.shuffle.partitions=3840 + + --workingPath${workingPath} + --index${esIndexName} + --esHost${esIndexHost} + + + + + + yarn cluster diff --git a/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcherTest.java b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcherTest.java index 82374b335..8fa95abe5 100644 --- a/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcherTest.java +++ b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcherTest.java @@ -8,15 +8,23 @@ import java.util.Collection; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; import eu.dnetlib.broker.objects.OaBrokerMainEntity; +import eu.dnetlib.broker.objects.OaBrokerRelatedDatasource; import eu.dnetlib.dhp.broker.oa.matchers.simple.EnrichMissingPublicationDate; import eu.dnetlib.dhp.broker.oa.util.UpdateInfo; +@ExtendWith(MockitoExtension.class) class UpdateMatcherTest { UpdateMatcher matcher = new EnrichMissingPublicationDate(); + @Mock + private OaBrokerRelatedDatasource targetDs; + @BeforeEach void setUp() throws Exception { } @@ -30,7 +38,7 @@ class UpdateMatcherTest { final OaBrokerMainEntity p4 = new OaBrokerMainEntity(); final Collection> list = matcher - .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null); + .searchUpdatesForRecord(res, targetDs, Arrays.asList(p1, p2, p3, p4), null); assertTrue(list.isEmpty()); } @@ -46,7 +54,7 @@ class UpdateMatcherTest { res.setPublicationdate("2018"); final Collection> list = matcher - .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null); + .searchUpdatesForRecord(res, targetDs, Arrays.asList(p1, p2, p3, p4), null); assertTrue(list.isEmpty()); } @@ -62,7 +70,7 @@ class UpdateMatcherTest { p2.setPublicationdate("2018"); final Collection> list = matcher - .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null); + .searchUpdatesForRecord(res, targetDs, Arrays.asList(p1, p2, p3, p4), null); assertTrue(list.size() == 1); } @@ -79,7 +87,7 @@ class UpdateMatcherTest { p2.setPublicationdate("2018"); final Collection> list = matcher - .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null); + .searchUpdatesForRecord(res, targetDs, Arrays.asList(p1, p2, p3, p4), null); assertTrue(list.isEmpty()); } @@ -98,7 +106,7 @@ class UpdateMatcherTest { p4.setPublicationdate("2018"); final Collection> list = matcher - .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null); + .searchUpdatesForRecord(res, targetDs, Arrays.asList(p1, p2, p3, p4), null); assertTrue(list.isEmpty()); } @@ -117,7 +125,7 @@ class UpdateMatcherTest { p4.setPublicationdate("2018"); final Collection> list = matcher - .searchUpdatesForRecord(res, Arrays.asList(p1, p2, p3, p4), null); + .searchUpdatesForRecord(res, targetDs, Arrays.asList(p1, p2, p3, p4), null); assertTrue(list.size() == 1); } diff --git a/pom.xml b/pom.xml index 411ef9521..eec6c303f 100644 --- a/pom.xml +++ b/pom.xml @@ -624,6 +624,6 @@ 3.3.3 3.4.2 [2.12,3.0) - 3.0.0 + 3.1.0
From 4e6f46e8fa1f8cc419487d5f9423757129cfb3d7 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 15 Jul 2020 11:22:20 +0200 Subject: [PATCH 058/100] filter blocks with one record only --- .../java/eu/dnetlib/dhp/oa/dedup/Deduper.java | 1 + .../dnetlib/dhp/oa/dedup/SparkBlockStats.java | 20 ++++++++++--------- 2 files changed, 12 insertions(+), 9 deletions(-) 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 180f9f846..5e8a50fcc 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 @@ -51,6 +51,7 @@ public class Deduper implements Serializable { .map(it -> Block.from(it, a)) .collect(Collectors.toList()) .iterator()) + .filter(b -> b.getDocuments().size() > 1) .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/SparkBlockStats.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java index d5de30967..8016361db 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java @@ -9,6 +9,7 @@ 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.FilterFunction; import org.apache.spark.api.java.function.PairFunction; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SaveMode; @@ -100,16 +101,9 @@ public class SparkBlockStats extends AbstractSparkAction { }); // create blocks for deduplication - JavaPairRDD blocks = Deduper.createSortedBlocks(mapDocuments, dedupConf); - - JavaRDD blockStats = blocks + JavaRDD blockStats = Deduper.createSortedBlocks(mapDocuments, dedupConf) .repartition(numPartitions) - .map( - b -> new BlockStats( - b._1(), - (long) b._2().getDocuments().size(), - computeComparisons( - (long) b._2().getDocuments().size(), (long) dedupConf.getWf().getSlidingWindowSize()))); + .map(b -> asBlockStats(dedupConf, b)); // save the blockstats in the workingdir spark @@ -120,4 +114,12 @@ public class SparkBlockStats extends AbstractSparkAction { } } + private BlockStats asBlockStats(DedupConfig dedupConf, Tuple2 b) { + return new BlockStats( + b._1(), + (long) b._2().getDocuments().size(), + computeComparisons( + (long) b._2().getDocuments().size(), (long) dedupConf.getWf().getSlidingWindowSize())); + } + } From b90389bac4fa6c76e17f8baf7b83f24a5e0e34be Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 15 Jul 2020 11:24:48 +0200 Subject: [PATCH 059/100] code formatting --- .../src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java index 8016361db..1e13485e5 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java @@ -101,7 +101,8 @@ public class SparkBlockStats extends AbstractSparkAction { }); // create blocks for deduplication - JavaRDD blockStats = Deduper.createSortedBlocks(mapDocuments, dedupConf) + JavaRDD blockStats = Deduper + .createSortedBlocks(mapDocuments, dedupConf) .repartition(numPartitions) .map(b -> asBlockStats(dedupConf, b)); From 5033c25587c9fbd6226b1c76c08e0bfa3c428bbb Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 15 Jul 2020 11:26:00 +0200 Subject: [PATCH 060/100] code formatting --- pom.xml | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/pom.xml b/pom.xml index 411ef9521..305bacda3 100644 --- a/pom.xml +++ b/pom.xml @@ -323,13 +323,13 @@ [2.0.0,3.0.0) - - eu.dnetlib.dhp - dnet-openaire-broker-common - ${dnet.openaire.broker.common} - - + eu.dnetlib.dhp + dnet-openaire-broker-common + ${dnet.openaire.broker.common} + + + org.apache.cxf cxf-rt-transports-http 3.1.5 From 805de4eca15822092f99bc6c1e64b0cbda95b668 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 16 Jul 2020 10:11:32 +0200 Subject: [PATCH 061/100] fix: filter the blocks with size = 1 --- .../src/main/java/eu/dnetlib/dhp/oa/dedup/Deduper.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 5e8a50fcc..68201677e 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 @@ -51,8 +51,8 @@ public class Deduper implements Serializable { .map(it -> Block.from(it, a)) .collect(Collectors.toList()) .iterator()) - .filter(b -> b.getDocuments().size() > 1) .mapToPair(block -> new Tuple2<>(block.getKey(), block)) - .reduceByKey((b1, b2) -> Block.from(b1, b2, of, maxQueueSize)); + .reduceByKey((b1, b2) -> Block.from(b1, b2, of, maxQueueSize)) + .filter(b -> b._2().getDocuments().size() > 1); } } From b098cc3cbe6f0d3281b47e1cef89eba61e663ed0 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 16 Jul 2020 13:45:53 +0200 Subject: [PATCH 062/100] avoid repeating identical values for fields: source, description --- .../eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 db9a68d3d..53d4c888e 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 @@ -329,7 +329,7 @@ public class XmlRecordFactory implements Serializable { .stream() .filter(Objects::nonNull) .map(c -> XmlSerializationUtils.asXmlElement("description", c.getValue())) - .collect(Collectors.toList())); + .collect(Collectors.toCollection(HashSet::new))); } if (r.getEmbargoenddate() != null) { metadata @@ -370,7 +370,7 @@ public class XmlRecordFactory implements Serializable { .stream() .filter(Objects::nonNull) .map(c -> XmlSerializationUtils.asXmlElement("source", c.getValue())) - .collect(Collectors.toList())); + .collect(Collectors.toCollection(HashSet::new))); } if (r.getFormat() != null) { metadata From cc5d13da85c4897f502b879af42fef40c639a7cf Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 16 Jul 2020 13:46:39 +0200 Subject: [PATCH 063/100] introduced parameter shouldIndex (true|false) --- .../eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml index 32bf7ce83..91ced378c 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-provision/src/main/resources/eu/dnetlib/dhp/oa/provision/oozie_app/workflow.xml @@ -578,10 +578,18 @@ --isLookupUrl${isLookupUrl} --otherDsTypeId${otherDsTypeId} - + + + + ${wf:conf('shouldIndex') eq 'true'} + ${wf:conf('shouldIndex') eq 'false'} + + + + yarn From db8b90a1566546ed8d34cdc7a19b5f3b3b9f28d9 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 16 Jul 2020 19:05:13 +0200 Subject: [PATCH 064/100] renamed CORE -> BETA --- ...{graph_core_construction.xml => graph_prod_construction.xml} | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/{graph_core_construction.xml => graph_prod_construction.xml} (99%) diff --git a/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_core_construction.xml b/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_prod_construction.xml similarity index 99% rename from dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_core_construction.xml rename to dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_prod_construction.xml index 3bce81c50..047433320 100644 --- a/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_core_construction.xml +++ b/dhp-workflows/dhp-worfklow-profiles/src/main/resources/eu/dnetlib/dhp/wf/profiles/graph_prod_construction.xml @@ -7,7 +7,7 @@ - Graph Construction [OCEAN] + Graph Construction [PROD] Data Provision 30 From 17816095084a80afab58ffda3b997d08e6c3d571 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 16 Jul 2020 19:06:56 +0200 Subject: [PATCH 065/100] code formatting --- .../oa/graph/merge/MergeGraphSparkJob.java | 184 +++++++++--------- 1 file changed, 94 insertions(+), 90 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJob.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJob.java index d33b9e846..8baa469bf 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJob.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJob.java @@ -1,11 +1,11 @@ + package eu.dnetlib.dhp.oa.graph.merge; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.oa.graph.clean.CleanGraphSparkJob; -import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +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.function.FilterFunction; @@ -16,13 +16,16 @@ 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.graph.clean.CleanGraphSparkJob; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; import scala.Tuple2; -import java.util.Objects; -import java.util.Optional; - -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; - /** * Combines the content from two aggregator graph tables of the same type, entities (or relationships) with the same ids * are picked preferring those from the BETA aggregator rather then from PROD. The identity of a relationship is defined @@ -30,101 +33,102 @@ import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; */ public class MergeGraphSparkJob { - private static final Logger log = LoggerFactory.getLogger(CleanGraphSparkJob.class); + private static final Logger log = LoggerFactory.getLogger(CleanGraphSparkJob.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( - CleanGraphSparkJob.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/graph/merge_graphs_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + String jsonConfiguration = IOUtils + .toString( + CleanGraphSparkJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/graph/merge_graphs_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 isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String betaInputPath = parser.get("betaInputPath"); - log.info("betaInputPath: {}", betaInputPath); + String betaInputPath = parser.get("betaInputPath"); + log.info("betaInputPath: {}", betaInputPath); - String prodInputPath = parser.get("prodInputPath"); - log.info("prodInputPath: {}", prodInputPath); + String prodInputPath = parser.get("prodInputPath"); + log.info("prodInputPath: {}", prodInputPath); - 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); - mergeGraphTable(spark, betaInputPath, prodInputPath, entityClazz, entityClazz, outputPath); - }); - } + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + mergeGraphTable(spark, betaInputPath, prodInputPath, entityClazz, entityClazz, outputPath); + }); + } - private static

void mergeGraphTable( - SparkSession spark, - String betaInputPath, - String prodInputPath, - Class

p_clazz, - Class b_clazz, - String outputPath) { + private static

void mergeGraphTable( + SparkSession spark, + String betaInputPath, + String prodInputPath, + Class

p_clazz, + Class b_clazz, + String outputPath) { - Dataset> beta = readTableFromPath(spark, betaInputPath, b_clazz); - Dataset> prod = readTableFromPath(spark, prodInputPath, p_clazz); + Dataset> beta = readTableFromPath(spark, betaInputPath, b_clazz); + Dataset> prod = readTableFromPath(spark, prodInputPath, p_clazz); - prod.joinWith(beta, prod.col("_1").equalTo(beta.col("_1")), "full_outer") - .map((MapFunction, Tuple2>, P>) value -> { - Optional

p = Optional.ofNullable(value._1()).map(Tuple2::_2); - Optional b = Optional.ofNullable(value._2()).map(Tuple2::_2); - if (p.isPresent() & !b.isPresent()) { - return p.get(); - } - if (b.isPresent()) { - return (P) b.get(); - } - return null; - }, Encoders.bean(p_clazz)) - .filter((FilterFunction

) Objects::nonNull) - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .json(outputPath); - } + prod + .joinWith(beta, prod.col("_1").equalTo(beta.col("_1")), "full_outer") + .map((MapFunction, Tuple2>, P>) value -> { + Optional

p = Optional.ofNullable(value._1()).map(Tuple2::_2); + Optional b = Optional.ofNullable(value._2()).map(Tuple2::_2); + if (p.isPresent() & !b.isPresent()) { + return p.get(); + } + if (b.isPresent()) { + return (P) b.get(); + } + return null; + }, Encoders.bean(p_clazz)) + .filter((FilterFunction

) Objects::nonNull) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath); + } - private static Dataset> readTableFromPath( - SparkSession spark, String inputEntityPath, Class clazz) { + private static Dataset> readTableFromPath( + SparkSession spark, String inputEntityPath, Class clazz) { - log.info("Reading Graph table from: {}", inputEntityPath); - return spark - .read() - .textFile(inputEntityPath) - .map( - (MapFunction>) value -> { - final T t = OBJECT_MAPPER.readValue(value, clazz); - final String id = ModelSupport.idFn().apply(t); - return new Tuple2<>(id, t); - }, - Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz))); - } + log.info("Reading Graph table from: {}", inputEntityPath); + return spark + .read() + .textFile(inputEntityPath) + .map( + (MapFunction>) value -> { + final T t = OBJECT_MAPPER.readValue(value, clazz); + final String id = ModelSupport.idFn().apply(t); + return new Tuple2<>(id, t); + }, + Encoders.tuple(Encoders.STRING(), Encoders.kryo(clazz))); + } - 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()); + } } From 442f30930c741c4d0e4741e2946926c695b42e95 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Fri, 17 Jul 2020 12:25:36 +0200 Subject: [PATCH 066/100] removed duplicated fields --- .../dhp/oa/graph/raw/common/OafMapperUtils.java | 12 ++++++++++++ .../eu/dnetlib/dhp/oa/graph/raw/MappersTest.java | 2 ++ .../eu/dnetlib/dhp/oa/graph/raw/oaf_record.xml | 2 ++ 3 files changed, 16 insertions(+) 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 8ede40773..63db13b8f 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 @@ -4,7 +4,11 @@ package eu.dnetlib.dhp.oa.graph.raw.common; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Function; +import java.util.function.Predicate; import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; @@ -57,6 +61,7 @@ public class OafMapperUtils { .stream(values) .map(v -> field(v, info)) .filter(Objects::nonNull) + .filter(distinctByKey(f -> f.getValue())) .collect(Collectors.toList()); } @@ -65,6 +70,7 @@ public class OafMapperUtils { .stream() .map(v -> field(v, info)) .filter(Objects::nonNull) + .filter(distinctByKey(f -> f.getValue())) .collect(Collectors.toList()); } @@ -237,4 +243,10 @@ public class OafMapperUtils { public static String asString(final Object o) { return o == null ? "" : o.toString(); } + + public static Predicate distinctByKey( + final Function keyExtractor) { + final Map seen = new ConcurrentHashMap<>(); + return t -> seen.putIfAbsent(keyExtractor.apply(t), Boolean.TRUE) == null; + } } 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 b1f0ecf0d..ffd21c86e 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 @@ -71,6 +71,7 @@ public class MappersTest { assertValidId(p.getCollectedfrom().get(0).getKey()); assertTrue(StringUtils.isNotBlank(p.getTitle().get(0).getValue())); assertFalse(p.getDataInfo().getInvisible()); + assertTrue(p.getSource().size() == 1); assertTrue(p.getAuthor().size() > 0); final Optional author = p @@ -79,6 +80,7 @@ public class MappersTest { .filter(a -> a.getPid() != null && !a.getPid().isEmpty()) .findFirst(); assertTrue(author.isPresent()); + final StructuredProperty pid = author .get() .getPid() diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_record.xml b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_record.xml index ead22aa96..be78bf35c 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_record.xml +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_record.xml @@ -34,6 +34,8 @@ info:eu-repo/semantics/altIdentifier/eissn/2367-8194 info:eu-repo/grantAgreement/EC/FP7/226852 One Ecosystem 2: e13718 + One Ecosystem 2: e13718 + One Ecosystem 2: e13718 Ecosystem Services hotspots Natura 2000 Quiet Protected Areas From 47c7122773cde57c6e81bd3fbdc679fa9e768a1a Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 17 Jul 2020 12:56:35 +0200 Subject: [PATCH 067/100] changed priority from beta to production --- .../eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJob.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJob.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJob.java index 8baa469bf..67068e072 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJob.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJob.java @@ -96,11 +96,11 @@ public class MergeGraphSparkJob { .map((MapFunction, Tuple2>, P>) value -> { Optional

p = Optional.ofNullable(value._1()).map(Tuple2::_2); Optional b = Optional.ofNullable(value._2()).map(Tuple2::_2); - if (p.isPresent() & !b.isPresent()) { - return p.get(); + if (b.isPresent() & !p.isPresent()) { + return (P)b.get(); } - if (b.isPresent()) { - return (P) b.get(); + if (p.isPresent()) { + return p.get(); } return null; }, Encoders.bean(p_clazz)) From 346a1d2b5a4092f03e7dda9578e7fc79ca75544c Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Sat, 18 Jul 2020 09:40:36 +0200 Subject: [PATCH 068/100] update eventId generator --- .../dhp/broker/model/EventFactory.java | 15 +- .../broker/oa/partial/oozie_app/workflow.xml | 147 +----------------- 2 files changed, 11 insertions(+), 151 deletions(-) diff --git a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/model/EventFactory.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/model/EventFactory.java index 4a58cfd36..0cb0d7801 100644 --- a/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/model/EventFactory.java +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/model/EventFactory.java @@ -34,7 +34,10 @@ public class EventFactory { final MappedFields map = createMapFromResult(updateInfo); final String eventId = calculateEventId( - updateInfo.getTopicPath(), updateInfo.getTarget().getOpenaireId(), updateInfo.getHighlightValueAsString()); + updateInfo.getTopicPath(), updateInfo.getTargetDs().getOpenaireId(), updateInfo + .getTarget() + .getOpenaireId(), + updateInfo.getHighlightValueAsString()); res.setEventId(eventId); res.setProducerId(PRODUCER_ID); @@ -93,11 +96,13 @@ public class EventFactory { return map; } - private static String calculateEventId(final String topic, final String publicationId, final String value) { + private static String calculateEventId(final String topic, final String dsId, final String publicationId, + final String value) { return "event-" - + DigestUtils.md5Hex(topic).substring(0, 6) + "-" - + DigestUtils.md5Hex(publicationId).substring(0, 8) + "-" - + DigestUtils.md5Hex(value).substring(0, 8); + + DigestUtils.md5Hex(topic).substring(0, 4) + "-" + + DigestUtils.md5Hex(dsId).substring(0, 4) + "-" + + DigestUtils.md5Hex(publicationId).substring(0, 7) + "-" + + DigestUtils.md5Hex(value).substring(0, 5); } private static long calculateExpiryDate(final long now) { diff --git a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml index 2271a9e0e..80505ca1e 100644 --- a/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/partial/oozie_app/workflow.xml @@ -64,157 +64,12 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - - yarn - cluster - JoinStep0 - eu.dnetlib.dhp.broker.oa.JoinStep0Job - dhp-broker-events-${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.sql.shuffle.partitions=3840 - - --graphPath${graphInputPath} - --workingPath${workingPath} - - - - - - - - yarn - cluster - JoinStep1 - eu.dnetlib.dhp.broker.oa.JoinStep1Job - dhp-broker-events-${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.sql.shuffle.partitions=3840 - - --graphPath${graphInputPath} - --workingPath${workingPath} - - - - - - - - yarn - cluster - JoinStep2 - eu.dnetlib.dhp.broker.oa.JoinStep2Job - dhp-broker-events-${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.sql.shuffle.partitions=3840 - - --graphPath${graphInputPath} - --workingPath${workingPath} - - - - - - - - yarn - cluster - JoinStep3 - eu.dnetlib.dhp.broker.oa.JoinStep3Job - dhp-broker-events-${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.sql.shuffle.partitions=3840 - - --graphPath${graphInputPath} - --workingPath${workingPath} - - - - - - - - yarn - cluster - JoinStep4 - eu.dnetlib.dhp.broker.oa.JoinStep4Job - dhp-broker-events-${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.sql.shuffle.partitions=3840 - - --graphPath${graphInputPath} - --workingPath${workingPath} - - - - - - - - yarn - cluster - PrepareGroupsJob - eu.dnetlib.dhp.broker.oa.PrepareGroupsJob - dhp-broker-events-${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.sql.shuffle.partitions=3840 - - --graphPath${graphInputPath} - --workingPath${workingPath} - - - - - yarn From 331a3cbdd04aa5c61088bc64ff78e387e1710f37 Mon Sep 17 00:00:00 2001 From: "michele.artini" Date: Mon, 20 Jul 2020 09:50:29 +0200 Subject: [PATCH 069/100] fixed originalId --- .../graph/raw/AbstractMdRecordToOafMapper.java | 16 +++++++++++++++- .../eu/dnetlib/dhp/oa/graph/raw/MappersTest.java | 6 ++++++ 2 files changed, 21 insertions(+), 1 deletion(-) 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 c43ee29fe..3e042834f 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 @@ -266,7 +266,9 @@ public abstract class AbstractMdRecordToOafMapper { r.setDataInfo(info); r.setLastupdatetimestamp(lastUpdateTimestamp); r.setId(createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false)); - r.setOriginalId(Arrays.asList(doc.valueOf("//dri:objIdentifier"))); + + r.setOriginalId(Arrays.asList(findOriginalId(doc))); + r.setCollectedfrom(Arrays.asList(collectedFrom)); r.setPid(prepareResultPids(doc, info)); r.setDateofcollection(doc.valueOf("//dr:dateOfCollection")); @@ -429,6 +431,18 @@ public abstract class AbstractMdRecordToOafMapper { return null; } + private String findOriginalId(final Document doc) { + final Node n = doc.selectSingleNode("//*[local-name()='provenance']/*[local-name()='originDescription']"); + if (n != null) { + final String id = n.valueOf("./*[local-name()='identifier']"); + if (StringUtils.isNotBlank(id)) { + return id; + } + } + return doc.valueOf("//*[local-name()='header']/*[local-name()='identifier']"); + + } + protected Qualifier prepareQualifier(final Node node, final String xpath, final String schemeId) { return prepareQualifier(node.valueOf(xpath).trim(), schemeId); } 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 b1f0ecf0d..542fd00f3 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 @@ -68,6 +68,10 @@ public class MappersTest { final Relation r2 = (Relation) list.get(2); assertValidId(p.getId()); + + assertTrue(p.getOriginalId().size() == 1); + assertEquals("10.3897/oneeco.2.e13718", p.getOriginalId().get(0)); + assertValidId(p.getCollectedfrom().get(0).getKey()); assertTrue(StringUtils.isNotBlank(p.getTitle().get(0).getValue())); assertFalse(p.getDataInfo().getInvisible()); @@ -169,6 +173,8 @@ public class MappersTest { final Relation r2 = (Relation) list.get(2); assertValidId(d.getId()); + assertTrue(d.getOriginalId().size() == 1); + assertEquals("oai:zenodo.org:3234526", d.getOriginalId().get(0)); assertValidId(d.getCollectedfrom().get(0).getKey()); assertTrue(StringUtils.isNotBlank(d.getTitle().get(0).getValue())); assertTrue(d.getAuthor().size() > 0); From de72b1c859b777458470501aae045f6db419dcd5 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 20 Jul 2020 09:59:11 +0200 Subject: [PATCH 070/100] cleanup --- .../java/eu/dnetlib/dhp/oa/dedup/SparkPropagateRelation.java | 2 -- 1 file changed, 2 deletions(-) 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 03e6674e4..ae5bf9252 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 @@ -12,8 +12,6 @@ import org.apache.spark.sql.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.base.Joiner; - import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.DataInfo; From 105176105c14095d782b951b27a8c84f277cd1ae Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 20 Jul 2020 09:59:47 +0200 Subject: [PATCH 071/100] updated dnet-pace-core dependency to version 4.0.4 to include the latest clustering function --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 5ece9aaf6..cec3dd75a 100644 --- a/pom.xml +++ b/pom.xml @@ -315,7 +315,7 @@ eu.dnetlib dnet-pace-core - 4.0.2 + 4.0.4 eu.dnetlib From e0c4cf6f7bff8894eb12e1d72e99f57d8aafc17a Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 20 Jul 2020 10:48:01 +0200 Subject: [PATCH 072/100] added parameter to drive the graph merge strategy: priority (BETA|PROD) --- .../oa/graph/merge/MergeGraphSparkJob.java | 42 +++++++++++++++---- .../dhp/oa/graph/merge/oozie_app/workflow.xml | 13 +++++- .../dhp/oa/graph/merge_graphs_parameters.json | 6 +++ 3 files changed, 53 insertions(+), 8 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJob.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJob.java index 67068e072..b723de955 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJob.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJob.java @@ -37,6 +37,8 @@ public class MergeGraphSparkJob { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final String PRIORITY_DEFAULT = "BETA"; // BETA | PROD + public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils @@ -47,6 +49,11 @@ public class MergeGraphSparkJob { final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); parser.parseArgument(args); + String priority = Optional + .ofNullable(parser.get("priority")) + .orElse(PRIORITY_DEFAULT); + log.info("priority: {}", priority); + Boolean isSparkSessionManaged = Optional .ofNullable(parser.get("isSparkSessionManaged")) .map(Boolean::valueOf) @@ -76,12 +83,13 @@ public class MergeGraphSparkJob { isSparkSessionManaged, spark -> { removeOutputDir(spark, outputPath); - mergeGraphTable(spark, betaInputPath, prodInputPath, entityClazz, entityClazz, outputPath); + mergeGraphTable(spark, priority, betaInputPath, prodInputPath, entityClazz, entityClazz, outputPath); }); } private static

void mergeGraphTable( SparkSession spark, + String priority, String betaInputPath, String prodInputPath, Class

p_clazz, @@ -96,13 +104,13 @@ public class MergeGraphSparkJob { .map((MapFunction, Tuple2>, P>) value -> { Optional

p = Optional.ofNullable(value._1()).map(Tuple2::_2); Optional b = Optional.ofNullable(value._2()).map(Tuple2::_2); - if (b.isPresent() & !p.isPresent()) { - return (P)b.get(); + switch (priority) { + default: + case "BETA": + return mergeWithPriorityToBETA(p, b); + case "PROD": + return mergeWithPriorityToPROD(p, b); } - if (p.isPresent()) { - return p.get(); - } - return null; }, Encoders.bean(p_clazz)) .filter((FilterFunction

) Objects::nonNull) .write() @@ -111,6 +119,26 @@ public class MergeGraphSparkJob { .json(outputPath); } + private static

P mergeWithPriorityToPROD(Optional

p, Optional b) { + if (b.isPresent() & !p.isPresent()) { + return (P) b.get(); + } + if (p.isPresent()) { + return p.get(); + } + return null; + } + + private static

P mergeWithPriorityToBETA(Optional

p, Optional b) { + if (p.isPresent() & !b.isPresent()) { + return p.get(); + } + if (b.isPresent()) { + return (P) b.get(); + } + return null; + } + private static Dataset> readTableFromPath( SparkSession spark, String inputEntityPath, Class clazz) { diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge/oozie_app/workflow.xml index 0a512fb6a..07a125fb6 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge/oozie_app/workflow.xml @@ -13,6 +13,10 @@ graphOutputPath the output merged graph root path + + priority + decides from which infrastructure the content must win in case of ID clash + sparkDriverMemory @@ -88,6 +92,7 @@ --prodInputPath${prodInputGgraphPath}/publication --outputPath${graphOutputPath}/publication --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Publication + --priority${priority} @@ -114,6 +119,7 @@ --prodInputPath${prodInputGgraphPath}/dataset --outputPath${graphOutputPath}/dataset --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Dataset + --priority${priority} @@ -140,6 +146,7 @@ --prodInputPath${prodInputGgraphPath}/otherresearchproduct --outputPath${graphOutputPath}/otherresearchproduct --graphTableClassNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct + --priority${priority} @@ -166,6 +173,7 @@ --prodInputPath${prodInputGgraphPath}/software --outputPath${graphOutputPath}/software --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Software + --priority${priority} @@ -192,6 +200,7 @@ --prodInputPath${prodInputGgraphPath}/datasource --outputPath${graphOutputPath}/datasource --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Datasource + --priority${priority} @@ -218,6 +227,7 @@ --prodInputPath${prodInputGgraphPath}/organization --outputPath${graphOutputPath}/organization --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Organization + --priority${priority} @@ -244,6 +254,7 @@ --prodInputPath${prodInputGgraphPath}/project --outputPath${graphOutputPath}/project --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Project + --priority${priority} @@ -266,11 +277,11 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.shuffle.partitions=7680 - --betaInputPath${betaInputGgraphPath}/relation --prodInputPath${prodInputGgraphPath}/relation --outputPath${graphOutputPath}/relation --graphTableClassNameeu.dnetlib.dhp.schema.oaf.Relation + --priority${priority} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge_graphs_parameters.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge_graphs_parameters.json index 6018b7e93..1a612807b 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge_graphs_parameters.json +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge_graphs_parameters.json @@ -28,5 +28,11 @@ "paramLongName": "graphTableClassName", "paramDescription": "class name moelling the graph table", "paramRequired": true + }, + { + "paramName": "pr", + "paramLongName": "priority", + "paramDescription": "decides from which infrastructure the content must win in case of ID clash", + "paramRequired": false } ] \ No newline at end of file From 124e7ce19c57005127c9f5b4bbe63b7b5eee1a90 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 20 Jul 2020 17:33:37 +0200 Subject: [PATCH 073/100] in case of missing attribute //dr:CobjCategory/@type the resulttype is derived by looking up the vocabulary dnet:result_typologies with the 1st instance type available --- .../raw/AbstractMdRecordToOafMapper.java | 41 +++++++++++++++---- .../dnetlib/dhp/oa/graph/raw/MappersTest.java | 30 ++++++++++---- .../dnetlib/dhp/oa/graph/raw/oaf_record.xml | 3 +- .../dnetlib/dhp/oa/graph/raw/odf_dataset.xml | 3 +- .../dnetlib/dhp/oa/graph/raw/odf_software.xml | 3 +- 5 files changed, 60 insertions(+), 20 deletions(-) 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 3e042834f..150cf2cea 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 @@ -32,7 +32,11 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyTerm; +import eu.dnetlib.dhp.schema.common.ModelConstants; import org.apache.commons.lang3.StringUtils; +import org.apache.spark.sql.sources.In; +import org.apache.zookeeper.Op; import org.dom4j.Document; import org.dom4j.DocumentFactory; import org.dom4j.DocumentHelper; @@ -99,7 +103,6 @@ public abstract class AbstractMdRecordToOafMapper { final Document doc = DocumentHelper .parseText(xml.replaceAll(DATACITE_SCHEMA_KERNEL_4, DATACITE_SCHEMA_KERNEL_3)); - final String type = doc.valueOf("//dr:CobjCategory/@type"); final KeyValue collectedFrom = getProvenanceDatasource( doc, "//oaf:collectedFrom/@id", "//oaf:collectedFrom/@name"); @@ -118,12 +121,32 @@ public abstract class AbstractMdRecordToOafMapper { final DataInfo info = prepareDataInfo(doc, invisible); final long lastUpdateTimestamp = new Date().getTime(); - return createOafs(doc, type, collectedFrom, hostedBy, info, lastUpdateTimestamp); + final List instances = prepareInstances(doc, info, collectedFrom, hostedBy); + + final String type = getResultType(doc, instances); + + return createOafs(doc, type, instances, collectedFrom, info, lastUpdateTimestamp); } catch (final Exception e) { throw new RuntimeException(e); } } + protected String getResultType(final Document doc, final List instances) { + String type = doc.valueOf("//dr:CobjCategory/@type"); + + if (StringUtils.isBlank(type) & vocs.vocabularyExists(ModelConstants.DNET_RESULT_TYPOLOGIES)) { + String instanceType = instances + .stream() + .map(i -> i.getInstancetype().getClassid()) + .findFirst() + .orElse("0000"); // Unknown + Qualifier resultType = vocs.getSynonymAsQualifier(ModelConstants.DNET_RESULT_TYPOLOGIES, instanceType); + return resultType.getClassid(); + } + + return type; + } + private KeyValue getProvenanceDatasource(final Document doc, final String xpathId, final String xpathName) { final String dsId = doc.valueOf(xpathId); final String dsName = doc.valueOf(xpathName); @@ -138,8 +161,8 @@ public abstract class AbstractMdRecordToOafMapper { protected List createOafs( final Document doc, final String type, + final List instances, final KeyValue collectedFrom, - final KeyValue hostedBy, final DataInfo info, final long lastUpdateTimestamp) { @@ -148,14 +171,14 @@ public abstract class AbstractMdRecordToOafMapper { switch (type.toLowerCase()) { case "publication": final Publication p = new Publication(); - populateResultFields(p, doc, collectedFrom, hostedBy, info, lastUpdateTimestamp); + populateResultFields(p, doc, instances, collectedFrom, info, lastUpdateTimestamp); p.setResulttype(PUBLICATION_DEFAULT_RESULTTYPE); p.setJournal(prepareJournal(doc, info)); oafs.add(p); break; case "dataset": final Dataset d = new Dataset(); - populateResultFields(d, doc, collectedFrom, hostedBy, info, lastUpdateTimestamp); + populateResultFields(d, doc, instances, collectedFrom, info, lastUpdateTimestamp); d.setResulttype(DATASET_DEFAULT_RESULTTYPE); d.setStoragedate(prepareDatasetStorageDate(doc, info)); d.setDevice(prepareDatasetDevice(doc, info)); @@ -168,7 +191,7 @@ public abstract class AbstractMdRecordToOafMapper { break; case "software": final Software s = new Software(); - populateResultFields(s, doc, collectedFrom, hostedBy, info, lastUpdateTimestamp); + populateResultFields(s, doc, instances, collectedFrom, info, lastUpdateTimestamp); s.setResulttype(SOFTWARE_DEFAULT_RESULTTYPE); s.setDocumentationUrl(prepareSoftwareDocumentationUrls(doc, info)); s.setLicense(prepareSoftwareLicenses(doc, info)); @@ -180,7 +203,7 @@ public abstract class AbstractMdRecordToOafMapper { case "otherresearchproducts": default: final OtherResearchProduct o = new OtherResearchProduct(); - populateResultFields(o, doc, collectedFrom, hostedBy, info, lastUpdateTimestamp); + populateResultFields(o, doc, instances, collectedFrom, info, lastUpdateTimestamp); o.setResulttype(ORP_DEFAULT_RESULTTYPE); o.setContactperson(prepareOtherResearchProductContactPersons(doc, info)); o.setContactgroup(prepareOtherResearchProductContactGroups(doc, info)); @@ -259,8 +282,8 @@ public abstract class AbstractMdRecordToOafMapper { private void populateResultFields( final Result r, final Document doc, + final List instances, final KeyValue collectedFrom, - final KeyValue hostedBy, final DataInfo info, final long lastUpdateTimestamp) { r.setDataInfo(info); @@ -293,7 +316,7 @@ public abstract class AbstractMdRecordToOafMapper { r.setCoverage(prepareCoverages(doc, info)); r.setContext(prepareContexts(doc, info)); r.setExternalReference(new ArrayList<>()); // NOT PRESENT IN MDSTORES - final List instances = prepareInstances(doc, info, collectedFrom, hostedBy); + r.setInstance(instances); r.setBestaccessright(getBestAccessRights(instances)); } 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 204458562..b37590d50 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 @@ -6,12 +6,15 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.when; import java.io.IOException; import java.util.List; import java.util.Optional; +import eu.dnetlib.dhp.oa.graph.clean.CleaningFunctionTest; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.junit.jupiter.api.BeforeEach; @@ -35,20 +38,20 @@ import eu.dnetlib.dhp.schema.oaf.StructuredProperty; @ExtendWith(MockitoExtension.class) public class MappersTest { + @Mock + private ISLookUpService isLookUpService; + @Mock private VocabularyGroup vocs; @BeforeEach public void setUp() throws Exception { - when(vocs.getTermAsQualifier(anyString(), anyString())) - .thenAnswer( - invocation -> OafMapperUtils - .qualifier( - invocation.getArgument(1), invocation.getArgument(1), invocation.getArgument(0), - invocation.getArgument(0))); - - when(vocs.termExists(anyString(), anyString())).thenReturn(true); + lenient().when(isLookUpService.quickSearchProfile(VocabularyGroup.VOCABULARIES_XQUERY)).thenReturn(vocs()); + lenient() + .when(isLookUpService.quickSearchProfile(VocabularyGroup.VOCABULARY_SYNONYMS_XQUERY)) + .thenReturn(synonyms()); + vocs = VocabularyGroup.loadVocsFromIS(isLookUpService); } @Test @@ -269,4 +272,15 @@ public class MappersTest { assertEquals(':', id.charAt(15)); assertEquals(':', id.charAt(16)); } + + private List vocs() throws IOException { + return IOUtils + .readLines(CleaningFunctionTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/terms.txt")); + } + + private List synonyms() throws IOException { + return IOUtils + .readLines(CleaningFunctionTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/synonyms.txt")); + } + } diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_record.xml b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_record.xml index be78bf35c..3b2658bcf 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_record.xml +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_record.xml @@ -49,7 +49,8 @@ regulating services supporting services Research Article - 0001 + + 0001 2017-01-01 corda_______::226852 OPEN diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_dataset.xml b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_dataset.xml index 5525a2753..6648d1804 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_dataset.xml +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_dataset.xml @@ -82,7 +82,8 @@

All files are in MATLAB .mat format.

- 0021 + + 0021 2019-01-01 OPEN und diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_software.xml b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_software.xml index fd3fdd473..6a9170ce1 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_software.xml +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_software.xml @@ -52,7 +52,8 @@ subjectScheme="EDAM Ontology" valueURI="http://edamontology.org/topic_3534">Protein binding sites - 0029 + + 0029 2018-06-06 From 54ac5839237425723231a566220b8c2347f8595b Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 20 Jul 2020 17:37:08 +0200 Subject: [PATCH 074/100] code formatting --- .../oa/graph/raw/AbstractMdRecordToOafMapper.java | 12 ++++++------ .../eu/dnetlib/dhp/oa/graph/raw/MappersTest.java | 12 ++++++------ 2 files changed, 12 insertions(+), 12 deletions(-) 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 150cf2cea..e9450c5ca 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 @@ -32,8 +32,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyTerm; -import eu.dnetlib.dhp.schema.common.ModelConstants; import org.apache.commons.lang3.StringUtils; import org.apache.spark.sql.sources.In; import org.apache.zookeeper.Op; @@ -43,7 +41,9 @@ import org.dom4j.DocumentHelper; import org.dom4j.Node; import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyGroup; +import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyTerm; import eu.dnetlib.dhp.schema.common.LicenseComparator; +import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.Author; import eu.dnetlib.dhp.schema.oaf.Context; import eu.dnetlib.dhp.schema.oaf.DataInfo; @@ -136,10 +136,10 @@ public abstract class AbstractMdRecordToOafMapper { if (StringUtils.isBlank(type) & vocs.vocabularyExists(ModelConstants.DNET_RESULT_TYPOLOGIES)) { String instanceType = instances - .stream() - .map(i -> i.getInstancetype().getClassid()) - .findFirst() - .orElse("0000"); // Unknown + .stream() + .map(i -> i.getInstancetype().getClassid()) + .findFirst() + .orElse("0000"); // Unknown Qualifier resultType = vocs.getSynonymAsQualifier(ModelConstants.DNET_RESULT_TYPOLOGIES, instanceType); return resultType.getClassid(); } 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 b37590d50..b157f7ec6 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 @@ -13,8 +13,6 @@ import java.io.IOException; import java.util.List; import java.util.Optional; -import eu.dnetlib.dhp.oa.graph.clean.CleaningFunctionTest; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.junit.jupiter.api.BeforeEach; @@ -23,6 +21,7 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import eu.dnetlib.dhp.oa.graph.clean.CleaningFunctionTest; import eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils; import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyGroup; import eu.dnetlib.dhp.schema.common.ModelConstants; @@ -34,6 +33,7 @@ import eu.dnetlib.dhp.schema.oaf.Publication; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.schema.oaf.Software; import eu.dnetlib.dhp.schema.oaf.StructuredProperty; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; @ExtendWith(MockitoExtension.class) public class MappersTest { @@ -48,8 +48,8 @@ public class MappersTest { public void setUp() throws Exception { lenient().when(isLookUpService.quickSearchProfile(VocabularyGroup.VOCABULARIES_XQUERY)).thenReturn(vocs()); lenient() - .when(isLookUpService.quickSearchProfile(VocabularyGroup.VOCABULARY_SYNONYMS_XQUERY)) - .thenReturn(synonyms()); + .when(isLookUpService.quickSearchProfile(VocabularyGroup.VOCABULARY_SYNONYMS_XQUERY)) + .thenReturn(synonyms()); vocs = VocabularyGroup.loadVocsFromIS(isLookUpService); } @@ -275,12 +275,12 @@ public class MappersTest { private List vocs() throws IOException { return IOUtils - .readLines(CleaningFunctionTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/terms.txt")); + .readLines(CleaningFunctionTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/terms.txt")); } private List synonyms() throws IOException { return IOUtils - .readLines(CleaningFunctionTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/synonyms.txt")); + .readLines(CleaningFunctionTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/synonyms.txt")); } } From 32f5e466e3995468833ff4c9e028c660cc7b6bae Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 20 Jul 2020 17:42:58 +0200 Subject: [PATCH 075/100] imports cleanup --- .../dhp/schema/common/ModelConstants.java | 2 - .../raw/AbstractMdRecordToOafMapper.java | 54 ++----------------- 2 files changed, 4 insertions(+), 52 deletions(-) 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 c5905e45b..bf48605d2 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,8 +1,6 @@ package eu.dnetlib.dhp.schema.common; -import java.security.Key; - import eu.dnetlib.dhp.schema.oaf.DataInfo; import eu.dnetlib.dhp.schema.oaf.KeyValue; import eu.dnetlib.dhp.schema.oaf.Qualifier; 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 e9450c5ca..94da4c152 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,67 +1,21 @@ 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.dataInfo; -import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.field; -import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.journal; -import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.keyValue; -import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.listFields; -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 static eu.dnetlib.dhp.schema.common.ModelConstants.DATASET_DEFAULT_RESULTTYPE; -import static eu.dnetlib.dhp.schema.common.ModelConstants.DNET_ACCESS_MODES; -import static eu.dnetlib.dhp.schema.common.ModelConstants.DNET_PID_TYPES; -import static eu.dnetlib.dhp.schema.common.ModelConstants.IS_PRODUCED_BY; -import static eu.dnetlib.dhp.schema.common.ModelConstants.NOT_AVAILABLE; -import static eu.dnetlib.dhp.schema.common.ModelConstants.ORP_DEFAULT_RESULTTYPE; -import static eu.dnetlib.dhp.schema.common.ModelConstants.OUTCOME; -import static eu.dnetlib.dhp.schema.common.ModelConstants.PRODUCES; -import static eu.dnetlib.dhp.schema.common.ModelConstants.PUBLICATION_DEFAULT_RESULTTYPE; -import static eu.dnetlib.dhp.schema.common.ModelConstants.REPOSITORY_PROVENANCE_ACTIONS; -import static eu.dnetlib.dhp.schema.common.ModelConstants.RESULT_PROJECT; -import static eu.dnetlib.dhp.schema.common.ModelConstants.SOFTWARE_DEFAULT_RESULTTYPE; -import static eu.dnetlib.dhp.schema.common.ModelConstants.UNKNOWN; +import static eu.dnetlib.dhp.oa.graph.raw.common.OafMapperUtils.*; +import static eu.dnetlib.dhp.schema.common.ModelConstants.*; -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 java.util.Optional; +import java.util.*; import org.apache.commons.lang3.StringUtils; -import org.apache.spark.sql.sources.In; -import org.apache.zookeeper.Op; import org.dom4j.Document; import org.dom4j.DocumentFactory; import org.dom4j.DocumentHelper; import org.dom4j.Node; import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyGroup; -import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyTerm; import eu.dnetlib.dhp.schema.common.LicenseComparator; import eu.dnetlib.dhp.schema.common.ModelConstants; -import eu.dnetlib.dhp.schema.oaf.Author; -import eu.dnetlib.dhp.schema.oaf.Context; -import eu.dnetlib.dhp.schema.oaf.DataInfo; -import eu.dnetlib.dhp.schema.oaf.Dataset; -import eu.dnetlib.dhp.schema.oaf.Field; -import eu.dnetlib.dhp.schema.oaf.GeoLocation; -import eu.dnetlib.dhp.schema.oaf.Instance; -import eu.dnetlib.dhp.schema.oaf.Journal; -import eu.dnetlib.dhp.schema.oaf.KeyValue; -import eu.dnetlib.dhp.schema.oaf.OAIProvenance; -import eu.dnetlib.dhp.schema.oaf.Oaf; -import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct; -import eu.dnetlib.dhp.schema.oaf.Publication; -import eu.dnetlib.dhp.schema.oaf.Qualifier; -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 eu.dnetlib.dhp.schema.oaf.*; public abstract class AbstractMdRecordToOafMapper { From ebf60020acb917a78adffd07e3760c3d7598b8a8 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 20 Jul 2020 19:01:10 +0200 Subject: [PATCH 076/100] map results as OPRs in case of missing //CobjCategory/@type and the vocabulary dnet:result_typologies doesn't resolve the super type --- .../dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) 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 94da4c152..5159fa9bb 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 @@ -93,9 +93,16 @@ public abstract class AbstractMdRecordToOafMapper { .stream() .map(i -> i.getInstancetype().getClassid()) .findFirst() + .map(s -> UNKNOWN.equalsIgnoreCase(s) ? "0000" : s) .orElse("0000"); // Unknown - Qualifier resultType = vocs.getSynonymAsQualifier(ModelConstants.DNET_RESULT_TYPOLOGIES, instanceType); - return resultType.getClassid(); + return Optional + .ofNullable(vocs.getSynonymAsQualifier(ModelConstants.DNET_RESULT_TYPOLOGIES, instanceType)) + .map(q -> q.getClassid()) + .orElse("0000"); + /* + * .orElseThrow( () -> new IllegalArgumentException( String.format("'%s' not mapped in %s", instanceType, + * DNET_RESULT_TYPOLOGIES))); + */ } return type; From 9ab594ccf63b2f355db0f0e9b827a31aae9914b4 Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Tue, 21 Jul 2020 10:36:21 +0200 Subject: [PATCH 077/100] fixed test --- .../src/test/java/eu/dnetlib/dhp/sx/ebi/TestEBI.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/ebi/TestEBI.scala b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/ebi/TestEBI.scala index fa390a21b..92e14895d 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/ebi/TestEBI.scala +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/ebi/TestEBI.scala @@ -6,7 +6,7 @@ class TestEBI { - @Test +// @Test def testEBIData() = { SparkAddLinkUpdates.main("-mt local[*] -w /home/sandro/Downloads".split(" ")) From 56bbfdc65da8244a2c4beecb5f5fe93c4d5f3cd1 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 23 Jul 2020 08:54:10 +0200 Subject: [PATCH 078/100] introduced parameter 'numParitions', driving the hive DB table data partitioning. Currently specified only for table 'project' --- .../graph/hive/GraphHiveTableImporterJob.java | 22 ++++++++++++++----- .../dhp/oa/graph/hive/oozie_app/workflow.xml | 1 + .../graph/hive_table_importer_parameters.json | 6 +++++ 3 files changed, 24 insertions(+), 5 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveTableImporterJob.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveTableImporterJob.java index f88f7457f..76e1d57a1 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveTableImporterJob.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/hive/GraphHiveTableImporterJob.java @@ -9,6 +9,7 @@ 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; @@ -42,6 +43,12 @@ public class GraphHiveTableImporterJob { .orElse(Boolean.TRUE); log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + int numPartitions = Optional + .ofNullable(parser.get("numPartitions")) + .map(Integer::valueOf) + .orElse(-1); + log.info("numPartitions: {}", numPartitions); + String inputPath = parser.get("inputPath"); log.info("inputPath: {}", inputPath); @@ -60,16 +67,21 @@ public class GraphHiveTableImporterJob { conf.set("hive.metastore.uris", hiveMetastoreUris); runWithSparkHiveSession( - conf, isSparkSessionManaged, spark -> loadGraphTable(spark, inputPath, hiveDbName, clazz)); + conf, isSparkSessionManaged, spark -> loadGraphTable(spark, inputPath, hiveDbName, clazz, numPartitions)); } // protected for testing private static void loadGraphTable(SparkSession spark, String inputPath, String hiveDbName, - Class clazz) { + Class clazz, int numPartitions) { - spark - .read() - .textFile(inputPath) + Dataset dataset = spark.read().textFile(inputPath); + + if (numPartitions > 0) { + log.info("repartitioning {} to {} partitions", clazz.getSimpleName(), numPartitions); + dataset = dataset.repartition(numPartitions); + } + + dataset .map((MapFunction) s -> OBJECT_MAPPER.readValue(s, clazz), Encoders.bean(clazz)) .write() .mode(SaveMode.Overwrite) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/workflow.xml index 8566d7667..09930336a 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/workflow.xml @@ -282,6 +282,7 @@ --hiveDbName${hiveDbName} --classNameeu.dnetlib.dhp.schema.oaf.Project --hiveMetastoreUris${hiveMetastoreUris} + --numPartitions100
diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive_table_importer_parameters.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive_table_importer_parameters.json index 5b5b0743c..f38a0412c 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive_table_importer_parameters.json +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive_table_importer_parameters.json @@ -5,6 +5,12 @@ "paramDescription": "when true will stop SparkSession after job execution", "paramRequired": false }, + { + "paramName": "np", + "paramLongName": "numPartitions", + "paramDescription": "number of dataset partitions", + "paramRequired": false + }, { "paramName": "in", "paramLongName": "inputPath", From 477fa6234d5e9198b09fbe87c72f0a76de56a7b9 Mon Sep 17 00:00:00 2001 From: Giorgos Alexiou Date: Thu, 11 Jun 2020 21:01:14 +0300 Subject: [PATCH 079/100] Script re-organisation and adding table invalidations needed for impala --- .../graph/stats/oozie_app/config-default.xml | 24 +- .../oa/graph/stats/oozie_app/impala-shell.sh | 16 + .../oa/graph/stats/oozie_app/impala-stats.sql | 77 ++ .../graph/stats/oozie_app/scripts/step1.sql | 14 +- .../graph/stats/oozie_app/scripts/step10.sql | 10 + .../stats/oozie_app/scripts/step10_1.sql | 7 - .../stats/oozie_app/scripts/step10_2.sql | 1 - .../stats/oozie_app/scripts/step10_3.sql | 1 - .../graph/stats/oozie_app/scripts/step11.sql | 10 +- .../graph/stats/oozie_app/scripts/step12.sql | 44 + .../stats/oozie_app/scripts/step12_1.sql | 6 - .../stats/oozie_app/scripts/step12_2.sql | 2 - .../stats/oozie_app/scripts/step12_3.sql | 20 - .../stats/oozie_app/scripts/step12_4.sql | 2 - .../stats/oozie_app/scripts/step12_5.sql | 2 - .../stats/oozie_app/scripts/step12_6.sql | 2 - .../stats/oozie_app/scripts/step12_7.sql | 2 - .../stats/oozie_app/scripts/step12_8.sql | 1 - .../graph/stats/oozie_app/scripts/step13.sql | 45 +- .../graph/stats/oozie_app/scripts/step14.sql | 7 - .../graph/stats/oozie_app/scripts/step2.sql | 32 + .../graph/stats/oozie_app/scripts/step2_1.sql | 10 - .../graph/stats/oozie_app/scripts/step2_2.sql | 19 - .../graph/stats/oozie_app/scripts/step2_3.sql | 1 - .../graph/stats/oozie_app/scripts/step2_4.sql | 1 - .../graph/stats/oozie_app/scripts/step2_5.sql | 1 - .../graph/stats/oozie_app/scripts/step2_6.sql | 1 - .../graph/stats/oozie_app/scripts/step2_7.sql | 1 - .../graph/stats/oozie_app/scripts/step2_8.sql | 1 - .../graph/stats/oozie_app/scripts/step2_9.sql | 1 - .../graph/stats/oozie_app/scripts/step3.sql | 4 +- .../graph/stats/oozie_app/scripts/step4.sql | 26 + .../graph/stats/oozie_app/scripts/step4_1.sql | 9 - .../stats/oozie_app/scripts/step4_10.sql | 1 - .../graph/stats/oozie_app/scripts/step4_2.sql | 7 - .../graph/stats/oozie_app/scripts/step4_3.sql | 2 - .../graph/stats/oozie_app/scripts/step4_4.sql | 1 - .../graph/stats/oozie_app/scripts/step4_5.sql | 1 - .../graph/stats/oozie_app/scripts/step4_6.sql | 3 - .../graph/stats/oozie_app/scripts/step4_7.sql | 1 - .../graph/stats/oozie_app/scripts/step4_8.sql | 1 - .../graph/stats/oozie_app/scripts/step4_9.sql | 1 - .../graph/stats/oozie_app/scripts/step5.sql | 32 + .../graph/stats/oozie_app/scripts/step5_1.sql | 9 - .../stats/oozie_app/scripts/step5_10.sql | 1 - .../graph/stats/oozie_app/scripts/step5_2.sql | 7 - .../graph/stats/oozie_app/scripts/step5_3.sql | 2 - .../graph/stats/oozie_app/scripts/step5_4.sql | 1 - .../graph/stats/oozie_app/scripts/step5_5.sql | 1 - .../graph/stats/oozie_app/scripts/step5_6.sql | 3 - .../graph/stats/oozie_app/scripts/step5_7.sql | 1 - .../graph/stats/oozie_app/scripts/step5_8.sql | 1 - .../graph/stats/oozie_app/scripts/step5_9.sql | 1 - .../graph/stats/oozie_app/scripts/step6.sql | 31 + .../graph/stats/oozie_app/scripts/step6_1.sql | 9 - .../stats/oozie_app/scripts/step6_10.sql | 1 - .../graph/stats/oozie_app/scripts/step6_2.sql | 7 - .../graph/stats/oozie_app/scripts/step6_3.sql | 2 - .../graph/stats/oozie_app/scripts/step6_4.sql | 1 - .../graph/stats/oozie_app/scripts/step6_5.sql | 1 - .../graph/stats/oozie_app/scripts/step6_6.sql | 3 - .../graph/stats/oozie_app/scripts/step6_7.sql | 1 - .../graph/stats/oozie_app/scripts/step6_8.sql | 1 - .../graph/stats/oozie_app/scripts/step6_9.sql | 1 - .../graph/stats/oozie_app/scripts/step7.sql | 21 + .../graph/stats/oozie_app/scripts/step7_1.sql | 8 - .../graph/stats/oozie_app/scripts/step7_2.sql | 3 - .../graph/stats/oozie_app/scripts/step7_3.sql | 3 - .../graph/stats/oozie_app/scripts/step7_4.sql | 5 - .../graph/stats/oozie_app/scripts/step7_5.sql | 1 - .../graph/stats/oozie_app/scripts/step8.sql | 24 + .../graph/stats/oozie_app/scripts/step8_1.sql | 8 - .../stats/oozie_app/scripts/step8_10.sql | 2 - .../stats/oozie_app/scripts/step8_11.sql | 2 - .../graph/stats/oozie_app/scripts/step8_2.sql | 2 - .../graph/stats/oozie_app/scripts/step8_3.sql | 1 - .../graph/stats/oozie_app/scripts/step8_4.sql | 1 - .../graph/stats/oozie_app/scripts/step8_5.sql | 1 - .../graph/stats/oozie_app/scripts/step8_6.sql | 1 - .../graph/stats/oozie_app/scripts/step8_7.sql | 1 - .../graph/stats/oozie_app/scripts/step8_8.sql | 1 - .../graph/stats/oozie_app/scripts/step8_9.sql | 1 - .../graph/stats/oozie_app/scripts/step9.sql | 33 + .../graph/stats/oozie_app/scripts/step9_1.sql | 10 - .../graph/stats/oozie_app/scripts/step9_2.sql | 3 - .../graph/stats/oozie_app/scripts/step9_3.sql | 7 - .../graph/stats/oozie_app/scripts/step9_4.sql | 1 - .../graph/stats/oozie_app/scripts/step9_5.sql | 1 - .../graph/stats/oozie_app/scripts/step9_7.sql | 2 - .../graph/stats/oozie_app/scripts/step9_8.sql | 2 - .../graph/stats/oozie_app/scripts/step9_9.sql | 1 - .../dhp/oa/graph/stats/oozie_app/workflow.xml | 1036 ++++------------- 92 files changed, 607 insertions(+), 1110 deletions(-) create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/impala-shell.sh create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/impala-stats.sql create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10_1.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10_2.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10_3.sql create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_1.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_2.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_3.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_4.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_5.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_6.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_7.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_8.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_1.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_2.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_3.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_4.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_5.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_6.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_7.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_8.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_9.sql create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_1.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_10.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_2.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_3.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_4.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_5.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_6.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_7.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_8.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_9.sql create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_1.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_10.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_2.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_3.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_4.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_5.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_6.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_7.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_8.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_9.sql create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_1.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_10.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_2.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_3.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_4.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_5.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_6.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_7.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_8.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_9.sql create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_1.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_2.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_3.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_4.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_5.sql create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_1.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_10.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_11.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_2.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_3.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_4.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_5.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_6.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_7.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_8.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_9.sql create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_1.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_2.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_3.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_4.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_5.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_7.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_8.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_9.sql diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/config-default.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/config-default.xml index ba7002cff..74dbb6f77 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/config-default.xml @@ -1,11 +1,12 @@ + jobTracker - ${jobTracker} + yarnRM nameNode - ${nameNode} + hdfs://nameservice1 oozie.use.system.libpath @@ -16,15 +17,22 @@ spark2 - hiveMetastoreUris + hive_db_name + + stats_wf_db_galexiou_oozie_beta + + + hive_source_db_name + + openaire2 + + + hive_metastore_uris thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 - hiveJdbcUrl + hive_jdbc_url jdbc:hive2://iis-cdh5-test-m3.ocean.icm.edu.pl:10000 - - oozie.wf.workflow.notification.url - {serviceUrl}/v1/oozieNotification/jobUpdate?jobId=$jobId%26status=$status - + diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/impala-shell.sh b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/impala-shell.sh new file mode 100644 index 000000000..a5a58493b --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/impala-shell.sh @@ -0,0 +1,16 @@ +export PYTHON_EGG_CACHE=/home/giorgos.alexiou/.python-eggs +export link_folder=/tmp/impala-shell-python-egg-cache-$(whoami) +if ! [ -L $link_folder ] +then + rm -Rf "$link_folder" + ln -sfn ${PYTHON_EGG_CACHE}${link_folder} ${link_folder} +fi + +echo "getting file from " $3 + +hdfs dfs -copyToLocal $3 + +echo "running impala shell now" +impala-shell -d $1 -f $2 +echo "impala shell finished" +rm $2 diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/impala-stats.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/impala-stats.sql new file mode 100644 index 000000000..b8833352d --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/impala-stats.sql @@ -0,0 +1,77 @@ + +INVALIDATE METADATA; +-- CREATE TABLE chart_country_year AS SELECT org.country AS country, r.year AS year, count(distinct r.id) AS publications FROM result r, result_datasources rd, datasource d, datasource_organizations dor, organization org WHERE r.id=rd.id AND rd.datasource=d.id AND rd.datasource=dor.id AND dor.organization=org.id AND r.type='publication' AND r.bestlicence='Open Access' and r.year>='1990' AND r.year<=CAST(date_part('year',now()) AS STRING) group by org.country, r.year ORDER BY org.country, r.year; +-- CREATE TABLE chart_country_datasources AS SELECT org.country AS country, d.name AS datasource, count(distinct r.id) AS publications FROM result r, result_datasources rd, datasource d, datasource_organizations dor, organization org WHERE r.id=rd.id AND rd.datasource=d.id AND d.id=dor.id AND dor.organization=org.id AND r.type='publication' AND r.bestlicence='Open Access' GROUP BY org.country, d.name ORDER BY org.country, publications DESC; +-- CREATE TABLE chart_country_type AS SELECT org.country AS country, rc.type AS type, count(distinct r.id) AS publications FROM result r, result_datasources rd, datasource d, result_classifications rc, datasource_organizations dor, organization org WHERE r.id=rd.id AND r.id=rc.id AND rd.datasource=d.id AND rd.datasource=dor.id AND dor.organization=org.id AND r.type='publication' AND r.bestlicence='Open Access' GROUP BY org.country, rc.type; +-- CREATE TABLE chart_country_fp7 AS SELECT org.country AS country, r.year AS year, count(distinct r.id) AS publications FROM result r, result_datasources rd, datasource d, datasource_organizations dor, organization org, project_results pr, project p WHERE r.id=rd.id AND rd.datasource=d.id AND rd.datasource=dor.id AND dor.organization=org.id AND r.id=pr.result AND pr.id=p.id AND p.funding_lvl0='FP7' AND r.type='publication' AND r.year>='1990' AND r.year<=CAST(date_part('year',now()) AS STRING) GROUP BY org.country, r.year ORDER BY org.country, r.year; +-- CREATE TABLE chart_datasource_type AS SELECT rd.datasource, rc.type, count(distinct rd.id) FROM result_datasources rd, result_classifications rc WHERE rd.id=rc.id GROUP BY rd.datasource, rc.type; +-- CREATE TABLE chart_datasource_year AS SELECT rd.datasource, r.year, count(distinct rd.id) FROM result r, result_datasources rd WHERE rd.id=r.id GROUP By rd.datasource, r.year; +-- CREATE TABLE chart_datasource_funders AS SELECT rd.datasource, p.funder, count(distinct rd.id) FROM result_datasources rd, project p, project_results pr WHERE p.id=pr.id AND pr.result=rd.id GROUP BY rd.datasource, p.funder; +-- CREATE TABLE chart_datasource_projects_pubs AS SELECT rd.datasource, p.title, count(distinct rd.id) FROM result_datasources rd, project p, project_results pr, result r WHERE p.id=pr.id AND pr.result=rd.id AND pr.result=r.id AND r.type='publication' GROUP BY rd.datasource, p.title; +-- CREATE TABLE chart_datasource_projects_data AS SELECT rd.datasource, p.title, count(distinct rd.id) FROM result_datasources rd, project p, project_results pr, result r WHERE p.id=pr.id AND pr.result=rd.id AND pr.result=r.id and r.type='dataset' GROUP BY rd.datasource, p.title; +-- CREATE TABLE chart_project_year AS SELECT p.id, r.year, count( distinct r.id) FROM result r, project_results pr, project p WHERE r.id=pr.result AND p.id=pr.id AND r.year>=p.start_year GROUP BY p.id, r.year; +-- CREATE TABLE chart_project_license AS SELECT pr.id, r.bestlicence, count(distinct r.id) FROM result r, project_results pr WHERE r.id=pr.result AND r.type='publication' GROUP BY pr.id, r.bestlicence; +-- CREATE TABLE chart_project_repos AS SELECT pr.id, d.name, count (distinct r.id) FROM result r, project_results pr, datasource d, datasource_results dr WHERE r.id=dr.result AND d.id=dr.id AND r.id=pr.result AND r.type='publication' GROUP BY pr.id, d.name; +-- CREATE TABLE rd_distinct AS SELECT DISTINCT * FROM result_datasources; +-- DROP VIEW result_datasources; +-- ALTER TABLE rd_distinct RENAME TO result_datasources; +-- COMPUTE STATS chart_country_datasources; +-- COMPUTE STATS chart_country_fp7; +-- COMPUTE STATS chart_country_type; +-- COMPUTE STATS chart_country_year; +-- COMPUTE STATS chart_datasource_funders; +-- COMPUTE STATS chart_datasource_projects_data; +-- COMPUTE STATS chart_datasource_projects_pubs; +-- COMPUTE STATS chart_datasource_type; +-- COMPUTE STATS chart_datasource_year; +-- COMPUTE STATS chart_project_license; +-- COMPUTE STATS chart_project_repos; +-- COMPUTE STATS chart_project_year; +COMPUTE STATS dataset; +COMPUTE STATS dataset_citations; +COMPUTE STATS dataset_classifications; +COMPUTE STATS dataset_concepts; +COMPUTE STATS dataset_datasources; +COMPUTE STATS dataset_languages; +COMPUTE STATS dataset_oids; +COMPUTE STATS dataset_pids; +COMPUTE STATS dataset_topics; +COMPUTE STATS datasource; +COMPUTE STATS datasource_languages; +COMPUTE STATS datasource_oids; +COMPUTE STATS datasource_organizations; +COMPUTE STATS numbers_country; +COMPUTE STATS organization; +COMPUTE STATS otherresearchproduct; +COMPUTE STATS otherresearchproduct_citations; +COMPUTE STATS otherresearchproduct_classifications; +COMPUTE STATS otherresearchproduct_concepts; +COMPUTE STATS otherresearchproduct_datasources; +COMPUTE STATS otherresearchproduct_languages; +COMPUTE STATS otherresearchproduct_oids; +COMPUTE STATS otherresearchproduct_pids; +COMPUTE STATS otherresearchproduct_topics; +COMPUTE STATS project; +COMPUTE STATS project_oids; +COMPUTE STATS project_organizations; +COMPUTE STATS project_results; +COMPUTE STATS publication; +COMPUTE STATS publication_citations; +COMPUTE STATS publication_classifications; +COMPUTE STATS publication_concepts; +COMPUTE STATS publication_datasources; +COMPUTE STATS publication_languages; +COMPUTE STATS publication_oids; +COMPUTE STATS publication_pids; +COMPUTE STATS publication_topics; +COMPUTE STATS result_organization; +COMPUTE STATS result_projects; +COMPUTE STATS software; +COMPUTE STATS software_citations; +COMPUTE STATS software_classifications; +COMPUTE STATS software_concepts; +COMPUTE STATS software_datasources; +COMPUTE STATS software_languages; +COMPUTE STATS software_oids; +COMPUTE STATS software_pids; +COMPUTE STATS software_topic; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql index ec8a0e4ff..1290155e6 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql @@ -1,11 +1,11 @@ -- DROP database if EXISTS ${hive_db_name} cascade; -- CREATE database ${hive_db_name}; --- --- CREATE TABLE ${hive_db_name}.Persons ( --- PersonID int, +-- +-- CREATE TABLE ${hive_db_name}.Persons ( +-- PersonID int, -- LastName varchar(255)); --- --- INSERT INTO ${hive_db_name}.Persons VALUES (1, "test_db_spyros_rec_111"); +-- +-- INSERT INTO ${hive_db_name}.Persons VALUES (1, "test_db_spyros_rec_111"); -DROP database IF EXISTS ${stats_db_name} CASCADE; -CREATE database ${stats_db_name}; +drop database if exists ${hive_db_name} cascade; +create database ${hive_db_name}; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql new file mode 100644 index 000000000..e51892771 --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql @@ -0,0 +1,10 @@ +---------------------------------------------------------------- +---------------------------------------------------------------- +-- Organization table/view and Organization related tables/views +---------------------------------------------------------------- +---------------------------------------------------------------- +DROP TABLE IF EXISTS ${hive_db_name}.organization; +CREATE TABLE ${hive_db_name}.organization AS SELECT substr(o.id, 4) as id, o.legalname.value as name, o.country.classid as country from ${hive_source_db_name}.organization o WHERE o.datainfo.deletedbyinference=false; + +CREATE OR REPLACE VIEW ${hive_db_name}.organization_datasources AS SELECT organization AS id, id AS datasource FROM ${hive_db_name}.datasource_organizations; +CREATE OR REPLACE VIEW ${hive_db_name}.organization_projects AS SELECT id AS project, organization as id FROM ${hive_db_name}.project_organizations; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10_1.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10_1.sql deleted file mode 100644 index 568148753..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10_1.sql +++ /dev/null @@ -1,7 +0,0 @@ ----------------------------------------------------------------- ----------------------------------------------------------------- --- Organization table/view and Organization related tables/views ----------------------------------------------------------------- ----------------------------------------------------------------- -DROP TABLE IF EXISTS ${stats_db_name}.organization; -CREATE TABLE ${stats_db_name}.organization AS SELECT substr(o.id, 4) as id, o.legalname.value as name, o.country.classid as country from ${openaire_db_name}.organization o WHERE o.datainfo.deletedbyinference=false; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10_2.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10_2.sql deleted file mode 100644 index 4937c8425..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10_2.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE OR REPLACE VIEW ${stats_db_name}.organization_datasources AS SELECT organization AS id, id AS datasource FROM ${stats_db_name}.datasource_organizations; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10_3.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10_3.sql deleted file mode 100644 index 768cf2a6e..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10_3.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE OR REPLACE VIEW ${stats_db_name}.organization_projects AS SELECT id AS project, organization as id FROM ${stats_db_name}.project_organizations; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql index 145be77b0..e586f0cd6 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql @@ -3,8 +3,8 @@ -- Tables/views from external tables/views (Fundref, Country, CountyGDP, roarmap, rndexpediture) ------------------------------------------------------------------------------------------------ ------------------------------------------------------------------------------------------------ -CREATE OR REPLACE VIEW ${stats_db_name}.fundref AS SELECT * FROM ${external_stats_db_name}.fundref; -CREATE OR REPLACE VIEW ${stats_db_name}.country AS SELECT * FROM ${external_stats_db_name}.country; -CREATE OR REPLACE VIEW ${stats_db_name}.countrygdp AS SELECT * FROM ${external_stats_db_name}.countrygdp; -CREATE OR REPLACE VIEW ${stats_db_name}.roarmap AS SELECT * FROM ${external_stats_db_name}.roarmap; -CREATE OR REPLACE VIEW ${stats_db_name}.rndexpediture AS SELECT * FROM ${external_stats_db_name}.rndexpediture; +CREATE OR REPLACE VIEW ${hive_db_name}.fundref AS SELECT * FROM stats_ext.fundref; +CREATE OR REPLACE VIEW ${hive_db_name}.country AS SELECT * FROM stats_ext.country; +CREATE OR REPLACE VIEW ${hive_db_name}.countrygdp AS SELECT * FROM stats_ext.countrygdp; +CREATE OR REPLACE VIEW ${hive_db_name}.roarmap AS SELECT * FROM stats_ext.roarmap; +CREATE OR REPLACE VIEW ${hive_db_name}.rndexpediture AS SELECT * FROM stats_ext.rndexpediture; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql new file mode 100644 index 000000000..ba0d52aca --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql @@ -0,0 +1,44 @@ +---------------------------- +-- Post processing - Updates +---------------------------- + +--Datasource temporary table updates +UPDATE ${hive_db_name}.datasource_tmp set harvested ='true' WHERE datasource_tmp.id IN (SELECT DISTINCT d.id FROM ${hive_db_name}.datasource_tmp d, ${hive_db_name}.result_datasources rd where d.id=rd.datasource); + +-- Project temporary table update and final project table creation with final updates that can not be applied to ORC tables +-- UPDATE ${hive_db_name}.project_tmp SET haspubs='yes' WHERE project_tmp.id IN (SELECT pr.id FROM ${hive_db_name}.project_results pr, ${hive_db_name}.result r WHERE pr.result=r.id AND r.type='publication'); + + +CREATE TABLE ${hive_db_name}.project stored as parquet as +SELECT p.id , p.acronym, p.title, p.funder, p.funding_lvl0, p.funding_lvl1, p.funding_lvl2, p.ec39, p.type, p.startdate, p.enddate, p.start_year, p.end_year, p.duration, +CASE WHEN prr1.id IS NULL THEN 'no' ELSE 'yes' END as haspubs, +CASE WHEN prr1.id IS NULL THEN 0 ELSE prr1.np END as numpubs, +CASE WHEN prr2.id IS NULL THEN 0 ELSE prr2.daysForlastPub END as daysforlastpub, +CASE WHEN prr2.id IS NULL THEN 0 ELSE prr2.dp END as delayedpubs, +p.callidentifier, p.code +FROM ${hive_db_name}.project_tmp p +LEFT JOIN (SELECT pr.id, count(distinct pr.result) AS np + FROM ${hive_db_name}.project_results pr INNER JOIN ${hive_db_name}.result r ON pr.result=r.id + WHERE r.type='publication' + GROUP BY pr.id) AS prr1 on prr1.id = p.id +LEFT JOIN (SELECT pp.id, max(datediff(to_date(r.date), to_date(pp.enddate)) ) as daysForlastPub , count(distinct r.id) as dp + FROM ${hive_db_name}.project_tmp pp, ${hive_db_name}.project_results pr, ${hive_db_name}.result r + WHERE pp.id=pr.id AND pr.result=r.id AND r.type='publication' AND datediff(to_date(r.date), to_date(pp.enddate)) > 0 + GROUP BY pp.id) AS prr2 + on prr2.id = p.id; + + +-- Publication temporary table updates +UPDATE ${hive_db_name}.publication_tmp SET delayed = 'yes' WHERE publication_tmp.id IN (SELECT distinct r.id FROM ${hive_db_name}.result r, ${hive_db_name}.project_results pr, ${hive_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); + +-- Dataset temporary table updates +UPDATE ${hive_db_name}.dataset_tmp SET delayed = 'yes' WHERE dataset_tmp.id IN (SELECT distinct r.id FROM ${hive_db_name}.result r, ${hive_db_name}.project_results pr, ${hive_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); + +-- Software temporary table updates +UPDATE ${hive_db_name}.software_tmp SET delayed = 'yes' WHERE software_tmp.id IN (SELECT distinct r.id FROM ${hive_db_name}.result r, ${hive_db_name}.project_results pr, ${hive_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); + +-- Oherresearchproduct temporary table updates +UPDATE ${hive_db_name}.otherresearchproduct_tmp SET delayed = 'yes' WHERE otherresearchproduct_tmp.id IN (SELECT distinct r.id FROM ${hive_db_name}.result r, ${hive_db_name}.project_results pr, ${hive_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); + + +CREATE OR REPLACE VIEW ${hive_db_name}.project_results_publication AS SELECT result_projects.id AS result, result_projects.project AS project_results, result.date as resultdate, project.enddate as projectenddate, result_projects.daysfromend as daysfromend FROM ${hive_db_name}.result_projects, ${hive_db_name}.result, ${hive_db_name}.project WHERE result_projects.id=result.id and result.type='publication' and project.id=result_projects.project; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_1.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_1.sql deleted file mode 100644 index c1ec8873a..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_1.sql +++ /dev/null @@ -1,6 +0,0 @@ ----------------------------- --- Post processing - Updates ----------------------------- - ---Datasource temporary table updates -UPDATE ${stats_db_name}.datasource_tmp set harvested ='true' WHERE datasource_tmp.id IN (SELECT DISTINCT d.id FROM ${stats_db_name}.datasource_tmp d, ${stats_db_name}.result_datasources rd where d.id=rd.datasource); diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_2.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_2.sql deleted file mode 100644 index b772bc374..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_2.sql +++ /dev/null @@ -1,2 +0,0 @@ --- Project temporary table update and final project table creation with final updates that can not be applied to ORC tables -UPDATE ${stats_db_name}.project_tmp SET haspubs='yes' WHERE project_tmp.id IN (SELECT pr.id FROM ${stats_db_name}.project_results pr, ${stats_db_name}.result r WHERE pr.result=r.id AND r.type='publication'); diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_3.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_3.sql deleted file mode 100644 index b5e942ed6..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_3.sql +++ /dev/null @@ -1,20 +0,0 @@ -DROP TABLE IF EXISTS ${stats_db_name}.project; - -CREATE TABLE ${stats_db_name}.project stored as parquet as -SELECT p.id , p.acronym, p.title, p.funder, p.funding_lvl0, p.funding_lvl1, p.funding_lvl2, p.ec39, p.type, p.startdate, p.enddate, p.start_year, p.end_year, p.duration, -CASE WHEN prr1.id IS NULL THEN 'no' ELSE 'yes' END as haspubs, -CASE WHEN prr1.id IS NULL THEN 0 ELSE prr1.np END as numpubs, -CASE WHEN prr2.id IS NULL THEN 0 ELSE prr2.daysForlastPub END as daysforlastpub, -CASE WHEN prr2.id IS NULL THEN 0 ELSE prr2.dp END as delayedpubs, -p.callidentifier, p.code -FROM ${stats_db_name}.project_tmp p -LEFT JOIN (SELECT pr.id, count(distinct pr.result) AS np - FROM ${stats_db_name}.project_results pr INNER JOIN ${stats_db_name}.result r ON pr.result=r.id - WHERE r.type='publication' - GROUP BY pr.id) AS prr1 on prr1.id = p.id -LEFT JOIN (SELECT pp.id, max(datediff(to_date(r.date), to_date(pp.enddate)) ) as daysForlastPub , count(distinct r.id) as dp - FROM ${stats_db_name}.project_tmp pp, ${stats_db_name}.project_results pr, ${stats_db_name}.result r - WHERE pp.id=pr.id AND pr.result=r.id AND r.type='publication' AND datediff(to_date(r.date), to_date(pp.enddate)) > 0 - GROUP BY pp.id) AS prr2 - on prr2.id = p.id; - \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_4.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_4.sql deleted file mode 100644 index 2fab61e8d..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_4.sql +++ /dev/null @@ -1,2 +0,0 @@ --- Publication temporary table updates -UPDATE ${stats_db_name}.publication_tmp SET delayed = 'yes' WHERE publication_tmp.id IN (SELECT distinct r.id FROM stats_wf_db_obs.result r, ${stats_db_name}.project_results pr, ${stats_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_5.sql deleted file mode 100644 index 3c7ee35de..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_5.sql +++ /dev/null @@ -1,2 +0,0 @@ --- Dataset temporary table updates -UPDATE ${stats_db_name}.dataset_tmp SET delayed = 'yes' WHERE dataset_tmp.id IN (SELECT distinct r.id FROM stats_wf_db_obs.result r, ${stats_db_name}.project_results pr, ${stats_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_6.sql deleted file mode 100644 index 5d38693bc..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_6.sql +++ /dev/null @@ -1,2 +0,0 @@ --- Software temporary table updates -UPDATE ${stats_db_name}.software_tmp SET delayed = 'yes' WHERE software_tmp.id IN (SELECT distinct r.id FROM ${stats_db_name}.result r, ${stats_db_name}.project_results pr, ${stats_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_7.sql deleted file mode 100644 index 4a5d4ce61..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_7.sql +++ /dev/null @@ -1,2 +0,0 @@ --- Oherresearchproduct temporary table updates -UPDATE ${stats_db_name}.otherresearchproduct_tmp SET delayed = 'yes' WHERE otherresearchproduct_tmp.id IN (SELECT distinct r.id FROM ${stats_db_name}.result r, ${stats_db_name}.project_results pr, ${stats_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_8.sql deleted file mode 100644 index ad375bf3f..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12_8.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE OR REPLACE VIEW ${stats_db_name}.project_results_publication AS SELECT result_projects.id AS result, result_projects.project AS project_results, result.date as resultdate, project.enddate as projectenddate, result_projects.daysfromend as daysfromend FROM ${stats_db_name}.result_projects, ${stats_db_name}.result, ${stats_db_name}.project WHERE result_projects.id=result.id and result.type='publication' and project.id=result_projects.project; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql index f68ccab64..e28424dec 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql @@ -1,26 +1,39 @@ ------------------------------------------------------------------------------------------------------ -- Creating parquet tables from the updated temporary tables and removing unnecessary temporary tables ------------------------------------------------------------------------------------------------------ +DROP TABLE IF EXISTS ${hive_db_name}.datasource; +CREATE TABLE ${hive_db_name}.datasource stored as parquet as select * from ${hive_db_name}.datasource_tmp; -DROP TABLE IF EXISTS ${stats_db_name}.datasource; -CREATE TABLE ${stats_db_name}.datasource stored as parquet as select * from ${stats_db_name}.datasource_tmp; +DROP TABLE IF EXISTS ${hive_db_name}.publication; +CREATE TABLE ${hive_db_name}.publication stored as parquet as select * from ${hive_db_name}.publication_tmp; -DROP TABLE IF EXISTS ${stats_db_name}.publication; -CREATE TABLE ${stats_db_name}.publication stored as parquet as select * from ${stats_db_name}.publication_tmp; +DROP TABLE IF EXISTS ${hive_db_name}.dataset; +CREATE TABLE ${hive_db_name}.dataset stored as parquet as select * from ${hive_db_name}.dataset_tmp; -DROP TABLE IF EXISTS ${stats_db_name}.dataset; -CREATE TABLE ${stats_db_name}.dataset stored as parquet as select * from ${stats_db_name}.dataset_tmp; +DROP TABLE IF EXISTS ${hive_db_name}.software; +CREATE TABLE ${hive_db_name}.software stored as parquet as select * from ${hive_db_name}.software_tmp; -DROP TABLE IF EXISTS ${stats_db_name}.software; -CREATE TABLE ${stats_db_name}.software stored as parquet as select * from ${stats_db_name}.software_tmp; +DROP TABLE IF EXISTS ${hive_db_name}.otherresearchproduct; +CREATE TABLE ${hive_db_name}.otherresearchproduct stored as parquet as select * from ${hive_db_name}.otherresearchproduct_tmp; -DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct; -CREATE TABLE ${stats_db_name}.otherresearchproduct stored as parquet as select * from ${stats_db_name}.otherresearchproduct_tmp; +DROP TABLE ${hive_db_name}.project_tmp; +DROP TABLE ${hive_db_name}.datasource_tmp; +DROP TABLE ${hive_db_name}.publication_tmp; +DROP TABLE ${hive_db_name}.dataset_tmp; +DROP TABLE ${hive_db_name}.software_tmp; +DROP TABLE ${hive_db_name}.otherresearchproduct_tmp; -DROP TABLE ${stats_db_name}.project_tmp; -DROP TABLE ${stats_db_name}.datasource_tmp; -DROP TABLE ${stats_db_name}.publication_tmp; -DROP TABLE ${stats_db_name}.dataset_tmp; -DROP TABLE ${stats_db_name}.software_tmp; -DROP TABLE ${stats_db_name}.otherresearchproduct_tmp; +---------------------------------------------- +-- Re-creating views from final parquet tables +--------------------------------------------- +-- Result +CREATE OR REPLACE VIEW ${hive_db_name}.result as SELECT *, bestlicence as access_mode FROM ${hive_db_name}.publication UNION ALL SELECT *, bestlicence as access_mode FROM ${hive_db_name}.software UNION ALL SELECT *, bestlicence as access_mode FROM ${hive_db_name}.dataset UNION ALL SELECT *, bestlicence as access_mode FROM ${hive_db_name}.otherresearchproduct; + +-- cleanup +drop view if exists ${hive_db_name}.delayedpubs; +drop view if exists ${hive_db_name}.project_pub_count; +drop view if exists ${hive_db_name}.delayedpubs; +drop view if exists ${hive_db_name}.project_results_publication; + + CREATE TABLE ${hive_db_name}.numbers_country AS SELECT org.country AS country, count(distinct rd.datasource) AS datasources, count(distinct r.id) AS publications FROM ${hive_db_name}.result r, ${hive_db_name}.result_datasources rd, ${hive_db_name}.datasource d, ${hive_db_name}.datasource_organizations dor, ${hive_db_name}.organization org WHERE r.id=rd.id AND rd.datasource=d.id AND d.id=dor.id AND dor.organization=org.id AND r.type='publication' AND r.bestlicence='Open Access' GROUP BY org.country; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql deleted file mode 100644 index 24e9e3107..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql +++ /dev/null @@ -1,7 +0,0 @@ ----------------------------------------------- --- Re-creating views from final parquet tables ---------------------------------------------- - --- Result -CREATE OR REPLACE VIEW ${stats_db_name}.result as SELECT *, bestlicence as access_mode FROM ${stats_db_name}.publication UNION ALL SELECT *, bestlicence as access_mode FROM ${stats_db_name}.software UNION ALL SELECT *, bestlicence as access_mode FROM ${stats_db_name}.dataset UNION ALL SELECT *, bestlicence as access_mode FROM ${stats_db_name}.otherresearchproduct; - diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql new file mode 100644 index 000000000..cddf83e2b --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql @@ -0,0 +1,32 @@ +-------------------------------------------------------------- +-------------------------------------------------------------- +-- 2. Publication table/view and Publication related tables/views +-------------------------------------------------------------- +-------------------------------------------------------------- + +-- Publication temporary table +DROP TABLE IF EXISTS ${hive_db_name}.publication_tmp; +CREATE TABLE ${hive_db_name}.publication_tmp (id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); + +-- The following fails +-- +INSERT INTO ${hive_db_name}.publication_tmp SELECT substr(p.id, 4) as id, p.title[0].value as title, p.publisher.value as publisher, p.journal.name as journal, +p.dateofacceptance.value as date, date_format(p.dateofacceptance.value,'yyyy') as year, p.bestaccessright.classname as bestlicence, +p.embargoenddate.value as embargo_end_date, false as delayed, size(p.author) as authors , concat_ws('\u003B',p.source.value) as source, +case when size(p.description) > 0 then true else false end as abstract, +'publication' as type +from ${hive_source_db_name}.publication p +where p.datainfo.deletedbyinference=false; + + +CREATE TABLE ${hive_db_name}.publication_classifications AS SELECT substr(p.id, 4) as id, instancetype.classname as type from ${hive_source_db_name}.publication p LATERAL VIEW explode(p.instance.instancetype) instances as instancetype; +CREATE TABLE ${hive_db_name}.publication_concepts AS SELECT substr(p.id, 4) as id, contexts.context.id as concept from ${hive_source_db_name}.publication p LATERAL VIEW explode(p.context) contexts as context; + +CREATE TABLE ${hive_db_name}.publication_datasources as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM (SELECT substr(p.id, 4) as id, substr(instances.instance.hostedby.key, 4) as datasource +from ${hive_source_db_name}.publication p lateral view explode(p.instance) instances as instance) p LEFT OUTER JOIN +(SELECT substr(d.id, 4) id from ${hive_source_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; + +CREATE TABLE ${hive_db_name}.publication_languages AS select substr(p.id, 4) as id, p.language.classname as language from ${hive_source_db_name}.publication p; +CREATE TABLE ${hive_db_name}.publication_oids AS SELECT substr(p.id, 4) as id, oids.ids as oid from ${hive_source_db_name}.publication p LATERAL VIEW explode(p.originalid) oids as ids; +create table ${hive_db_name}.publication_pids as select substr(p.id, 4) as id, ppid.qualifier.classname as type, ppid.value as pid from ${hive_source_db_name}.publication p lateral view explode(p.pid) pids as ppid; +create table ${hive_db_name}.publication_topics as select substr(p.id, 4) as id, subjects.subject.qualifier.classname as type, subjects.subject.value as topic from ${hive_source_db_name}.publication p lateral view explode(p.subject) subjects as subject; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_1.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_1.sql deleted file mode 100644 index d20bb73e2..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_1.sql +++ /dev/null @@ -1,10 +0,0 @@ --------------------------------------------------------------- --------------------------------------------------------------- --- 2. Publication table/view and Publication related tables/views --------------------------------------------------------------- --------------------------------------------------------------- - --- Publication temporary table -DROP TABLE IF EXISTS ${stats_db_name}.publication_tmp; - -CREATE TABLE ${stats_db_name}.publication_tmp (id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_2.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_2.sql deleted file mode 100644 index 1c67596db..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_2.sql +++ /dev/null @@ -1,19 +0,0 @@ --- The following throws the following exception on CRN HUE Hive: --- Error while compiling statement: FAILED: SemanticException [Error 10011]: Line 2:34 Invalid function 'date_format' --- But runs OK on OCEAN HUE Hive - -INSERT INTO ${stats_db_name}.publication_tmp SELECT substr(p.id, 4) as id, p.title[0].value as title, p.publisher.value as publisher, p.journal.name as journal , -p.dateofacceptance.value as date, date_format(p.dateofacceptance.value,'yyyy') as year, p.bestaccessright.classname as bestlicence, -p.embargoenddate.value as embargo_end_date, false as delayed, size(p.author) as authors , concat_ws('\u003B',p.source.value) as source, -case when size(p.description) > 0 then true else false end as abstract, -'publication' as type -from ${openaire_db_name}.publication p -where p.datainfo.deletedbyinference=false; - --- INSERT INTO ${hive_db_name}.publication_tmp SELECT substr(p.id, 4) as id, p.title[0].value as title, p.publisher.value as publisher, p.journal.name as journal, --- p.dateofacceptance.value as date, date_format(p.dateofacceptance.value,'yyyy') as year, p.bestaccessright.classname as bestlicence, --- p.embargoenddate.value as embargo_end_date, false as delayed, size(p.author) as authors , concat_ws('\u003B',p.source.value) as source, --- case when size(p.description) > 0 then true else false end as abstract, --- 'publication' as type --- from openaire.publication p --- where p.datainfo.deletedbyinference=false; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_3.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_3.sql deleted file mode 100644 index bf0e85550..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_3.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE ${stats_db_name}.publication_classifications AS SELECT substr(p.id, 4) as id, instancetype.classname as type from ${openaire_db_name}.publication p LATERAL VIEW explode(p.instance.instancetype) instances as instancetype; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_4.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_4.sql deleted file mode 100644 index a38bcf62d..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_4.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE ${stats_db_name}.publication_concepts AS SELECT substr(p.id, 4) as id, contexts.context.id as concept from ${openaire_db_name}.publication p LATERAL VIEW explode(p.context) contexts as context; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_5.sql deleted file mode 100644 index 299833452..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_5.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE ${stats_db_name}.publication_datasources as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM (SELECT substr(p.id, 4) as id, substr(instances.instance.hostedby.key, 4) as datasource from ${openaire_db_name}.publication p lateral view explode(p.instance) instances as instance) p LEFT OUTER JOIN (SELECT substr(d.id, 4) id from ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_6.sql deleted file mode 100644 index fd4dbe05c..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_6.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE ${stats_db_name}.publication_languages AS select substr(p.id, 4) as id, p.language.classname as language from ${openaire_db_name}.publication p; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_7.sql deleted file mode 100644 index 24cd59670..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_7.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE ${stats_db_name}.publication_oids AS SELECT substr(p.id, 4) as id, oids.ids as oid from ${openaire_db_name}.publication p LATERAL VIEW explode(p.originalid) oids as ids; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_8.sql deleted file mode 100644 index 1ddb2bd3b..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_8.sql +++ /dev/null @@ -1 +0,0 @@ -create table ${stats_db_name}.publication_pids as select substr(p.id, 4) as id, ppid.qualifier.classname as type, ppid.value as pid from ${openaire_db_name}.publication p lateral view explode(p.pid) pids as ppid; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_9.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_9.sql deleted file mode 100644 index 5507eeb7a..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2_9.sql +++ /dev/null @@ -1 +0,0 @@ -create table ${stats_db_name}.publication_topics as select substr(p.id, 4) as id, subjects.subject.qualifier.classname as type, subjects.subject.value as topic from ${openaire_db_name}.publication p lateral view explode(p.subject) subjects as subject; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql index 567b94994..95d167b2d 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql @@ -1,2 +1,4 @@ +-- The following fails +-- -- 3. Publication_citations -CREATE TABLE ${stats_db_name}.publication_citations AS SELECT substr(p.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS result FROM ${openaire_db_name}.publication p lateral view explode(p.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; +CREATE TABLE ${hive_db_name}.publication_citations AS SELECT substr(p.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS result from ${hive_source_db_name}.publication p lateral view explode(p.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql new file mode 100644 index 000000000..de842d6df --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql @@ -0,0 +1,26 @@ +------------------------------------------------------ +------------------------------------------------------ +-- 4. Dataset table/view and Dataset related tables/views +------------------------------------------------------ +------------------------------------------------------ + +-- Dataset temporary table supporting updates +DROP TABLE IF EXISTS ${hive_db_name}.dataset_tmp; +CREATE TABLE ${hive_db_name}.dataset_tmp (id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); + +INSERT INTO ${hive_db_name}.dataset_tmp SELECT substr(d.id, 4) as id, d.title[0].value as title, d.publisher.value as publisher, cast(null as string) as journal, d.dateofacceptance.value as date, +date_format(d.dateofacceptance.value,'yyyy') as year, d.bestaccessright.classname as bestlicence, d.embargoenddate.value as embargo_end_date, false as delayed, size(d.author) as authors, +concat_ws ('\u003B',d.source.value) as source, +case when size(d.description) > 0 then true else false end as abstract, 'dataset' as type from ${hive_source_db_name}.dataset d where d.datainfo.deletedbyinference=false; + + +-- Dataset_citations +CREATE TABLE ${hive_db_name}.dataset_citations as select substr(d.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") as result from ${hive_source_db_name}.dataset d lateral view explode(d.extrainfo) citations as citation where xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; + +CREATE TABLE ${hive_db_name}.dataset_classifications AS SELECT substr(p.id, 4) as id, instancetype.classname as type from ${hive_source_db_name}.dataset p LATERAL VIEW explode(p.instance.instancetype) instances as instancetype; +CREATE TABLE ${hive_db_name}.dataset_concepts AS SELECT substr(p.id, 4) as id, contexts.context.id as concept from ${hive_source_db_name}.dataset p LATERAL VIEW explode(p.context) contexts as context; +CREATE TABLE ${hive_db_name}.dataset_datasources as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM (SELECT substr(p.id, 4) as id, substr(instances.instance.hostedby.key, 4) as datasource from ${hive_source_db_name}.dataset p lateral view explode(p.instance) instances as instance) p LEFT OUTER JOIN (SELECT substr(d.id, 4) id from ${hive_source_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; +CREATE TABLE ${hive_db_name}.dataset_languages AS SELECT substr(p.id, 4) as id, p.language.classname as language from ${hive_source_db_name}.dataset p; +CREATE TABLE ${hive_db_name}.dataset_oids AS SELECT substr(p.id, 4) as id, oids.ids as oid from ${hive_source_db_name}.dataset p LATERAL VIEW explode(p.originalid) oids as ids; +CREATE TABLE ${hive_db_name}.dataset_pids AS SELECT substr(p.id, 4) as id, ppid.qualifier.classname as type, ppid.value as pid from ${hive_source_db_name}.dataset p lateral view explode(p.pid) pids as ppid; +CREATE TABLE ${hive_db_name}.dataset_topics AS SELECT substr(p.id, 4) as id, subjects.subject.qualifier.classname as type, subjects.subject.value as topic from ${hive_source_db_name}.dataset p lateral view explode(p.subject) subjects as subject; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_1.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_1.sql deleted file mode 100644 index 3668059a6..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_1.sql +++ /dev/null @@ -1,9 +0,0 @@ ------------------------------------------------------- ------------------------------------------------------- --- 4. Dataset table/view and Dataset related tables/views ------------------------------------------------------- ------------------------------------------------------- - --- Dataset temporary table supporting updates -DROP TABLE IF EXISTS ${stats_db_name}.dataset_tmp; -CREATE TABLE ${stats_db_name}.dataset_tmp (id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_10.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_10.sql deleted file mode 100644 index 85a8eac7f..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_10.sql +++ /dev/null @@ -1 +0,0 @@ -create table ${stats_db_name}.dataset_topics as select substr(p.id, 4) as id, subjects.subject.qualifier.classname as type, subjects.subject.value as topic from ${openaire_db_name}.dataset p lateral view explode(p.subject) subjects as subject; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_2.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_2.sql deleted file mode 100644 index 3c8bb7c0f..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_2.sql +++ /dev/null @@ -1,7 +0,0 @@ -INSERT INTO ${stats_db_name}.dataset_tmp select substr(d.id, 4) as id, d.title[0].value as title, d.publisher.value as publisher, cast(null as string) as journal, -d.dateofacceptance.value as date, date_format(d.dateofacceptance.value,'yyyy') as year, d.bestaccessright.classname as bestlicence, -d.embargoenddate.value as embargo_end_date, false as delayed, size(d.author) as authors , concat_ws('\u003B',d.source.value) as source, - case when size(d.description) > 0 then true else false end as abstract, -'dataset' as type -from ${openaire_db_name}.dataset d -where d.datainfo.deletedbyinference=false; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_3.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_3.sql deleted file mode 100644 index 54e23621a..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_3.sql +++ /dev/null @@ -1,2 +0,0 @@ --- Dataset_citations -Create table ${stats_db_name}.dataset_citations as select substr(d.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") as result from ${openaire_db_name}.dataset d lateral view explode(d.extrainfo) citations as citation where xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_4.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_4.sql deleted file mode 100644 index 7bc81c7f8..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_4.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE ${stats_db_name}.dataset_classifications AS SELECT substr(p.id, 4) as id, instancetype.classname as type from ${openaire_db_name}.dataset p LATERAL VIEW explode(p.instance.instancetype) instances as instancetype; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_5.sql deleted file mode 100644 index 73aff989d..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_5.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE ${stats_db_name}.dataset_concepts AS SELECT substr(p.id, 4) as id, contexts.context.id as concept from ${openaire_db_name}.dataset p LATERAL VIEW explode(p.context) contexts as context; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_6.sql deleted file mode 100644 index 1ca686f70..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_6.sql +++ /dev/null @@ -1,3 +0,0 @@ -CREATE TABLE ${stats_db_name}.dataset_datasources as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM (SELECT substr(p.id, 4) as id, substr(instances.instance.hostedby.key, 4) as datasource -from ${openaire_db_name}.dataset p lateral view explode(p.instance) instances as instance) p LEFT OUTER JOIN -(SELECT substr(d.id, 4) id from ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_7.sql deleted file mode 100644 index 70c0b3e4f..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_7.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE ${stats_db_name}.dataset_languages AS select substr(p.id, 4) as id, p.language.classname as language from ${openaire_db_name}.dataset p; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_8.sql deleted file mode 100644 index c750ad00b..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_8.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE ${stats_db_name}.dataset_oids AS SELECT substr(p.id, 4) as id, oids.ids as oid from ${openaire_db_name}.dataset p LATERAL VIEW explode(p.originalid) oids as ids; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_9.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_9.sql deleted file mode 100644 index 1d9f4c596..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4_9.sql +++ /dev/null @@ -1 +0,0 @@ -create table ${stats_db_name}.dataset_pids as select substr(p.id, 4) as id, ppid.qualifier.classname as type, ppid.value as pid from ${openaire_db_name}.dataset p lateral view explode(p.pid) pids as ppid; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql new file mode 100644 index 000000000..21175a5da --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql @@ -0,0 +1,32 @@ +-------------------------------------------------------- +-------------------------------------------------------- +-- 5. Software table/view and Software related tables/views +-------------------------------------------------------- +-------------------------------------------------------- + +-- Software temporary table supporting updates +DROP TABLE IF EXISTS ${hive_db_name}.software_tmp; +CREATE TABLE ${hive_db_name}.software_tmp ( id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); +INSERT INTO ${hive_db_name}.software_tmp select substr(s.id, 4) as id, s.title[0].value as title, s.publisher.value as publisher, cast(null as string) as journal, +s.dateofacceptance.value as date, date_format(s.dateofacceptance.value,'yyyy') as year, s.bestaccessright.classname as bestlicence, +s.embargoenddate.value as embargo_end_date, false as delayed, size(s.author) as authors , concat_ws('\u003B',s.source.value) as source, + case when size(s.description) > 0 then true else false end as abstract, +'software' as type +from ${hive_source_db_name}.software s +where s.datainfo.deletedbyinference=false; + + +-- Software_citations +Create table ${hive_db_name}.software_citations as select substr(s.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") as result from ${hive_source_db_name}.software s lateral view explode(s.extrainfo) citations as citation where xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; + +CREATE TABLE ${hive_db_name}.software_classifications AS SELECT substr(p.id, 4) as id, instancetype.classname as type from ${hive_source_db_name}.software p LATERAL VIEW explode(p.instance.instancetype) instances as instancetype; +CREATE TABLE ${hive_db_name}.software_concepts AS SELECT substr(p.id, 4) as id, contexts.context.id as concept from ${hive_source_db_name}.software p LATERAL VIEW explode(p.context) contexts as context; + +CREATE TABLE ${hive_db_name}.software_datasources as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM (SELECT substr(p.id, 4) as id, substr(instances.instance.hostedby.key, 4) as datasource +from ${hive_source_db_name}.software p lateral view explode(p.instance) instances as instance) p LEFT OUTER JOIN +(SELECT substr(d.id, 4) id from ${hive_source_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; + +CREATE TABLE ${hive_db_name}.software_languages AS select substr(p.id, 4) as id, p.language.classname as language from ${hive_source_db_name}.software p; +CREATE TABLE ${hive_db_name}.software_oids AS SELECT substr(p.id, 4) as id, oids.ids as oid from ${hive_source_db_name}.software p LATERAL VIEW explode(p.originalid) oids as ids; +create table ${hive_db_name}.software_pids as select substr(p.id, 4) as id, ppid.qualifier.classname as type, ppid.value as pid from ${hive_source_db_name}.software p lateral view explode(p.pid) pids as ppid; +create table ${hive_db_name}.software_topics as select substr(p.id, 4) as id, subjects.subject.qualifier.classname as type, subjects.subject.value as topic from ${hive_source_db_name}.software p lateral view explode(p.subject) subjects as subject; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_1.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_1.sql deleted file mode 100644 index 89f869760..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_1.sql +++ /dev/null @@ -1,9 +0,0 @@ --------------------------------------------------------- --------------------------------------------------------- --- 5. Software table/view and Software related tables/views --------------------------------------------------------- --------------------------------------------------------- - --- Software temporary table supporting updates -DROP TABLE IF EXISTS ${stats_db_name}.software_tmp; -CREATE TABLE ${stats_db_name}.software_tmp (id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_10.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_10.sql deleted file mode 100644 index acc70958b..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_10.sql +++ /dev/null @@ -1 +0,0 @@ -create table ${stats_db_name}.software_topics as select substr(p.id, 4) as id, subjects.subject.qualifier.classname as type, subjects.subject.value as topic from ${openaire_db_name}.software p lateral view explode(p.subject) subjects as subject; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_2.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_2.sql deleted file mode 100644 index 4a3f10c64..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_2.sql +++ /dev/null @@ -1,7 +0,0 @@ -INSERT INTO ${stats_db_name}.software_tmp select substr(s.id, 4) as id, s.title[0].value as title, s.publisher.value as publisher, cast(null as string) as journal, -s.dateofacceptance.value as date, date_format(s.dateofacceptance.value,'yyyy') as year, s.bestaccessright.classname as bestlicence, -s.embargoenddate.value as embargo_end_date, false as delayed, size(s.author) as authors , concat_ws('\u003B',s.source.value) as source, - case when size(s.description) > 0 then true else false end as abstract, -'software' as type -from ${openaire_db_name}.software s -where s.datainfo.deletedbyinference=false; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_3.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_3.sql deleted file mode 100644 index f8073a583..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_3.sql +++ /dev/null @@ -1,2 +0,0 @@ --- Software_citations -Create table ${stats_db_name}.software_citations as select substr(s.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") as result from ${openaire_db_name}.software s lateral view explode(s.extrainfo) citations as citation where xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_4.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_4.sql deleted file mode 100644 index b340cfc24..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_4.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE ${stats_db_name}.software_classifications AS SELECT substr(p.id, 4) as id, instancetype.classname as type from ${openaire_db_name}.software p LATERAL VIEW explode(p.instance.instancetype) instances as instancetype; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_5.sql deleted file mode 100644 index d2b0f3d1d..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_5.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE ${stats_db_name}.software_concepts AS SELECT substr(p.id, 4) as id, contexts.context.id as concept from ${openaire_db_name}.software p LATERAL VIEW explode(p.context) contexts as context; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_6.sql deleted file mode 100644 index dfe32075b..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_6.sql +++ /dev/null @@ -1,3 +0,0 @@ -CREATE TABLE ${stats_db_name}.software_datasources as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM (SELECT substr(p.id, 4) as id, substr(instances.instance.hostedby.key, 4) as datasource -from ${openaire_db_name}.software p lateral view explode(p.instance) instances as instance) p LEFT OUTER JOIN -(SELECT substr(d.id, 4) id from ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_7.sql deleted file mode 100644 index 0fdb0fab3..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_7.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE ${stats_db_name}.software_languages AS select substr(p.id, 4) as id, p.language.classname as language from ${openaire_db_name}.software p; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_8.sql deleted file mode 100644 index 79c4b313d..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_8.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE ${stats_db_name}.software_oids AS SELECT substr(p.id, 4) as id, oids.ids as oid from ${openaire_db_name}.software p LATERAL VIEW explode(p.originalid) oids as ids; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_9.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_9.sql deleted file mode 100644 index a8aaa5d83..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5_9.sql +++ /dev/null @@ -1 +0,0 @@ -create table ${stats_db_name}.software_pids as select substr(p.id, 4) as id, ppid.qualifier.classname as type, ppid.value as pid from ${openaire_db_name}.software p lateral view explode(p.pid) pids as ppid; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql new file mode 100644 index 000000000..58967a840 --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql @@ -0,0 +1,31 @@ +-------------------------------------------------------------------------------- +-------------------------------------------------------------------------------- +-- 6. Otherresearchproduct table/view and Otherresearchproduct related tables/views +-------------------------------------------------------------------------------- +-------------------------------------------------------------------------------- + +-- Otherresearchproduct temporary table supporting updates +DROP TABLE IF EXISTS ${hive_db_name}.otherresearchproduct_tmp; +CREATE TABLE ${hive_db_name}.otherresearchproduct_tmp ( id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); +INSERT INTO ${hive_db_name}.otherresearchproduct_tmp select substr(o.id, 4) as id, o.title[0].value as title, o.publisher.value as publisher, cast(null as string) as journal, +o.dateofacceptance.value as date, date_format(o.dateofacceptance.value,'yyyy') as year, o.bestaccessright.classname as bestlicence, +o.embargoenddate.value as embargo_end_date, false as delayed, size(o.author) as authors , concat_ws('\u003B',o.source.value) as source, +case when size(o.description) > 0 then true else false end as abstract, +'other' as type +from ${hive_source_db_name}.otherresearchproduct o +where o.datainfo.deletedbyinference=false; + +-- Otherresearchproduct_citations +Create table ${hive_db_name}.otherresearchproduct_citations as select substr(o.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") as result from ${hive_source_db_name}.otherresearchproduct o lateral view explode(o.extrainfo) citations as citation where xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; + +CREATE TABLE ${hive_db_name}.otherresearchproduct_classifications AS SELECT substr(p.id, 4) as id, instancetype.classname as type from ${hive_source_db_name}.otherresearchproduct p LATERAL VIEW explode(p.instance.instancetype) instances as instancetype; +CREATE TABLE ${hive_db_name}.otherresearchproduct_concepts AS SELECT substr(p.id, 4) as id, contexts.context.id as concept from ${hive_source_db_name}.otherresearchproduct p LATERAL VIEW explode(p.context) contexts as context; + +CREATE TABLE ${hive_db_name}.otherresearchproduct_datasources as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM (SELECT substr(p.id, 4) as id, substr(instances.instance.hostedby.key, 4) as datasource +from ${hive_source_db_name}.otherresearchproduct p lateral view explode(p.instance) instances as instance) p LEFT OUTER JOIN +(SELECT substr(d.id, 4) id from ${hive_source_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; + +CREATE TABLE ${hive_db_name}.otherresearchproduct_languages AS select substr(p.id, 4) as id, p.language.classname as language from ${hive_source_db_name}.otherresearchproduct p; +CREATE TABLE ${hive_db_name}.otherresearchproduct_oids AS SELECT substr(p.id, 4) as id, oids.ids as oid from ${hive_source_db_name}.otherresearchproduct p LATERAL VIEW explode(p.originalid) oids as ids; +create table ${hive_db_name}.otherresearchproduct_pids as select substr(p.id, 4) as id, ppid.qualifier.classname as type, ppid.value as pid from ${hive_source_db_name}.otherresearchproduct p lateral view explode(p.pid) pids as ppid; +create table ${hive_db_name}.otherresearchproduct_topics as select substr(p.id, 4) as id, subjects.subject.qualifier.classname as type, subjects.subject.value as topic from ${hive_source_db_name}.otherresearchproduct p lateral view explode(p.subject) subjects as subject; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_1.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_1.sql deleted file mode 100644 index 4e59465fa..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_1.sql +++ /dev/null @@ -1,9 +0,0 @@ --------------------------------------------------------------------------------- --------------------------------------------------------------------------------- --- 6. Otherresearchproduct table/view and Otherresearchproduct related tables/views --------------------------------------------------------------------------------- --------------------------------------------------------------------------------- - --- Otherresearchproduct temporary table supporting updates -DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_tmp; -CREATE TABLE ${stats_db_name}.otherresearchproduct_tmp ( id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_10.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_10.sql deleted file mode 100644 index e2e788634..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_10.sql +++ /dev/null @@ -1 +0,0 @@ -create table ${stats_db_name}.otherresearchproduct_topics as select substr(p.id, 4) as id, subjects.subject.qualifier.classname as type, subjects.subject.value as topic from ${openaire_db_name}.otherresearchproduct p lateral view explode(p.subject) subjects as subject; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_2.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_2.sql deleted file mode 100644 index 7c7ed370d..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_2.sql +++ /dev/null @@ -1,7 +0,0 @@ -INSERT INTO ${stats_db_name}.otherresearchproduct_tmp select substr(o.id, 4) as id, o.title[0].value as title, o.publisher.value as publisher, cast(null as string) as journal, -o.dateofacceptance.value as date, date_format(o.dateofacceptance.value,'yyyy') as year, o.bestaccessright.classname as bestlicence, -o.embargoenddate.value as embargo_end_date, false as delayed, size(o.author) as authors , concat_ws('\u003B',o.source.value) as source, -case when size(o.description) > 0 then true else false end as abstract, -'other' as type -from ${openaire_db_name}.otherresearchproduct o -where o.datainfo.deletedbyinference=false; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_3.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_3.sql deleted file mode 100644 index c6b9255df..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_3.sql +++ /dev/null @@ -1,2 +0,0 @@ --- Otherresearchproduct_citations -Create table ${stats_db_name}.otherresearchproduct_citations as select substr(o.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") as result from ${openaire_db_name}.otherresearchproduct o lateral view explode(o.extrainfo) citations as citation where xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_4.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_4.sql deleted file mode 100644 index 0a93764b6..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_4.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE ${stats_db_name}.otherresearchproduct_classifications AS SELECT substr(p.id, 4) as id, instancetype.classname as type from ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.instance.instancetype) instances as instancetype; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_5.sql deleted file mode 100644 index a37f2f25e..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_5.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE ${stats_db_name}.otherresearchproduct_concepts AS SELECT substr(p.id, 4) as id, contexts.context.id as concept from ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.context) contexts as context; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_6.sql deleted file mode 100644 index b7aa69e9a..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_6.sql +++ /dev/null @@ -1,3 +0,0 @@ -CREATE TABLE ${stats_db_name}.otherresearchproduct_datasources as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM (SELECT substr(p.id, 4) as id, substr(instances.instance.hostedby.key, 4) as datasource -from ${openaire_db_name}.otherresearchproduct p lateral view explode(p.instance) instances as instance) p LEFT OUTER JOIN -(SELECT substr(d.id, 4) id from ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_7.sql deleted file mode 100644 index ba9208cf6..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_7.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE ${stats_db_name}.otherresearchproduct_languages AS select substr(p.id, 4) as id, p.language.classname as language from ${openaire_db_name}.otherresearchproduct p; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_8.sql deleted file mode 100644 index 4f2c5c3bb..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_8.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE TABLE ${stats_db_name}.otherresearchproduct_oids AS SELECT substr(p.id, 4) as id, oids.ids as oid from ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.originalid) oids as ids; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_9.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_9.sql deleted file mode 100644 index 4ab11adfe..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6_9.sql +++ /dev/null @@ -1 +0,0 @@ -create table ${stats_db_name}.otherresearchproduct_pids as select substr(p.id, 4) as id, ppid.qualifier.classname as type, ppid.value as pid from ${openaire_db_name}.otherresearchproduct p lateral view explode(p.pid) pids as ppid; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql new file mode 100644 index 000000000..fa5838b61 --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql @@ -0,0 +1,21 @@ +------------------------------------------------------ +------------------------------------------------------ +-- 7. Project table/view and Project related tables/views +------------------------------------------------------ +------------------------------------------------------ +-- Project_oids Table +DROP TABLE IF EXISTS ${hive_db_name}.project_oids; +CREATE TABLE ${hive_db_name}.project_oids AS SELECT substr(p.id, 4) as id, oids.ids as oid from ${hive_source_db_name}.project p LATERAL VIEW explode(p.originalid) oids as ids; +-- Project_organizations Table +DROP TABLE IF EXISTS ${hive_db_name}.project_organizations; +CREATE TABLE ${hive_db_name}.project_organizations AS SELECT substr(r.source, 4) as id, substr(r.target, 4) AS organization from ${hive_source_db_name}.relation r WHERE r.reltype='projectOrganization'; +-- Project_results Table +DROP TABLE IF EXISTS ${hive_db_name}.project_results; +CREATE TABLE ${hive_db_name}.project_results AS SELECT substr(r.target, 4) as id, substr(r.source, 4) AS result from ${hive_source_db_name}.relation r WHERE r.reltype='resultProject'; + +-- Project table +---------------- +-- Creating and populating temporary Project table +DROP TABLE IF EXISTS ${hive_db_name}.project_tmp; +CREATE TABLE ${hive_db_name}.project_tmp (id STRING, acronym STRING, title STRING, funder STRING, funding_lvl0 STRING, funding_lvl1 STRING, funding_lvl2 STRING, ec39 STRING, type STRING, startdate STRING, enddate STRING, start_year STRING, end_year STRING, duration INT, haspubs STRING, numpubs INT, daysforlastpub INT, delayedpubs INT, callidentifier STRING, code STRING) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); +INSERT INTO ${hive_db_name}.project_tmp SELECT substr(p.id, 4) as id, p.acronym.value as acronym, p.title.value as title, xpath_string(p.fundingtree[0].value, '//funder/name') as funder, xpath_string(p.fundingtree[0].value, '//funding_level_0/name') as funding_lvl0, xpath_string(p.fundingtree[0].value, '//funding_level_1/name') as funding_lvl1, xpath_string(p.fundingtree[0].value, '//funding_level_2/name') as funding_lvl2, p.ecsc39.value as ec39, p.contracttype.classname as type, p.startdate.value as startdate, p.enddate.value as enddate, date_format(p.startdate.value, 'yyyy') as start_year, date_format(p.enddate.value, 'yyyy') as end_year, 0 as duration, 'no' as haspubs, 0 as numpubs, 0 as daysforlastpub, 0 as delayedpubs, p.callidentifier.value as callidentifier, p.code.value as code from ${hive_source_db_name}.project p WHERE p.datainfo.deletedbyinference=false; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_1.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_1.sql deleted file mode 100644 index 0c775452d..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_1.sql +++ /dev/null @@ -1,8 +0,0 @@ ------------------------------------------------------- ------------------------------------------------------- --- 7. Project table/view and Project related tables/views ------------------------------------------------------- ------------------------------------------------------- --- Project_oids Table -DROP TABLE IF EXISTS ${stats_db_name}.project_oids; -CREATE TABLE ${stats_db_name}.project_oids AS SELECT substr(p.id, 4) as id, oids.ids as oid from ${openaire_db_name}.project p LATERAL VIEW explode(p.originalid) oids as ids; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_2.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_2.sql deleted file mode 100644 index 27770c390..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_2.sql +++ /dev/null @@ -1,3 +0,0 @@ --- Project_organizations Table -DROP TABLE IF EXISTS ${stats_db_name}.project_organizations; -CREATE TABLE ${stats_db_name}.project_organizations AS SELECT substr(r.source, 4) as id, substr(r.target, 4) AS organization from ${openaire_db_name}.relation r WHERE r.reltype='projectOrganization'; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_3.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_3.sql deleted file mode 100644 index fcefd7dcc..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_3.sql +++ /dev/null @@ -1,3 +0,0 @@ --- Project_results Table -DROP TABLE IF EXISTS ${stats_db_name}.project_results; -CREATE TABLE ${stats_db_name}.project_results AS SELECT substr(r.target, 4) as id, substr(r.source, 4) AS result from ${openaire_db_name}.relation r WHERE r.reltype='resultProject'; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_4.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_4.sql deleted file mode 100644 index a80573765..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_4.sql +++ /dev/null @@ -1,5 +0,0 @@ --- Project table ----------------- --- Creating and populating temporary Project table -DROP TABLE IF EXISTS ${stats_db_name}.project_tmp; -CREATE TABLE ${stats_db_name}.project_tmp (id STRING, acronym STRING, title STRING, funder STRING, funding_lvl0 STRING, funding_lvl1 STRING, funding_lvl2 STRING, ec39 STRING, type STRING, startdate STRING, enddate STRING, start_year STRING, end_year STRING, duration INT, haspubs STRING, numpubs INT, daysforlastpub INT, delayedpubs INT, callidentifier STRING, code STRING) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_5.sql deleted file mode 100644 index c633d9177..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7_5.sql +++ /dev/null @@ -1 +0,0 @@ -INSERT INTO ${stats_db_name}.project_tmp SELECT substr(p.id, 4) as id, p.acronym.value as acronym, p.title.value as title, xpath_string(p.fundingtree[0].value, '//funder/name') as funder, xpath_string(p.fundingtree[0].value, '//funding_level_0/name') as funding_lvl0, xpath_string(p.fundingtree[0].value, '//funding_level_1/name') as funding_lvl1, xpath_string(p.fundingtree[0].value, '//funding_level_2/name') as funding_lvl2, p.ecsc39.value as ec39, p.contracttype.classname as type, p.startdate.value as startdate, p.enddate.value as enddate, date_format(p.startdate.value, 'yyyy') as start_year, date_format(p.enddate.value, 'yyyy') as end_year, 0 as duration, 'no' as haspubs, 0 as numpubs, 0 as daysforlastpub, 0 as delayedpubs, p.callidentifier.value as callidentifier, p.code.value as code from ${openaire_db_name}.project p WHERE p.datainfo.deletedbyinference=false; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql new file mode 100644 index 000000000..3d346fb95 --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql @@ -0,0 +1,24 @@ +---------------------------------------------------- +---------------------------------------------------- +-- 8. Result table/view and Result related tables/views +---------------------------------------------------- +---------------------------------------------------- + +-- Views on temporary tables that should be re-created in the end +CREATE OR REPLACE VIEW ${hive_db_name}.result as SELECT *, bestlicence as access_mode FROM ${hive_db_name}.publication_tmp UNION ALL SELECT *,bestlicence as access_mode FROM ${hive_db_name}.software_tmp UNION ALL SELECT *,bestlicence as access_mode FROM ${hive_db_name}.dataset_tmp UNION ALL SELECT *,bestlicence as access_mode FROM ${hive_db_name}.otherresearchproduct_tmp; + +-- Views on final tables +CREATE OR REPLACE VIEW ${hive_db_name}.result_datasources as SELECT * FROM ${hive_db_name}.publication_datasources UNION ALL SELECT * FROM ${hive_db_name}.software_datasources UNION ALL SELECT * FROM ${hive_db_name}.dataset_datasources UNION ALL SELECT * FROM ${hive_db_name}.otherresearchproduct_datasources; +CREATE OR REPLACE VIEW ${hive_db_name}.result_citations as SELECT * FROM ${hive_db_name}.publication_citations UNION ALL SELECT * FROM ${hive_db_name}.software_citations UNION ALL SELECT * FROM ${hive_db_name}.dataset_citations UNION ALL SELECT * FROM ${hive_db_name}.otherresearchproduct_citations; +CREATE OR REPLACE VIEW ${hive_db_name}.result_classifications as SELECT * FROM ${hive_db_name}.publication_classifications UNION ALL SELECT * FROM ${hive_db_name}.software_classifications UNION ALL SELECT * FROM ${hive_db_name}.dataset_classifications UNION ALL SELECT * FROM ${hive_db_name}.otherresearchproduct_classifications; +CREATE OR REPLACE VIEW ${hive_db_name}.result_concepts as SELECT * FROM ${hive_db_name}.publication_concepts UNION ALL SELECT * FROM ${hive_db_name}.software_concepts UNION ALL SELECT * FROM ${hive_db_name}.dataset_concepts UNION ALL SELECT * FROM ${hive_db_name}.otherresearchproduct_concepts; +CREATE OR REPLACE VIEW ${hive_db_name}.result_languages as SELECT * FROM ${hive_db_name}.publication_languages UNION ALL SELECT * FROM ${hive_db_name}.software_languages UNION ALL SELECT * FROM ${hive_db_name}.dataset_languages UNION ALL SELECT * FROM ${hive_db_name}.otherresearchproduct_languages; +CREATE OR REPLACE VIEW ${hive_db_name}.result_oids as SELECT * FROM ${hive_db_name}.publication_oids UNION ALL SELECT * FROM ${hive_db_name}.software_oids UNION ALL SELECT * FROM ${hive_db_name}.dataset_oids UNION ALL SELECT * FROM ${hive_db_name}.otherresearchproduct_oids; +CREATE OR REPLACE VIEW ${hive_db_name}.result_pids as SELECT * FROM ${hive_db_name}.publication_pids UNION ALL SELECT * FROM ${hive_db_name}.software_pids UNION ALL SELECT * FROM ${hive_db_name}.dataset_pids UNION ALL SELECT * FROM ${hive_db_name}.otherresearchproduct_pids; +CREATE OR REPLACE VIEW ${hive_db_name}.result_topics as SELECT * FROM ${hive_db_name}.publication_topics UNION ALL SELECT * FROM ${hive_db_name}.software_topics UNION ALL SELECT * FROM ${hive_db_name}.dataset_topics UNION ALL SELECT * FROM ${hive_db_name}.otherresearchproduct_topics; + +DROP TABLE IF EXISTS ${hive_db_name}.result_organization; +CREATE TABLE ${hive_db_name}.result_organization AS SELECT substr(r.target, 4) as id, substr(r.source, 4) as organization from ${hive_source_db_name}.relation r where r.reltype='resultOrganization'; + +DROP TABLE IF EXISTS ${hive_db_name}.result_projects; +CREATE TABLE ${hive_db_name}.result_projects AS select pr.result as id, pr.id as project, datediff(p.enddate, p.startdate) as daysfromend from ${hive_db_name}.result r join ${hive_db_name}.project_results pr on r.id=pr.result join ${hive_db_name}.project_tmp p on p.id=pr.id; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_1.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_1.sql deleted file mode 100644 index e855e59ff..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_1.sql +++ /dev/null @@ -1,8 +0,0 @@ ----------------------------------------------------- ----------------------------------------------------- --- 8. Result table/view and Result related tables/views ----------------------------------------------------- ----------------------------------------------------- - --- Views on temporary tables that should be re-created in the end -CREATE OR REPLACE VIEW ${stats_db_name}.result as SELECT *, bestlicence as access_mode FROM ${stats_db_name}.publication_tmp UNION ALL SELECT *,bestlicence as access_mode FROM ${stats_db_name}.software_tmp UNION ALL SELECT *,bestlicence as access_mode FROM ${stats_db_name}.dataset_tmp UNION ALL SELECT *,bestlicence as access_mode FROM ${stats_db_name}.otherresearchproduct_tmp; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_10.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_10.sql deleted file mode 100644 index c9ad0760f..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_10.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS ${stats_db_name}.result_organization; -CREATE TABLE ${stats_db_name}.result_organization AS SELECT substr(r.target, 4) as id, substr(r.source, 4) as organization from ${openaire_db_name}.relation r where r.reltype='resultOrganization'; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_11.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_11.sql deleted file mode 100644 index a2c0d85be..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_11.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS ${stats_db_name}.result_projects; -CREATE TABLE ${stats_db_name}.result_projects AS select pr.result as id, pr.id as project, datediff(p.enddate, p.startdate) as daysfromend from ${stats_db_name}.result r join ${stats_db_name}.project_results pr on r.id=pr.result join ${stats_db_name}.project_tmp p on p.id=pr.id; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_2.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_2.sql deleted file mode 100644 index 94a230d21..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_2.sql +++ /dev/null @@ -1,2 +0,0 @@ --- Views on final tables -CREATE OR REPLACE VIEW ${stats_db_name}.result_datasources as SELECT * FROM ${stats_db_name}.publication_datasources UNION ALL SELECT * FROM ${stats_db_name}.software_datasources UNION ALL SELECT * FROM ${stats_db_name}.dataset_datasources UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_datasources; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_3.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_3.sql deleted file mode 100644 index 3d61dc8fa..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_3.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE OR REPLACE VIEW ${stats_db_name}.result_citations as SELECT * FROM ${stats_db_name}.publication_citations UNION ALL SELECT * FROM ${stats_db_name}.software_citations UNION ALL SELECT * FROM ${stats_db_name}.dataset_citations UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_citations; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_4.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_4.sql deleted file mode 100644 index c27c9964f..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_4.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE OR REPLACE VIEW ${stats_db_name}.result_classifications as SELECT * FROM ${stats_db_name}.publication_classifications UNION ALL SELECT * FROM ${stats_db_name}.software_classifications UNION ALL SELECT * FROM ${stats_db_name}.dataset_classifications UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_classifications; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_5.sql deleted file mode 100644 index 1f81adb73..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_5.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE OR REPLACE VIEW ${stats_db_name}.result_concepts as SELECT * FROM ${stats_db_name}.publication_concepts UNION ALL SELECT * FROM ${stats_db_name}.software_concepts UNION ALL SELECT * FROM ${stats_db_name}.dataset_concepts UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_concepts; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_6.sql deleted file mode 100644 index 2ea29f2f3..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_6.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE OR REPLACE VIEW ${stats_db_name}.result_languages as SELECT * FROM ${stats_db_name}.publication_languages UNION ALL SELECT * FROM ${stats_db_name}.software_languages UNION ALL SELECT * FROM ${stats_db_name}.dataset_languages UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_languages; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_7.sql deleted file mode 100644 index b4e69d413..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_7.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE OR REPLACE VIEW ${stats_db_name}.result_oids as SELECT * FROM ${stats_db_name}.publication_oids UNION ALL SELECT * FROM ${stats_db_name}.software_oids UNION ALL SELECT * FROM ${stats_db_name}.dataset_oids UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_oids; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_8.sql deleted file mode 100644 index 36e753f5c..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_8.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE OR REPLACE VIEW ${stats_db_name}.result_pids as SELECT * FROM ${stats_db_name}.publication_pids UNION ALL SELECT * FROM ${stats_db_name}.software_pids UNION ALL SELECT * FROM ${stats_db_name}.dataset_pids UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_pids; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_9.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_9.sql deleted file mode 100644 index 075ab266b..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8_9.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE OR REPLACE VIEW ${stats_db_name}.result_topics as SELECT * FROM ${stats_db_name}.publication_topics UNION ALL SELECT * FROM ${stats_db_name}.software_topics UNION ALL SELECT * FROM ${stats_db_name}.dataset_topics UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_topics; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql new file mode 100644 index 000000000..114eeeeaa --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql @@ -0,0 +1,33 @@ +------------------------------------------------------------ +------------------------------------------------------------ +-- 9. Datasource table/view and Datasource related tables/views +------------------------------------------------------------ +------------------------------------------------------------ +-- Datasource table creation & update +------------------------------------- +-- Creating and populating temporary datasource table +DROP TABLE IF EXISTS ${hive_db_name}.datasource_tmp; +create table ${hive_db_name}.datasource_tmp(`id` string, `name` string, `type` string, `dateofvalidation` string, `yearofvalidation` string, `harvested` boolean, `piwik_id` int, `latitude` string, `longitude` string, `websiteurl` string, `compatibility` string) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); +insert into ${hive_db_name}.datasource_tmp SELECT substr(d.id, 4) as id, officialname.value as name, datasourcetype.classname as type, dateofvalidation.value as dateofvalidation, date_format(d.dateofvalidation.value,'yyyy') as yearofvalidation, false as harvested, 0 as piwik_id, d.latitude.value as latitude, d.longitude.value as longitude, d.websiteurl.value as websiteurl, d.openairecompatibility.classid as compatibility +from ${hive_source_db_name}.datasource d +WHERE d.datainfo.deletedbyinference=false; + +-- Updating temporary table with everything that is not based on results -> This is done with the following "dual" table. To see if default values are there +-- Creating a temporary dual table that will be removed after the following insert +CREATE TABLE ${hive_db_name}.dual(dummy char(1)); +INSERT INTO ${hive_db_name}.dual values('X'); +INSERT INTO ${hive_db_name}.datasource_tmp (`id`, `name`, `type`, `dateofvalidation`, `yearofvalidation`, `harvested`, `piwik_id`, `latitude`, `longitude`, `websiteurl`, `compatibility`) +SELECT 'other', 'Other', 'Repository', NULL, NULL, false, 0, NULL, NULL, NULL, 'unknown' FROM ${hive_db_name}.dual WHERE 'other' not in (SELECT id FROM ${hive_db_name}.datasource_tmp WHERE name='Unknown Repository'); +DROP TABLE ${hive_db_name}.dual; + +UPDATE ${hive_db_name}.datasource_tmp SET name='Other' where name='Unknown Repository'; +UPDATE ${hive_db_name}.datasource_tmp SET yearofvalidation=null WHERE yearofvalidation='-1'; + +DROP TABLE IF EXISTS ${hive_db_name}.datasource_languages; +CREATE TABLE ${hive_db_name}.datasource_languages AS SELECT substr(d.id, 4) as id, langs.languages as language from ${hive_source_db_name}.datasource d LATERAL VIEW explode(d.odlanguages.value) langs as languages; +DROP TABLE IF EXISTS ${hive_db_name}.datasource_oids; +CREATE TABLE ${hive_db_name}.datasource_oids AS SELECT substr(d.id, 4) as id, oids.ids as oid from ${hive_source_db_name}.datasource d LATERAL VIEW explode(d.originalid) oids as ids; +DROP TABLE IF EXISTS ${hive_db_name}.datasource_organizations; +CREATE TABLE ${hive_db_name}.datasource_organizations AS select substr(r.target, 4) as id, substr(r.source, 4) as organization from ${hive_source_db_name}.relation r where r.reltype='datasourceOrganization'; + +CREATE OR REPLACE VIEW ${hive_db_name}.datasource_results AS SELECT datasource AS id, id AS result FROM ${hive_db_name}.result_datasources; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_1.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_1.sql deleted file mode 100644 index c2274d2e0..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_1.sql +++ /dev/null @@ -1,10 +0,0 @@ ------------------------------------------------------------- ------------------------------------------------------------- --- 9. Datasource table/view and Datasource related tables/views ------------------------------------------------------------- ------------------------------------------------------------- --- Datasource table creation & update -------------------------------------- --- Creating and populating temporary datasource table -DROP TABLE IF EXISTS ${stats_db_name}.datasource_tmp; -create table ${stats_db_name}.datasource_tmp(`id` string, `name` string, `type` string, `dateofvalidation` string, `yearofvalidation` string, `harvested` boolean, `piwik_id` int, `latitude` string, `longitude` string, `websiteurl` string, `compatibility` string) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_2.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_2.sql deleted file mode 100644 index fd89e0d5b..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_2.sql +++ /dev/null @@ -1,3 +0,0 @@ -insert into ${stats_db_name}.datasource_tmp SELECT substr(d.id, 4) as id, officialname.value as name, datasourcetype.classname as type, dateofvalidation.value as dateofvalidation, date_format(d.dateofvalidation.value,'yyyy') as yearofvalidation, false as harvested, 0 as piwik_id, d.latitude.value as latitude, d.longitude.value as longitude, d.websiteurl.value as websiteurl, d.openairecompatibility.classid as compatibility -from ${openaire_db_name}.datasource d -WHERE d.datainfo.deletedbyinference=false; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_3.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_3.sql deleted file mode 100644 index 2266bc724..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_3.sql +++ /dev/null @@ -1,7 +0,0 @@ --- Updating temporary table with everything that is not based on results -> This is done with the following "dual" table. To see if default values are there --- Creating a temporary dual table that will be removed after the following insert -CREATE TABLE ${stats_db_name}.dual(dummy char(1)); -INSERT INTO ${stats_db_name}.dual values('X'); -INSERT INTO ${stats_db_name}.datasource_tmp (`id`, `name`, `type`, `dateofvalidation`, `yearofvalidation`, `harvested`, `piwik_id`, `latitude`, `longitude`, `websiteurl`, `compatibility`) -SELECT 'other', 'Other', 'Repository', NULL, NULL, false, 0, NULL, NULL, NULL, 'unknown' FROM ${stats_db_name}.dual WHERE 'other' not in (SELECT id FROM ${stats_db_name}.datasource_tmp WHERE name='Unknown Repository'); -DROP TABLE ${stats_db_name}.dual; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_4.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_4.sql deleted file mode 100644 index dd5f9b314..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_4.sql +++ /dev/null @@ -1 +0,0 @@ -UPDATE ${stats_db_name}.datasource_tmp SET name='Other' where name='Unknown Repository'; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_5.sql deleted file mode 100644 index 77ee81b04..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_5.sql +++ /dev/null @@ -1 +0,0 @@ -UPDATE ${stats_db_name}.datasource_tmp SET yearofvalidation=null WHERE yearofvalidation='-1'; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_7.sql deleted file mode 100644 index 194633cb7..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_7.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS ${stats_db_name}.datasource_oids; -CREATE TABLE ${stats_db_name}.datasource_oids AS SELECT substr(d.id, 4) as id, oids.ids as oid from ${openaire_db_name}.datasource d LATERAL VIEW explode(d.originalid) oids as ids; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_8.sql deleted file mode 100644 index c25858d2c..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_8.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS ${stats_db_name}.datasource_organizations; -CREATE TABLE ${stats_db_name}.datasource_organizations AS select substr(r.target, 4) as id, substr(r.source, 4) as organization from ${openaire_db_name}.relation r where r.reltype='datasourceOrganization'; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_9.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_9.sql deleted file mode 100644 index ec4aa0569..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_9.sql +++ /dev/null @@ -1 +0,0 @@ -CREATE OR REPLACE VIEW ${stats_db_name}.datasource_results AS SELECT datasource AS id, id AS result FROM ${stats_db_name}.result_datasources; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index efc301573..269525c0f 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml @@ -1,25 +1,12 @@ - + - stats_db_name - the target stats database name + hive_db_name + the target hive database name - openaire_db_name - the original graph database name - - - external_stats_db_name - stats_ext - the external stats that should be added since they are not included in the graph database - - - hiveMetastoreUris - hive server metastore URIs - - - hiveJdbcUrl - hive server jdbc url + hive_source_db_name + the source hive database name hive_timeout @@ -42,892 +29,271 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + - ${hiveJdbcUrl} + ${jobTracker} + ${nameNode} + + + hive.metastore.uris + ${hive_metastore_uris} + + + ${hive_jdbc_url}/${hive_db_name} - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} + hive_db_name=${hive_db_name} + hive_source_db_name=${hive_source_db_name} - + - - + - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} + ${jobTracker} + ${nameNode} + + + hive.metastore.uris + ${hive_metastore_uris} + + + ${hive_jdbc_url}/${hive_db_name} + + hive_db_name=${hive_db_name} + hive_source_db_name=${hive_source_db_name} - - ${hiveJdbcUrl} + ${jobTracker} + ${nameNode} + + + hive.metastore.uris + ${hive_metastore_uris} + + + ${hive_jdbc_url}/${hive_db_name} - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} + hive_db_name=${hive_db_name} + hive_source_db_name=${hive_source_db_name} - + + - - + - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} + ${jobTracker} + ${nameNode} + + + hive.metastore.uris + ${hive_metastore_uris} + + + hive.variable.substitute + true + + + ${hive_jdbc_url}/${hive_db_name} + + hive_db_name=${hive_db_name} + hive_source_db_name=${hive_source_db_name} - + - - + - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} + ${jobTracker} + ${nameNode} + + + hive.metastore.uris + ${hive_metastore_uris} + + + ${hive_jdbc_url}/${hive_db_name} + + hive_db_name=${hive_db_name} + hive_source_db_name=${hive_source_db_name} - + - - + - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} + ${jobTracker} + ${nameNode} + + + hive.metastore.uris + ${hive_metastore_uris} + + + ${hive_jdbc_url}/${hive_db_name} + + hive_db_name=${hive_db_name} + hive_source_db_name=${hive_source_db_name} - + - - + - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} + ${jobTracker} + ${nameNode} + + + hive.metastore.uris + ${hive_metastore_uris} + + + ${hive_jdbc_url}/${hive_db_name} + + hive_db_name=${hive_db_name} + hive_source_db_name=${hive_source_db_name} - + - - + - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} + ${jobTracker} + ${nameNode} + + + hive.metastore.uris + ${hive_metastore_uris} + + + ${hive_jdbc_url}/${hive_db_name} + + hive_db_name=${hive_db_name} + hive_source_db_name=${hive_source_db_name} - + - - + - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} + ${jobTracker} + ${nameNode} + + + hive.metastore.uris + ${hive_metastore_uris} + + + ${hive_jdbc_url}/${hive_db_name} + + hive_db_name=${hive_db_name} + hive_source_db_name=${hive_source_db_name} - + - - + - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} + ${jobTracker} + ${nameNode} + + + hive.metastore.uris + ${hive_metastore_uris} + + + ${hive_jdbc_url}/${hive_db_name} + + hive_db_name=${hive_db_name} + hive_source_db_name=${hive_source_db_name} - - - ${hiveJdbcUrl} + + ${jobTracker} + ${nameNode} + + + hive.metastore.uris + ${hive_metastore_uris} + + + ${hive_jdbc_url}/${hive_db_name} - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - external_stats_db_name=${external_stats_db_name} + hive_db_name=${hive_db_name} + hive_source_db_name=${hive_source_db_name} - + - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - - - - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} + + + ${jobTracker} + ${nameNode} + + + hive.metastore.uris + ${hive_metastore_uris} + + + ${hive_jdbc_url}/${hive_db_name} + + hive_db_name=${hive_db_name} + hive_source_db_name=${hive_source_db_name} - - ${hiveJdbcUrl} + ${jobTracker} + ${nameNode} + + + hive.metastore.uris + ${hive_metastore_uris} + + + ${hive_jdbc_url}/${hive_db_name} - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} + hive_db_name=${hive_db_name} + hive_source_db_name=${hive_source_db_name} - - - ${hiveJdbcUrl} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - + + ${jobTracker} + ${nameNode} + + impala-shell.sh + ${hive_db_name} + impala-stats.sql + /user/giorgos.alexiou/graph/oozie_app/impala-stats.sql + impala-shell.sh + + - + From 657a40536b68a027493e85e754ebda4e98a3f928 Mon Sep 17 00:00:00 2001 From: Spyros Zoupanos Date: Sat, 13 Jun 2020 13:35:53 +0300 Subject: [PATCH 080/100] Corrections by Spyros: Scipt cleanup, corrections and re-arrangement --- .../graph/stats/oozie_app/config-default.xml | 22 +- .../oa/graph/stats/oozie_app/impala-shell.sh | 12 +- .../oa/graph/stats/oozie_app/impala-stats.sql | 77 ----- .../graph/stats/oozie_app/scripts/step1.sql | 6 +- .../graph/stats/oozie_app/scripts/step10.sql | 9 +- .../graph/stats/oozie_app/scripts/step11.sql | 10 +- .../graph/stats/oozie_app/scripts/step12.sql | 46 ++- .../graph/stats/oozie_app/scripts/step13.sql | 45 ++- .../graph/stats/oozie_app/scripts/step14.sql | 47 ++++ .../graph/stats/oozie_app/scripts/step2.sql | 30 +- .../graph/stats/oozie_app/scripts/step3.sql | 4 +- .../graph/stats/oozie_app/scripts/step4.sql | 40 ++- .../graph/stats/oozie_app/scripts/step5.sql | 40 +-- .../graph/stats/oozie_app/scripts/step6.sql | 44 +-- .../graph/stats/oozie_app/scripts/step7.sql | 21 +- .../graph/stats/oozie_app/scripts/step8.sql | 33 ++- .../graph/stats/oozie_app/scripts/step9.sql | 43 +-- .../graph/stats/oozie_app/scripts/step9_6.sql | 2 - .../dhp/oa/graph/stats/oozie_app/workflow.xml | 266 ++++++------------ 19 files changed, 350 insertions(+), 447 deletions(-) delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/impala-stats.sql create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_6.sql diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/config-default.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/config-default.xml index 74dbb6f77..2cd53a37b 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/config-default.xml @@ -1,12 +1,11 @@ - jobTracker - yarnRM + ${jobTracker} nameNode - hdfs://nameservice1 + ${nameNode} oozie.use.system.libpath @@ -16,16 +15,6 @@ oozie.action.sharelib.for.spark spark2 - - hive_db_name - - stats_wf_db_galexiou_oozie_beta - - - hive_source_db_name - - openaire2 - hive_metastore_uris thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 @@ -34,5 +23,8 @@ hive_jdbc_url jdbc:hive2://iis-cdh5-test-m3.ocean.icm.edu.pl:10000 - - + + oozie.wf.workflow.notification.url + {serviceUrl}/v1/oozieNotification/jobUpdate?jobId=$jobId%26status=$status + + \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/impala-shell.sh b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/impala-shell.sh index a5a58493b..70112dc7b 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/impala-shell.sh +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/impala-shell.sh @@ -1,4 +1,4 @@ -export PYTHON_EGG_CACHE=/home/giorgos.alexiou/.python-eggs +export PYTHON_EGG_CACHE=/home/$(whoami)/.python-eggs export link_folder=/tmp/impala-shell-python-egg-cache-$(whoami) if ! [ -L $link_folder ] then @@ -6,11 +6,13 @@ then ln -sfn ${PYTHON_EGG_CACHE}${link_folder} ${link_folder} fi -echo "getting file from " $3 - +echo "Getting file from " $3 hdfs dfs -copyToLocal $3 -echo "running impala shell now" +echo "Running impala shell make the new database visible" +impala-shell -q "INVALIDATE METADATA;" + +echo "Running impala shell to compute new table stats" impala-shell -d $1 -f $2 -echo "impala shell finished" +echo "Impala shell finished" rm $2 diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/impala-stats.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/impala-stats.sql deleted file mode 100644 index b8833352d..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/impala-stats.sql +++ /dev/null @@ -1,77 +0,0 @@ - -INVALIDATE METADATA; --- CREATE TABLE chart_country_year AS SELECT org.country AS country, r.year AS year, count(distinct r.id) AS publications FROM result r, result_datasources rd, datasource d, datasource_organizations dor, organization org WHERE r.id=rd.id AND rd.datasource=d.id AND rd.datasource=dor.id AND dor.organization=org.id AND r.type='publication' AND r.bestlicence='Open Access' and r.year>='1990' AND r.year<=CAST(date_part('year',now()) AS STRING) group by org.country, r.year ORDER BY org.country, r.year; --- CREATE TABLE chart_country_datasources AS SELECT org.country AS country, d.name AS datasource, count(distinct r.id) AS publications FROM result r, result_datasources rd, datasource d, datasource_organizations dor, organization org WHERE r.id=rd.id AND rd.datasource=d.id AND d.id=dor.id AND dor.organization=org.id AND r.type='publication' AND r.bestlicence='Open Access' GROUP BY org.country, d.name ORDER BY org.country, publications DESC; --- CREATE TABLE chart_country_type AS SELECT org.country AS country, rc.type AS type, count(distinct r.id) AS publications FROM result r, result_datasources rd, datasource d, result_classifications rc, datasource_organizations dor, organization org WHERE r.id=rd.id AND r.id=rc.id AND rd.datasource=d.id AND rd.datasource=dor.id AND dor.organization=org.id AND r.type='publication' AND r.bestlicence='Open Access' GROUP BY org.country, rc.type; --- CREATE TABLE chart_country_fp7 AS SELECT org.country AS country, r.year AS year, count(distinct r.id) AS publications FROM result r, result_datasources rd, datasource d, datasource_organizations dor, organization org, project_results pr, project p WHERE r.id=rd.id AND rd.datasource=d.id AND rd.datasource=dor.id AND dor.organization=org.id AND r.id=pr.result AND pr.id=p.id AND p.funding_lvl0='FP7' AND r.type='publication' AND r.year>='1990' AND r.year<=CAST(date_part('year',now()) AS STRING) GROUP BY org.country, r.year ORDER BY org.country, r.year; --- CREATE TABLE chart_datasource_type AS SELECT rd.datasource, rc.type, count(distinct rd.id) FROM result_datasources rd, result_classifications rc WHERE rd.id=rc.id GROUP BY rd.datasource, rc.type; --- CREATE TABLE chart_datasource_year AS SELECT rd.datasource, r.year, count(distinct rd.id) FROM result r, result_datasources rd WHERE rd.id=r.id GROUP By rd.datasource, r.year; --- CREATE TABLE chart_datasource_funders AS SELECT rd.datasource, p.funder, count(distinct rd.id) FROM result_datasources rd, project p, project_results pr WHERE p.id=pr.id AND pr.result=rd.id GROUP BY rd.datasource, p.funder; --- CREATE TABLE chart_datasource_projects_pubs AS SELECT rd.datasource, p.title, count(distinct rd.id) FROM result_datasources rd, project p, project_results pr, result r WHERE p.id=pr.id AND pr.result=rd.id AND pr.result=r.id AND r.type='publication' GROUP BY rd.datasource, p.title; --- CREATE TABLE chart_datasource_projects_data AS SELECT rd.datasource, p.title, count(distinct rd.id) FROM result_datasources rd, project p, project_results pr, result r WHERE p.id=pr.id AND pr.result=rd.id AND pr.result=r.id and r.type='dataset' GROUP BY rd.datasource, p.title; --- CREATE TABLE chart_project_year AS SELECT p.id, r.year, count( distinct r.id) FROM result r, project_results pr, project p WHERE r.id=pr.result AND p.id=pr.id AND r.year>=p.start_year GROUP BY p.id, r.year; --- CREATE TABLE chart_project_license AS SELECT pr.id, r.bestlicence, count(distinct r.id) FROM result r, project_results pr WHERE r.id=pr.result AND r.type='publication' GROUP BY pr.id, r.bestlicence; --- CREATE TABLE chart_project_repos AS SELECT pr.id, d.name, count (distinct r.id) FROM result r, project_results pr, datasource d, datasource_results dr WHERE r.id=dr.result AND d.id=dr.id AND r.id=pr.result AND r.type='publication' GROUP BY pr.id, d.name; --- CREATE TABLE rd_distinct AS SELECT DISTINCT * FROM result_datasources; --- DROP VIEW result_datasources; --- ALTER TABLE rd_distinct RENAME TO result_datasources; --- COMPUTE STATS chart_country_datasources; --- COMPUTE STATS chart_country_fp7; --- COMPUTE STATS chart_country_type; --- COMPUTE STATS chart_country_year; --- COMPUTE STATS chart_datasource_funders; --- COMPUTE STATS chart_datasource_projects_data; --- COMPUTE STATS chart_datasource_projects_pubs; --- COMPUTE STATS chart_datasource_type; --- COMPUTE STATS chart_datasource_year; --- COMPUTE STATS chart_project_license; --- COMPUTE STATS chart_project_repos; --- COMPUTE STATS chart_project_year; -COMPUTE STATS dataset; -COMPUTE STATS dataset_citations; -COMPUTE STATS dataset_classifications; -COMPUTE STATS dataset_concepts; -COMPUTE STATS dataset_datasources; -COMPUTE STATS dataset_languages; -COMPUTE STATS dataset_oids; -COMPUTE STATS dataset_pids; -COMPUTE STATS dataset_topics; -COMPUTE STATS datasource; -COMPUTE STATS datasource_languages; -COMPUTE STATS datasource_oids; -COMPUTE STATS datasource_organizations; -COMPUTE STATS numbers_country; -COMPUTE STATS organization; -COMPUTE STATS otherresearchproduct; -COMPUTE STATS otherresearchproduct_citations; -COMPUTE STATS otherresearchproduct_classifications; -COMPUTE STATS otherresearchproduct_concepts; -COMPUTE STATS otherresearchproduct_datasources; -COMPUTE STATS otherresearchproduct_languages; -COMPUTE STATS otherresearchproduct_oids; -COMPUTE STATS otherresearchproduct_pids; -COMPUTE STATS otherresearchproduct_topics; -COMPUTE STATS project; -COMPUTE STATS project_oids; -COMPUTE STATS project_organizations; -COMPUTE STATS project_results; -COMPUTE STATS publication; -COMPUTE STATS publication_citations; -COMPUTE STATS publication_classifications; -COMPUTE STATS publication_concepts; -COMPUTE STATS publication_datasources; -COMPUTE STATS publication_languages; -COMPUTE STATS publication_oids; -COMPUTE STATS publication_pids; -COMPUTE STATS publication_topics; -COMPUTE STATS result_organization; -COMPUTE STATS result_projects; -COMPUTE STATS software; -COMPUTE STATS software_citations; -COMPUTE STATS software_classifications; -COMPUTE STATS software_concepts; -COMPUTE STATS software_datasources; -COMPUTE STATS software_languages; -COMPUTE STATS software_oids; -COMPUTE STATS software_pids; -COMPUTE STATS software_topic; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql index 1290155e6..92e3be7a5 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql @@ -1,11 +1,11 @@ -- DROP database if EXISTS ${hive_db_name} cascade; -- CREATE database ${hive_db_name}; --- +-- -- CREATE TABLE ${hive_db_name}.Persons ( -- PersonID int, -- LastName varchar(255)); -- -- INSERT INTO ${hive_db_name}.Persons VALUES (1, "test_db_spyros_rec_111"); -drop database if exists ${hive_db_name} cascade; -create database ${hive_db_name}; +DROP database IF EXISTS ${stats_db_name} CASCADE; +CREATE database ${stats_db_name}; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql index e51892771..792c8cbdd 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql @@ -3,8 +3,9 @@ -- Organization table/view and Organization related tables/views ---------------------------------------------------------------- ---------------------------------------------------------------- -DROP TABLE IF EXISTS ${hive_db_name}.organization; -CREATE TABLE ${hive_db_name}.organization AS SELECT substr(o.id, 4) as id, o.legalname.value as name, o.country.classid as country from ${hive_source_db_name}.organization o WHERE o.datainfo.deletedbyinference=false; +DROP TABLE IF EXISTS ${stats_db_name}.organization; +CREATE TABLE ${stats_db_name}.organization AS SELECT substr(o.id, 4) as id, o.legalname.value AS name, o.country.classid AS country FROM ${openaire_db_name}.organization o WHERE o.datainfo.deletedbyinference=false; -CREATE OR REPLACE VIEW ${hive_db_name}.organization_datasources AS SELECT organization AS id, id AS datasource FROM ${hive_db_name}.datasource_organizations; -CREATE OR REPLACE VIEW ${hive_db_name}.organization_projects AS SELECT id AS project, organization as id FROM ${hive_db_name}.project_organizations; +CREATE OR REPLACE VIEW ${stats_db_name}.organization_datasources AS SELECT organization AS id, id AS datasource FROM ${stats_db_name}.datasource_organizations; + +CREATE OR REPLACE VIEW ${stats_db_name}.organization_projects AS SELECT id AS project, organization as id FROM ${stats_db_name}.project_organizations; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql index e586f0cd6..145be77b0 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql @@ -3,8 +3,8 @@ -- Tables/views from external tables/views (Fundref, Country, CountyGDP, roarmap, rndexpediture) ------------------------------------------------------------------------------------------------ ------------------------------------------------------------------------------------------------ -CREATE OR REPLACE VIEW ${hive_db_name}.fundref AS SELECT * FROM stats_ext.fundref; -CREATE OR REPLACE VIEW ${hive_db_name}.country AS SELECT * FROM stats_ext.country; -CREATE OR REPLACE VIEW ${hive_db_name}.countrygdp AS SELECT * FROM stats_ext.countrygdp; -CREATE OR REPLACE VIEW ${hive_db_name}.roarmap AS SELECT * FROM stats_ext.roarmap; -CREATE OR REPLACE VIEW ${hive_db_name}.rndexpediture AS SELECT * FROM stats_ext.rndexpediture; +CREATE OR REPLACE VIEW ${stats_db_name}.fundref AS SELECT * FROM ${external_stats_db_name}.fundref; +CREATE OR REPLACE VIEW ${stats_db_name}.country AS SELECT * FROM ${external_stats_db_name}.country; +CREATE OR REPLACE VIEW ${stats_db_name}.countrygdp AS SELECT * FROM ${external_stats_db_name}.countrygdp; +CREATE OR REPLACE VIEW ${stats_db_name}.roarmap AS SELECT * FROM ${external_stats_db_name}.roarmap; +CREATE OR REPLACE VIEW ${stats_db_name}.rndexpediture AS SELECT * FROM ${external_stats_db_name}.rndexpediture; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql index ba0d52aca..c18e2e1f8 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql @@ -3,42 +3,40 @@ ---------------------------- --Datasource temporary table updates -UPDATE ${hive_db_name}.datasource_tmp set harvested ='true' WHERE datasource_tmp.id IN (SELECT DISTINCT d.id FROM ${hive_db_name}.datasource_tmp d, ${hive_db_name}.result_datasources rd where d.id=rd.datasource); +UPDATE ${stats_db_name}.datasource_tmp SET harvested='true' WHERE datasource_tmp.id IN (SELECT DISTINCT d.id FROM ${stats_db_name}.datasource_tmp d, ${stats_db_name}.result_datasources rd WHERE d.id=rd.datasource); -- Project temporary table update and final project table creation with final updates that can not be applied to ORC tables --- UPDATE ${hive_db_name}.project_tmp SET haspubs='yes' WHERE project_tmp.id IN (SELECT pr.id FROM ${hive_db_name}.project_results pr, ${hive_db_name}.result r WHERE pr.result=r.id AND r.type='publication'); +UPDATE ${stats_db_name}.project_tmp SET haspubs='yes' WHERE project_tmp.id IN (SELECT pr.id FROM ${stats_db_name}.project_results pr, ${stats_db_name}.result r WHERE pr.result=r.id AND r.type='publication'); - -CREATE TABLE ${hive_db_name}.project stored as parquet as -SELECT p.id , p.acronym, p.title, p.funder, p.funding_lvl0, p.funding_lvl1, p.funding_lvl2, p.ec39, p.type, p.startdate, p.enddate, p.start_year, p.end_year, p.duration, -CASE WHEN prr1.id IS NULL THEN 'no' ELSE 'yes' END as haspubs, -CASE WHEN prr1.id IS NULL THEN 0 ELSE prr1.np END as numpubs, -CASE WHEN prr2.id IS NULL THEN 0 ELSE prr2.daysForlastPub END as daysforlastpub, -CASE WHEN prr2.id IS NULL THEN 0 ELSE prr2.dp END as delayedpubs, +DROP TABLE IF EXISTS ${stats_db_name}.project; +CREATE TABLE ${stats_db_name}.project stored as parquet as +SELECT p.id , p.acronym, p.title, p.funder, p.funding_lvl0, p.funding_lvl1, p.funding_lvl2, p.ec39, p.type, p.startdate, p.enddate, p.start_year, p.end_year, p.duration, +CASE WHEN prr1.id IS NULL THEN 'no' ELSE 'yes' END AS haspubs, +CASE WHEN prr1.id IS NULL THEN 0 ELSE prr1.np END AS numpubs, +CASE WHEN prr2.id IS NULL THEN 0 ELSE prr2.daysForlastPub END AS daysforlastpub, +CASE WHEN prr2.id IS NULL THEN 0 ELSE prr2.dp END AS delayedpubs, p.callidentifier, p.code -FROM ${hive_db_name}.project_tmp p +FROM ${stats_db_name}.project_tmp p LEFT JOIN (SELECT pr.id, count(distinct pr.result) AS np - FROM ${hive_db_name}.project_results pr INNER JOIN ${hive_db_name}.result r ON pr.result=r.id - WHERE r.type='publication' + FROM ${stats_db_name}.project_results pr INNER JOIN ${stats_db_name}.result r ON pr.result=r.id + WHERE r.type='publication' GROUP BY pr.id) AS prr1 on prr1.id = p.id -LEFT JOIN (SELECT pp.id, max(datediff(to_date(r.date), to_date(pp.enddate)) ) as daysForlastPub , count(distinct r.id) as dp - FROM ${hive_db_name}.project_tmp pp, ${hive_db_name}.project_results pr, ${hive_db_name}.result r - WHERE pp.id=pr.id AND pr.result=r.id AND r.type='publication' AND datediff(to_date(r.date), to_date(pp.enddate)) > 0 +LEFT JOIN (SELECT pp.id, max(datediff(to_date(r.date), to_date(pp.enddate)) ) AS daysForlastPub , count(distinct r.id) AS dp + FROM ${stats_db_name}.project_tmp pp, ${stats_db_name}.project_results pr, ${stats_db_name}.result r + WHERE pp.id=pr.id AND pr.result=r.id AND r.type='publication' AND datediff(to_date(r.date), to_date(pp.enddate)) > 0 GROUP BY pp.id) AS prr2 - on prr2.id = p.id; - - + ON prr2.id = p.id; + -- Publication temporary table updates -UPDATE ${hive_db_name}.publication_tmp SET delayed = 'yes' WHERE publication_tmp.id IN (SELECT distinct r.id FROM ${hive_db_name}.result r, ${hive_db_name}.project_results pr, ${hive_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); +UPDATE ${stats_db_name}.publication_tmp SET delayed = 'yes' WHERE publication_tmp.id IN (SELECT distinct r.id FROM stats_wf_db_obs.result r, ${stats_db_name}.project_results pr, ${stats_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); -- Dataset temporary table updates -UPDATE ${hive_db_name}.dataset_tmp SET delayed = 'yes' WHERE dataset_tmp.id IN (SELECT distinct r.id FROM ${hive_db_name}.result r, ${hive_db_name}.project_results pr, ${hive_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); +UPDATE ${stats_db_name}.dataset_tmp SET delayed = 'yes' WHERE dataset_tmp.id IN (SELECT distinct r.id FROM stats_wf_db_obs.result r, ${stats_db_name}.project_results pr, ${stats_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); -- Software temporary table updates -UPDATE ${hive_db_name}.software_tmp SET delayed = 'yes' WHERE software_tmp.id IN (SELECT distinct r.id FROM ${hive_db_name}.result r, ${hive_db_name}.project_results pr, ${hive_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); +UPDATE ${stats_db_name}.software_tmp SET delayed = 'yes' WHERE software_tmp.id IN (SELECT distinct r.id FROM ${stats_db_name}.result r, ${stats_db_name}.project_results pr, ${stats_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); -- Oherresearchproduct temporary table updates -UPDATE ${hive_db_name}.otherresearchproduct_tmp SET delayed = 'yes' WHERE otherresearchproduct_tmp.id IN (SELECT distinct r.id FROM ${hive_db_name}.result r, ${hive_db_name}.project_results pr, ${hive_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); +UPDATE ${stats_db_name}.otherresearchproduct_tmp SET delayed = 'yes' WHERE otherresearchproduct_tmp.id IN (SELECT distinct r.id FROM ${stats_db_name}.result r, ${stats_db_name}.project_results pr, ${stats_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); - -CREATE OR REPLACE VIEW ${hive_db_name}.project_results_publication AS SELECT result_projects.id AS result, result_projects.project AS project_results, result.date as resultdate, project.enddate as projectenddate, result_projects.daysfromend as daysfromend FROM ${hive_db_name}.result_projects, ${hive_db_name}.result, ${hive_db_name}.project WHERE result_projects.id=result.id and result.type='publication' and project.id=result_projects.project; +CREATE OR REPLACE VIEW ${stats_db_name}.project_results_publication AS SELECT result_projects.id AS result, result_projects.project AS project_results, result.date as resultdate, project.enddate as projectenddate, result_projects.daysfromend AS daysfromend FROM ${stats_db_name}.result_projects, ${stats_db_name}.result, ${stats_db_name}.project WHERE result_projects.id=result.id AND result.type='publication' AND project.id=result_projects.project; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql index e28424dec..25439852e 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql @@ -1,39 +1,38 @@ ------------------------------------------------------------------------------------------------------ -- Creating parquet tables from the updated temporary tables and removing unnecessary temporary tables ------------------------------------------------------------------------------------------------------ -DROP TABLE IF EXISTS ${hive_db_name}.datasource; -CREATE TABLE ${hive_db_name}.datasource stored as parquet as select * from ${hive_db_name}.datasource_tmp; -DROP TABLE IF EXISTS ${hive_db_name}.publication; -CREATE TABLE ${hive_db_name}.publication stored as parquet as select * from ${hive_db_name}.publication_tmp; +DROP TABLE IF EXISTS ${stats_db_name}.datasource; +CREATE TABLE ${stats_db_name}.datasource stored AS parquet AS SELECT * FROM ${stats_db_name}.datasource_tmp; -DROP TABLE IF EXISTS ${hive_db_name}.dataset; -CREATE TABLE ${hive_db_name}.dataset stored as parquet as select * from ${hive_db_name}.dataset_tmp; +DROP TABLE IF EXISTS ${stats_db_name}.publication; +CREATE TABLE ${stats_db_name}.publication stored AS parquet AS SELECT * FROM ${stats_db_name}.publication_tmp; -DROP TABLE IF EXISTS ${hive_db_name}.software; -CREATE TABLE ${hive_db_name}.software stored as parquet as select * from ${hive_db_name}.software_tmp; +DROP TABLE IF EXISTS ${stats_db_name}.dataset; +CREATE TABLE ${stats_db_name}.dataset stored AS parquet AS SELECT * FROM ${stats_db_name}.dataset_tmp; -DROP TABLE IF EXISTS ${hive_db_name}.otherresearchproduct; -CREATE TABLE ${hive_db_name}.otherresearchproduct stored as parquet as select * from ${hive_db_name}.otherresearchproduct_tmp; +DROP TABLE IF EXISTS ${stats_db_name}.software; +CREATE TABLE ${stats_db_name}.software stored AS parquet AS SELECT * FROM ${stats_db_name}.software_tmp; -DROP TABLE ${hive_db_name}.project_tmp; -DROP TABLE ${hive_db_name}.datasource_tmp; -DROP TABLE ${hive_db_name}.publication_tmp; -DROP TABLE ${hive_db_name}.dataset_tmp; -DROP TABLE ${hive_db_name}.software_tmp; -DROP TABLE ${hive_db_name}.otherresearchproduct_tmp; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct; +CREATE TABLE ${stats_db_name}.otherresearchproduct stored AS parquet AS SELECT * FROM ${stats_db_name}.otherresearchproduct_tmp; + +DROP TABLE ${stats_db_name}.project_tmp; +DROP TABLE ${stats_db_name}.datasource_tmp; +DROP TABLE ${stats_db_name}.publication_tmp; +DROP TABLE ${stats_db_name}.dataset_tmp; +DROP TABLE ${stats_db_name}.software_tmp; +DROP TABLE ${stats_db_name}.otherresearchproduct_tmp; ---------------------------------------------- -- Re-creating views from final parquet tables --------------------------------------------- -- Result -CREATE OR REPLACE VIEW ${hive_db_name}.result as SELECT *, bestlicence as access_mode FROM ${hive_db_name}.publication UNION ALL SELECT *, bestlicence as access_mode FROM ${hive_db_name}.software UNION ALL SELECT *, bestlicence as access_mode FROM ${hive_db_name}.dataset UNION ALL SELECT *, bestlicence as access_mode FROM ${hive_db_name}.otherresearchproduct; +CREATE OR REPLACE VIEW ${stats_db_name}.result AS SELECT *, bestlicence AS access_mode FROM ${stats_db_name}.publication UNION ALL SELECT *, bestlicence as access_mode FROM ${stats_db_name}.software UNION ALL SELECT *, bestlicence AS access_mode FROM ${stats_db_name}.dataset UNION ALL SELECT *, bestlicence AS access_mode FROM ${stats_db_name}.otherresearchproduct; --- cleanup -drop view if exists ${hive_db_name}.delayedpubs; -drop view if exists ${hive_db_name}.project_pub_count; -drop view if exists ${hive_db_name}.delayedpubs; -drop view if exists ${hive_db_name}.project_results_publication; - CREATE TABLE ${hive_db_name}.numbers_country AS SELECT org.country AS country, count(distinct rd.datasource) AS datasources, count(distinct r.id) AS publications FROM ${hive_db_name}.result r, ${hive_db_name}.result_datasources rd, ${hive_db_name}.datasource d, ${hive_db_name}.datasource_organizations dor, ${hive_db_name}.organization org WHERE r.id=rd.id AND rd.datasource=d.id AND d.id=dor.id AND dor.organization=org.id AND r.type='publication' AND r.bestlicence='Open Access' GROUP BY org.country; +------------------------------------------------------------------------------- +-- To see with Antonis if the following is needed and where it should be placed +------------------------------------------------------------------------------- +CREATE TABLE ${stats_db_name}.numbers_country AS SELECT org.country AS country, count(distinct rd.datasource) AS datasources, count(distinct r.id) AS publications FROM ${stats_db_name}.result r, ${stats_db_name}.result_datasources rd, ${stats_db_name}.datasource d, ${stats_db_name}.datasource_organizations dor, ${stats_db_name}.organization org WHERE r.id=rd.id AND rd.datasource=d.id AND d.id=dor.id AND dor.organization=org.id AND r.type='publication' AND r.bestlicence='Open Access' GROUP BY org.country; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql new file mode 100644 index 000000000..06d58fd58 --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql @@ -0,0 +1,47 @@ +COMPUTE STATS dataset; +COMPUTE STATS dataset_citations; +COMPUTE STATS dataset_classifications; +COMPUTE STATS dataset_concepts; +COMPUTE STATS dataset_datasources; +COMPUTE STATS dataset_languages; +COMPUTE STATS dataset_oids; +COMPUTE STATS dataset_pids; +COMPUTE STATS dataset_topics; +COMPUTE STATS datasource; +COMPUTE STATS datasource_languages; +COMPUTE STATS datasource_oids; +COMPUTE STATS datasource_organizations; +COMPUTE STATS numbers_country; +COMPUTE STATS organization; +COMPUTE STATS otherresearchproduct; +COMPUTE STATS otherresearchproduct_citations; +COMPUTE STATS otherresearchproduct_classifications; +COMPUTE STATS otherresearchproduct_concepts; +COMPUTE STATS otherresearchproduct_datasources; +COMPUTE STATS otherresearchproduct_languages; +COMPUTE STATS otherresearchproduct_oids; +COMPUTE STATS otherresearchproduct_pids; +COMPUTE STATS otherresearchproduct_topics; +COMPUTE STATS project; +COMPUTE STATS project_oids; +COMPUTE STATS project_organizations; +COMPUTE STATS project_results; +COMPUTE STATS publication; +COMPUTE STATS publication_citations; +COMPUTE STATS publication_classifications; +COMPUTE STATS publication_concepts; +COMPUTE STATS publication_datasources; +COMPUTE STATS publication_languages; +COMPUTE STATS publication_oids; +COMPUTE STATS publication_pids; +COMPUTE STATS publication_topics; +COMPUTE STATS result_organization; +COMPUTE STATS result_projects; +COMPUTE STATS software; +COMPUTE STATS software_citations; +COMPUTE STATS software_classifications; +COMPUTE STATS software_concepts; +COMPUTE STATS software_datasources; +COMPUTE STATS software_languages; +COMPUTE STATS software_oids; +COMPUTE STATS software_pids; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql index cddf83e2b..7e0a07184 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql @@ -5,28 +5,28 @@ -------------------------------------------------------------- -- Publication temporary table -DROP TABLE IF EXISTS ${hive_db_name}.publication_tmp; -CREATE TABLE ${hive_db_name}.publication_tmp (id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); +DROP TABLE IF EXISTS ${stats_db_name}.publication_tmp; --- The following fails --- -INSERT INTO ${hive_db_name}.publication_tmp SELECT substr(p.id, 4) as id, p.title[0].value as title, p.publisher.value as publisher, p.journal.name as journal, +CREATE TABLE ${stats_db_name}.publication_tmp (id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); + +INSERT INTO ${stats_db_name}.publication_tmp SELECT substr(p.id, 4) as id, p.title[0].value as title, p.publisher.value as publisher, p.journal.name as journal , p.dateofacceptance.value as date, date_format(p.dateofacceptance.value,'yyyy') as year, p.bestaccessright.classname as bestlicence, p.embargoenddate.value as embargo_end_date, false as delayed, size(p.author) as authors , concat_ws('\u003B',p.source.value) as source, case when size(p.description) > 0 then true else false end as abstract, 'publication' as type -from ${hive_source_db_name}.publication p +from ${openaire_db_name}.publication p where p.datainfo.deletedbyinference=false; +CREATE TABLE ${stats_db_name}.publication_classifications AS SELECT substr(p.id, 4) as id, instancetype.classname as type from ${openaire_db_name}.publication p LATERAL VIEW explode(p.instance.instancetype) instances as instancetype; -CREATE TABLE ${hive_db_name}.publication_classifications AS SELECT substr(p.id, 4) as id, instancetype.classname as type from ${hive_source_db_name}.publication p LATERAL VIEW explode(p.instance.instancetype) instances as instancetype; -CREATE TABLE ${hive_db_name}.publication_concepts AS SELECT substr(p.id, 4) as id, contexts.context.id as concept from ${hive_source_db_name}.publication p LATERAL VIEW explode(p.context) contexts as context; +CREATE TABLE ${stats_db_name}.publication_concepts AS SELECT substr(p.id, 4) as id, contexts.context.id as concept from ${openaire_db_name}.publication p LATERAL VIEW explode(p.context) contexts as context; -CREATE TABLE ${hive_db_name}.publication_datasources as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM (SELECT substr(p.id, 4) as id, substr(instances.instance.hostedby.key, 4) as datasource -from ${hive_source_db_name}.publication p lateral view explode(p.instance) instances as instance) p LEFT OUTER JOIN -(SELECT substr(d.id, 4) id from ${hive_source_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; +CREATE TABLE ${stats_db_name}.publication_datasources as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM (SELECT substr(p.id, 4) as id, substr(instances.instance.hostedby.key, 4) as datasource from ${openaire_db_name}.publication p lateral view explode(p.instance) instances as instance) p LEFT OUTER JOIN (SELECT substr(d.id, 4) id from ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; -CREATE TABLE ${hive_db_name}.publication_languages AS select substr(p.id, 4) as id, p.language.classname as language from ${hive_source_db_name}.publication p; -CREATE TABLE ${hive_db_name}.publication_oids AS SELECT substr(p.id, 4) as id, oids.ids as oid from ${hive_source_db_name}.publication p LATERAL VIEW explode(p.originalid) oids as ids; -create table ${hive_db_name}.publication_pids as select substr(p.id, 4) as id, ppid.qualifier.classname as type, ppid.value as pid from ${hive_source_db_name}.publication p lateral view explode(p.pid) pids as ppid; -create table ${hive_db_name}.publication_topics as select substr(p.id, 4) as id, subjects.subject.qualifier.classname as type, subjects.subject.value as topic from ${hive_source_db_name}.publication p lateral view explode(p.subject) subjects as subject; +CREATE TABLE ${stats_db_name}.publication_languages AS select substr(p.id, 4) as id, p.language.classname as language FROM ${openaire_db_name}.publication p; + +CREATE TABLE ${stats_db_name}.publication_oids AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.publication p LATERAL VIEW explode(p.originalid) oids AS ids; + +CREATE TABLE ${stats_db_name}.publication_pids AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value as pid FROM ${openaire_db_name}.publication p LATERAL VIEW explode(p.pid) pids AS ppid; + +CREATE TABLE ${stats_db_name}.publication_topics as select substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS TYPE, subjects.subject.value AS topic FROM ${openaire_db_name}.publication p LATERAL VIEW explode(p.subject) subjects AS subject; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql index 95d167b2d..c1345d1ba 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql @@ -1,4 +1,2 @@ --- The following fails --- -- 3. Publication_citations -CREATE TABLE ${hive_db_name}.publication_citations AS SELECT substr(p.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS result from ${hive_source_db_name}.publication p lateral view explode(p.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; +CREATE TABLE ${stats_db_name}.publication_citations AS SELECT substr(p.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS result FROM ${openaire_db_name}.publication p lateral view explode(p.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql index de842d6df..d7202c2de 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql @@ -5,22 +5,32 @@ ------------------------------------------------------ -- Dataset temporary table supporting updates -DROP TABLE IF EXISTS ${hive_db_name}.dataset_tmp; -CREATE TABLE ${hive_db_name}.dataset_tmp (id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); - -INSERT INTO ${hive_db_name}.dataset_tmp SELECT substr(d.id, 4) as id, d.title[0].value as title, d.publisher.value as publisher, cast(null as string) as journal, d.dateofacceptance.value as date, -date_format(d.dateofacceptance.value,'yyyy') as year, d.bestaccessright.classname as bestlicence, d.embargoenddate.value as embargo_end_date, false as delayed, size(d.author) as authors, -concat_ws ('\u003B',d.source.value) as source, -case when size(d.description) > 0 then true else false end as abstract, 'dataset' as type from ${hive_source_db_name}.dataset d where d.datainfo.deletedbyinference=false; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_tmp; +CREATE TABLE ${stats_db_name}.dataset_tmp (id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) into 100 buckets stored AS orc tblproperties('transactional'='true'); +INSERT INTO ${stats_db_name}.dataset_tmp SELECT substr(d.id, 4) AS id, d.title[0].value AS title, d.publisher.value AS publisher, cast(null AS string) AS journal, +d.dateofacceptance.value as date, date_format(d.dateofacceptance.value,'yyyy') AS year, d.bestaccessright.classname AS bestlicence, +d.embargoenddate.value AS embargo_end_date, false AS delayed, size(d.author) AS authors , concat_ws('\u003B',d.source.value) AS source, + CASE WHEN SIZE(d.description) > 0 THEN TRUE ELSE FALSE end AS abstract, +'dataset' AS type +FROM ${openaire_db_name}.dataset d +WHERE d.datainfo.deletedbyinference=FALSE; -- Dataset_citations -CREATE TABLE ${hive_db_name}.dataset_citations as select substr(d.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") as result from ${hive_source_db_name}.dataset d lateral view explode(d.extrainfo) citations as citation where xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; +CREATE TABLE ${stats_db_name}.dataset_citations AS SELECT substr(d.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS result FROM ${openaire_db_name}.dataset d LATERAL VIEW explode(d.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; -CREATE TABLE ${hive_db_name}.dataset_classifications AS SELECT substr(p.id, 4) as id, instancetype.classname as type from ${hive_source_db_name}.dataset p LATERAL VIEW explode(p.instance.instancetype) instances as instancetype; -CREATE TABLE ${hive_db_name}.dataset_concepts AS SELECT substr(p.id, 4) as id, contexts.context.id as concept from ${hive_source_db_name}.dataset p LATERAL VIEW explode(p.context) contexts as context; -CREATE TABLE ${hive_db_name}.dataset_datasources as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM (SELECT substr(p.id, 4) as id, substr(instances.instance.hostedby.key, 4) as datasource from ${hive_source_db_name}.dataset p lateral view explode(p.instance) instances as instance) p LEFT OUTER JOIN (SELECT substr(d.id, 4) id from ${hive_source_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; -CREATE TABLE ${hive_db_name}.dataset_languages AS SELECT substr(p.id, 4) as id, p.language.classname as language from ${hive_source_db_name}.dataset p; -CREATE TABLE ${hive_db_name}.dataset_oids AS SELECT substr(p.id, 4) as id, oids.ids as oid from ${hive_source_db_name}.dataset p LATERAL VIEW explode(p.originalid) oids as ids; -CREATE TABLE ${hive_db_name}.dataset_pids AS SELECT substr(p.id, 4) as id, ppid.qualifier.classname as type, ppid.value as pid from ${hive_source_db_name}.dataset p lateral view explode(p.pid) pids as ppid; -CREATE TABLE ${hive_db_name}.dataset_topics AS SELECT substr(p.id, 4) as id, subjects.subject.qualifier.classname as type, subjects.subject.value as topic from ${hive_source_db_name}.dataset p lateral view explode(p.subject) subjects as subject; +CREATE TABLE ${stats_db_name}.dataset_classifications AS SELECT substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype; + +CREATE TABLE ${stats_db_name}.dataset_concepts AS SELECT substr(p.id, 4) as id, contexts.context.id as concept from ${openaire_db_name}.dataset p LATERAL VIEW explode(p.context) contexts as context; + +CREATE TABLE ${stats_db_name}.dataset_datasources AS SELECT p.id, case when d.id IS NULL THEN 'other' ELSE p.datasource END AS datasource FROM (SELECT substr(p.id, 4) as id, substr(instances.instance.hostedby.key, 4) AS datasource +FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.instance) instances AS instance) p LEFT OUTER JOIN +(SELECT substr(d.id, 4) id FROM ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d ON p.datasource = d.id; + +CREATE TABLE ${stats_db_name}.dataset_languages AS SELECT substr(p.id, 4) AS id, p.language.classname AS language FROM ${openaire_db_name}.dataset p; + +CREATE TABLE ${stats_db_name}.dataset_oids AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.originalid) oids AS ids; + +CREATE TABLE ${stats_db_name}.dataset_pids AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.pid) pids AS ppid; + +CREATE TABLE ${stats_db_name}.dataset_topics AS SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.subject) subjects AS subject; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql index 21175a5da..b2b3c4c75 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql @@ -5,28 +5,32 @@ -------------------------------------------------------- -- Software temporary table supporting updates -DROP TABLE IF EXISTS ${hive_db_name}.software_tmp; -CREATE TABLE ${hive_db_name}.software_tmp ( id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); -INSERT INTO ${hive_db_name}.software_tmp select substr(s.id, 4) as id, s.title[0].value as title, s.publisher.value as publisher, cast(null as string) as journal, -s.dateofacceptance.value as date, date_format(s.dateofacceptance.value,'yyyy') as year, s.bestaccessright.classname as bestlicence, -s.embargoenddate.value as embargo_end_date, false as delayed, size(s.author) as authors , concat_ws('\u003B',s.source.value) as source, - case when size(s.description) > 0 then true else false end as abstract, +DROP TABLE IF EXISTS ${stats_db_name}.software_tmp; +CREATE TABLE ${stats_db_name}.software_tmp (id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) INTO 100 buckets stored AS orc tblproperties('transactional'='true'); + +INSERT INTO ${stats_db_name}.software_tmp SELECT substr(s.id, 4) as id, s.title[0].value AS title, s.publisher.value AS publisher, CAST(NULL AS string) AS journal, +s.dateofacceptance.value AS DATE, date_format(s.dateofacceptance.value,'yyyy') AS YEAR, s.bestaccessright.classname AS bestlicence, +s.embargoenddate.value AS embargo_end_date, FALSE AS delayed, SIZE(s.author) AS authors , concat_ws('\u003B',s.source.value) AS source, + CASE WHEN SIZE(s.description) > 0 THEN TRUE ELSE FALSE END AS abstract, 'software' as type -from ${hive_source_db_name}.software s +from ${openaire_db_name}.software s where s.datainfo.deletedbyinference=false; - -- Software_citations -Create table ${hive_db_name}.software_citations as select substr(s.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") as result from ${hive_source_db_name}.software s lateral view explode(s.extrainfo) citations as citation where xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; +CREATE TABLE ${stats_db_name}.software_citations AS SELECT substr(s.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS RESULT FROM ${openaire_db_name}.software s LATERAL VIEW explode(s.extrainfo) citations as citation where xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; -CREATE TABLE ${hive_db_name}.software_classifications AS SELECT substr(p.id, 4) as id, instancetype.classname as type from ${hive_source_db_name}.software p LATERAL VIEW explode(p.instance.instancetype) instances as instancetype; -CREATE TABLE ${hive_db_name}.software_concepts AS SELECT substr(p.id, 4) as id, contexts.context.id as concept from ${hive_source_db_name}.software p LATERAL VIEW explode(p.context) contexts as context; +CREATE TABLE ${stats_db_name}.software_classifications AS SELECT substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype; -CREATE TABLE ${hive_db_name}.software_datasources as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM (SELECT substr(p.id, 4) as id, substr(instances.instance.hostedby.key, 4) as datasource -from ${hive_source_db_name}.software p lateral view explode(p.instance) instances as instance) p LEFT OUTER JOIN -(SELECT substr(d.id, 4) id from ${hive_source_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; +CREATE TABLE ${stats_db_name}.software_concepts AS SELECT substr(p.id, 4) AS id, contexts.context.id AS concept FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.context) contexts AS context; -CREATE TABLE ${hive_db_name}.software_languages AS select substr(p.id, 4) as id, p.language.classname as language from ${hive_source_db_name}.software p; -CREATE TABLE ${hive_db_name}.software_oids AS SELECT substr(p.id, 4) as id, oids.ids as oid from ${hive_source_db_name}.software p LATERAL VIEW explode(p.originalid) oids as ids; -create table ${hive_db_name}.software_pids as select substr(p.id, 4) as id, ppid.qualifier.classname as type, ppid.value as pid from ${hive_source_db_name}.software p lateral view explode(p.pid) pids as ppid; -create table ${hive_db_name}.software_topics as select substr(p.id, 4) as id, subjects.subject.qualifier.classname as type, subjects.subject.value as topic from ${hive_source_db_name}.software p lateral view explode(p.subject) subjects as subject; +CREATE TABLE ${stats_db_name}.software_datasources AS SELECT p.id, CASE WHEN d.id IS NULL THEN 'other' ELSE p.datasource end as datasource FROM (SELECT substr(p.id, 4) AS id, substr(instances.instance.hostedby.key, 4) AS datasource +FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.instance) instances AS instance) p LEFT OUTER JOIN +(SELECT substr(d.id, 4) id FROM ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d ON p.datasource = d.id; + +CREATE TABLE ${stats_db_name}.software_languages AS select substr(p.id, 4) AS id, p.language.classname AS language FROM ${openaire_db_name}.software p; + +CREATE TABLE ${stats_db_name}.software_oids AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.originalid) oids AS ids; + +CREATE TABLE ${stats_db_name}.software_pids AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.pid) pids AS ppid; + +CREATE TABLE ${stats_db_name}.software_topics AS SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.subject) subjects AS subject; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql index 58967a840..cb2c67c9e 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql @@ -5,27 +5,33 @@ -------------------------------------------------------------------------------- -- Otherresearchproduct temporary table supporting updates -DROP TABLE IF EXISTS ${hive_db_name}.otherresearchproduct_tmp; -CREATE TABLE ${hive_db_name}.otherresearchproduct_tmp ( id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); -INSERT INTO ${hive_db_name}.otherresearchproduct_tmp select substr(o.id, 4) as id, o.title[0].value as title, o.publisher.value as publisher, cast(null as string) as journal, -o.dateofacceptance.value as date, date_format(o.dateofacceptance.value,'yyyy') as year, o.bestaccessright.classname as bestlicence, -o.embargoenddate.value as embargo_end_date, false as delayed, size(o.author) as authors , concat_ws('\u003B',o.source.value) as source, -case when size(o.description) > 0 then true else false end as abstract, -'other' as type -from ${hive_source_db_name}.otherresearchproduct o -where o.datainfo.deletedbyinference=false; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_tmp; +CREATE TABLE ${stats_db_name}.otherresearchproduct_tmp ( id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties('transactional'='true'); + +INSERT INTO ${stats_db_name}.otherresearchproduct_tmp SELECT substr(o.id, 4) AS id, o.title[0].value AS title, o.publisher.value AS publisher, CAST(NULL AS string) AS journal, +o.dateofacceptance.value AS DATE, date_format(o.dateofacceptance.value,'yyyy') AS year, o.bestaccessright.classname AS bestlicence, +o.embargoenddate.value as embargo_end_date, FALSE AS delayed, SIZE(o.author) AS authors , concat_ws('\u003B',o.source.value) AS source, +CASE WHEN SIZE(o.description) > 0 THEN TRUE ELSE FALSE END AS abstract, +'other' AS type +FROM ${openaire_db_name}.otherresearchproduct o +WHERE o.datainfo.deletedbyinference=FALSE; -- Otherresearchproduct_citations -Create table ${hive_db_name}.otherresearchproduct_citations as select substr(o.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") as result from ${hive_source_db_name}.otherresearchproduct o lateral view explode(o.extrainfo) citations as citation where xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; +CREATE TABLE ${stats_db_name}.otherresearchproduct_citations AS SELECT substr(o.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS RESULT FROM ${openaire_db_name}.otherresearchproduct o LATERAL VIEW explode(o.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; -CREATE TABLE ${hive_db_name}.otherresearchproduct_classifications AS SELECT substr(p.id, 4) as id, instancetype.classname as type from ${hive_source_db_name}.otherresearchproduct p LATERAL VIEW explode(p.instance.instancetype) instances as instancetype; -CREATE TABLE ${hive_db_name}.otherresearchproduct_concepts AS SELECT substr(p.id, 4) as id, contexts.context.id as concept from ${hive_source_db_name}.otherresearchproduct p LATERAL VIEW explode(p.context) contexts as context; +CREATE TABLE ${stats_db_name}.otherresearchproduct_classifications AS SELECT substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype; -CREATE TABLE ${hive_db_name}.otherresearchproduct_datasources as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM (SELECT substr(p.id, 4) as id, substr(instances.instance.hostedby.key, 4) as datasource -from ${hive_source_db_name}.otherresearchproduct p lateral view explode(p.instance) instances as instance) p LEFT OUTER JOIN -(SELECT substr(d.id, 4) id from ${hive_source_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; +CREATE TABLE ${stats_db_name}.otherresearchproduct_concepts AS SELECT substr(p.id, 4) AS id, contexts.context.id AS concept FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.context) contexts AS context; -CREATE TABLE ${hive_db_name}.otherresearchproduct_languages AS select substr(p.id, 4) as id, p.language.classname as language from ${hive_source_db_name}.otherresearchproduct p; -CREATE TABLE ${hive_db_name}.otherresearchproduct_oids AS SELECT substr(p.id, 4) as id, oids.ids as oid from ${hive_source_db_name}.otherresearchproduct p LATERAL VIEW explode(p.originalid) oids as ids; -create table ${hive_db_name}.otherresearchproduct_pids as select substr(p.id, 4) as id, ppid.qualifier.classname as type, ppid.value as pid from ${hive_source_db_name}.otherresearchproduct p lateral view explode(p.pid) pids as ppid; -create table ${hive_db_name}.otherresearchproduct_topics as select substr(p.id, 4) as id, subjects.subject.qualifier.classname as type, subjects.subject.value as topic from ${hive_source_db_name}.otherresearchproduct p lateral view explode(p.subject) subjects as subject; + +CREATE TABLE ${stats_db_name}.otherresearchproduct_datasources AS SELECT p.id, CASE WHEN d.id IS NULL THEN 'other' ELSE p.datasource END AS datasource FROM (SELECT substr(p.id, 4) AS id, substr(instances.instance.hostedby.key, 4) AS datasource +from ${openaire_db_name}.otherresearchproduct p lateral view explode(p.instance) instances as instance) p LEFT OUTER JOIN +(SELECT substr(d.id, 4) id from ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; + +CREATE TABLE ${stats_db_name}.otherresearchproduct_languages AS SELECT substr(p.id, 4) AS id, p.language.classname AS language FROM ${openaire_db_name}.otherresearchproduct p; + +CREATE TABLE ${stats_db_name}.otherresearchproduct_oids AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.originalid) oids AS ids; + +CREATE TABLE ${stats_db_name}.otherresearchproduct_pids AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.pid) pids AS ppid; + +CREATE TABLE ${stats_db_name}.otherresearchproduct_topics AS SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.subject) subjects AS subject; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql index fa5838b61..927543f5a 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql @@ -4,18 +4,21 @@ ------------------------------------------------------ ------------------------------------------------------ -- Project_oids Table -DROP TABLE IF EXISTS ${hive_db_name}.project_oids; -CREATE TABLE ${hive_db_name}.project_oids AS SELECT substr(p.id, 4) as id, oids.ids as oid from ${hive_source_db_name}.project p LATERAL VIEW explode(p.originalid) oids as ids; +DROP TABLE IF EXISTS ${stats_db_name}.project_oids; +CREATE TABLE ${stats_db_name}.project_oids AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.project p LATERAL VIEW explode(p.originalid) oids AS ids; + -- Project_organizations Table -DROP TABLE IF EXISTS ${hive_db_name}.project_organizations; -CREATE TABLE ${hive_db_name}.project_organizations AS SELECT substr(r.source, 4) as id, substr(r.target, 4) AS organization from ${hive_source_db_name}.relation r WHERE r.reltype='projectOrganization'; +DROP TABLE IF EXISTS ${stats_db_name}.project_organizations; +CREATE TABLE ${stats_db_name}.project_organizations AS SELECT substr(r.source, 4) AS id, substr(r.target, 4) AS organization from ${openaire_db_name}.relation r WHERE r.reltype='projectOrganization'; + -- Project_results Table -DROP TABLE IF EXISTS ${hive_db_name}.project_results; -CREATE TABLE ${hive_db_name}.project_results AS SELECT substr(r.target, 4) as id, substr(r.source, 4) AS result from ${hive_source_db_name}.relation r WHERE r.reltype='resultProject'; +DROP TABLE IF EXISTS ${stats_db_name}.project_results; +CREATE TABLE ${stats_db_name}.project_results AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS result FROM ${openaire_db_name}.relation r WHERE r.reltype='resultProject'; -- Project table ---------------- -- Creating and populating temporary Project table -DROP TABLE IF EXISTS ${hive_db_name}.project_tmp; -CREATE TABLE ${hive_db_name}.project_tmp (id STRING, acronym STRING, title STRING, funder STRING, funding_lvl0 STRING, funding_lvl1 STRING, funding_lvl2 STRING, ec39 STRING, type STRING, startdate STRING, enddate STRING, start_year STRING, end_year STRING, duration INT, haspubs STRING, numpubs INT, daysforlastpub INT, delayedpubs INT, callidentifier STRING, code STRING) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); -INSERT INTO ${hive_db_name}.project_tmp SELECT substr(p.id, 4) as id, p.acronym.value as acronym, p.title.value as title, xpath_string(p.fundingtree[0].value, '//funder/name') as funder, xpath_string(p.fundingtree[0].value, '//funding_level_0/name') as funding_lvl0, xpath_string(p.fundingtree[0].value, '//funding_level_1/name') as funding_lvl1, xpath_string(p.fundingtree[0].value, '//funding_level_2/name') as funding_lvl2, p.ecsc39.value as ec39, p.contracttype.classname as type, p.startdate.value as startdate, p.enddate.value as enddate, date_format(p.startdate.value, 'yyyy') as start_year, date_format(p.enddate.value, 'yyyy') as end_year, 0 as duration, 'no' as haspubs, 0 as numpubs, 0 as daysforlastpub, 0 as delayedpubs, p.callidentifier.value as callidentifier, p.code.value as code from ${hive_source_db_name}.project p WHERE p.datainfo.deletedbyinference=false; +DROP TABLE IF EXISTS ${stats_db_name}.project_tmp; +CREATE TABLE ${stats_db_name}.project_tmp (id STRING, acronym STRING, title STRING, funder STRING, funding_lvl0 STRING, funding_lvl1 STRING, funding_lvl2 STRING, ec39 STRING, type STRING, startdate STRING, enddate STRING, start_year STRING, end_year STRING, duration INT, haspubs STRING, numpubs INT, daysforlastpub INT, delayedpubs INT, callidentifier STRING, code STRING) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties('transactional'='true'); + +INSERT INTO ${stats_db_name}.project_tmp SELECT substr(p.id, 4) AS id, p.acronym.value AS acronym, p.title.value AS title, xpath_string(p.fundingtree[0].value, '//funder/name') AS funder, xpath_string(p.fundingtree[0].value, '//funding_level_0/name') AS funding_lvl0, xpath_string(p.fundingtree[0].value, '//funding_level_1/name') AS funding_lvl1, xpath_string(p.fundingtree[0].value, '//funding_level_2/name') AS funding_lvl2, p.ecsc39.value AS ec39, p.contracttype.classname AS type, p.startdate.value AS startdate, p.enddate.value AS enddate, date_format(p.startdate.value, 'yyyy') AS start_year, date_format(p.enddate.value, 'yyyy') AS end_year, 0 AS duration, 'no' AS haspubs, 0 AS numpubs, 0 AS daysforlastpub, 0 AS delayedpubs, p.callidentifier.value AS callidentifier, p.code.value AS code FROM ${openaire_db_name}.project p WHERE p.datainfo.deletedbyinference=false; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql index 3d346fb95..de87b0073 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql @@ -5,20 +5,27 @@ ---------------------------------------------------- -- Views on temporary tables that should be re-created in the end -CREATE OR REPLACE VIEW ${hive_db_name}.result as SELECT *, bestlicence as access_mode FROM ${hive_db_name}.publication_tmp UNION ALL SELECT *,bestlicence as access_mode FROM ${hive_db_name}.software_tmp UNION ALL SELECT *,bestlicence as access_mode FROM ${hive_db_name}.dataset_tmp UNION ALL SELECT *,bestlicence as access_mode FROM ${hive_db_name}.otherresearchproduct_tmp; +CREATE OR REPLACE VIEW ${stats_db_name}.result as SELECT *, bestlicence AS access_mode FROM ${stats_db_name}.publication_tmp UNION ALL SELECT *,bestlicence AS access_mode FROM ${stats_db_name}.software_tmp UNION ALL SELECT *,bestlicence AS access_mode FROM ${stats_db_name}.dataset_tmp UNION ALL SELECT *,bestlicence AS access_mode FROM ${stats_db_name}.otherresearchproduct_tmp; -- Views on final tables -CREATE OR REPLACE VIEW ${hive_db_name}.result_datasources as SELECT * FROM ${hive_db_name}.publication_datasources UNION ALL SELECT * FROM ${hive_db_name}.software_datasources UNION ALL SELECT * FROM ${hive_db_name}.dataset_datasources UNION ALL SELECT * FROM ${hive_db_name}.otherresearchproduct_datasources; -CREATE OR REPLACE VIEW ${hive_db_name}.result_citations as SELECT * FROM ${hive_db_name}.publication_citations UNION ALL SELECT * FROM ${hive_db_name}.software_citations UNION ALL SELECT * FROM ${hive_db_name}.dataset_citations UNION ALL SELECT * FROM ${hive_db_name}.otherresearchproduct_citations; -CREATE OR REPLACE VIEW ${hive_db_name}.result_classifications as SELECT * FROM ${hive_db_name}.publication_classifications UNION ALL SELECT * FROM ${hive_db_name}.software_classifications UNION ALL SELECT * FROM ${hive_db_name}.dataset_classifications UNION ALL SELECT * FROM ${hive_db_name}.otherresearchproduct_classifications; -CREATE OR REPLACE VIEW ${hive_db_name}.result_concepts as SELECT * FROM ${hive_db_name}.publication_concepts UNION ALL SELECT * FROM ${hive_db_name}.software_concepts UNION ALL SELECT * FROM ${hive_db_name}.dataset_concepts UNION ALL SELECT * FROM ${hive_db_name}.otherresearchproduct_concepts; -CREATE OR REPLACE VIEW ${hive_db_name}.result_languages as SELECT * FROM ${hive_db_name}.publication_languages UNION ALL SELECT * FROM ${hive_db_name}.software_languages UNION ALL SELECT * FROM ${hive_db_name}.dataset_languages UNION ALL SELECT * FROM ${hive_db_name}.otherresearchproduct_languages; -CREATE OR REPLACE VIEW ${hive_db_name}.result_oids as SELECT * FROM ${hive_db_name}.publication_oids UNION ALL SELECT * FROM ${hive_db_name}.software_oids UNION ALL SELECT * FROM ${hive_db_name}.dataset_oids UNION ALL SELECT * FROM ${hive_db_name}.otherresearchproduct_oids; -CREATE OR REPLACE VIEW ${hive_db_name}.result_pids as SELECT * FROM ${hive_db_name}.publication_pids UNION ALL SELECT * FROM ${hive_db_name}.software_pids UNION ALL SELECT * FROM ${hive_db_name}.dataset_pids UNION ALL SELECT * FROM ${hive_db_name}.otherresearchproduct_pids; -CREATE OR REPLACE VIEW ${hive_db_name}.result_topics as SELECT * FROM ${hive_db_name}.publication_topics UNION ALL SELECT * FROM ${hive_db_name}.software_topics UNION ALL SELECT * FROM ${hive_db_name}.dataset_topics UNION ALL SELECT * FROM ${hive_db_name}.otherresearchproduct_topics; +CREATE OR REPLACE VIEW ${stats_db_name}.result_datasources AS SELECT * FROM ${stats_db_name}.publication_datasources UNION ALL SELECT * FROM ${stats_db_name}.software_datasources UNION ALL SELECT * FROM ${stats_db_name}.dataset_datasources UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_datasources; -DROP TABLE IF EXISTS ${hive_db_name}.result_organization; -CREATE TABLE ${hive_db_name}.result_organization AS SELECT substr(r.target, 4) as id, substr(r.source, 4) as organization from ${hive_source_db_name}.relation r where r.reltype='resultOrganization'; +CREATE OR REPLACE VIEW ${stats_db_name}.result_citations AS SELECT * FROM ${stats_db_name}.publication_citations UNION ALL SELECT * FROM ${stats_db_name}.software_citations UNION ALL SELECT * FROM ${stats_db_name}.dataset_citations UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_citations; -DROP TABLE IF EXISTS ${hive_db_name}.result_projects; -CREATE TABLE ${hive_db_name}.result_projects AS select pr.result as id, pr.id as project, datediff(p.enddate, p.startdate) as daysfromend from ${hive_db_name}.result r join ${hive_db_name}.project_results pr on r.id=pr.result join ${hive_db_name}.project_tmp p on p.id=pr.id; +CREATE OR REPLACE VIEW ${stats_db_name}.result_classifications AS SELECT * FROM ${stats_db_name}.publication_classifications UNION ALL SELECT * FROM ${stats_db_name}.software_classifications UNION ALL SELECT * FROM ${stats_db_name}.dataset_classifications UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_classifications; + +CREATE OR REPLACE VIEW ${stats_db_name}.result_concepts AS SELECT * FROM ${stats_db_name}.publication_concepts UNION ALL SELECT * FROM ${stats_db_name}.software_concepts UNION ALL SELECT * FROM ${stats_db_name}.dataset_concepts UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_concepts; + +CREATE OR REPLACE VIEW ${stats_db_name}.result_languages AS SELECT * FROM ${stats_db_name}.publication_languages UNION ALL SELECT * FROM ${stats_db_name}.software_languages UNION ALL SELECT * FROM ${stats_db_name}.dataset_languages UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_languages; + +CREATE OR REPLACE VIEW ${stats_db_name}.result_oids AS SELECT * FROM ${stats_db_name}.publication_oids UNION ALL SELECT * FROM ${stats_db_name}.software_oids UNION ALL SELECT * FROM ${stats_db_name}.dataset_oids UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_oids; + +CREATE OR REPLACE VIEW ${stats_db_name}.result_pids AS SELECT * FROM ${stats_db_name}.publication_pids UNION ALL SELECT * FROM ${stats_db_name}.software_pids UNION ALL SELECT * FROM ${stats_db_name}.dataset_pids UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_pids; + +CREATE OR REPLACE VIEW ${stats_db_name}.result_topics AS SELECT * FROM ${stats_db_name}.publication_topics UNION ALL SELECT * FROM ${stats_db_name}.software_topics UNION ALL SELECT * FROM ${stats_db_name}.dataset_topics UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_topics; + +DROP TABLE IF EXISTS ${stats_db_name}.result_organization; +CREATE TABLE ${stats_db_name}.result_organization AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS organization FROM ${openaire_db_name}.relation r WHERE r.reltype='resultOrganization'; + +DROP TABLE IF EXISTS ${stats_db_name}.result_projects; +CREATE TABLE ${stats_db_name}.result_projects AS select pr.result AS id, pr.id AS project, datediff(p.enddate, p.startdate) AS daysfromend FROM ${stats_db_name}.result r JOIN ${stats_db_name}.project_results pr ON r.id=pr.result JOIN ${stats_db_name}.project_tmp p ON p.id=pr.id; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql index 114eeeeaa..5a120d920 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql @@ -6,28 +6,31 @@ -- Datasource table creation & update ------------------------------------- -- Creating and populating temporary datasource table -DROP TABLE IF EXISTS ${hive_db_name}.datasource_tmp; -create table ${hive_db_name}.datasource_tmp(`id` string, `name` string, `type` string, `dateofvalidation` string, `yearofvalidation` string, `harvested` boolean, `piwik_id` int, `latitude` string, `longitude` string, `websiteurl` string, `compatibility` string) clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); -insert into ${hive_db_name}.datasource_tmp SELECT substr(d.id, 4) as id, officialname.value as name, datasourcetype.classname as type, dateofvalidation.value as dateofvalidation, date_format(d.dateofvalidation.value,'yyyy') as yearofvalidation, false as harvested, 0 as piwik_id, d.latitude.value as latitude, d.longitude.value as longitude, d.websiteurl.value as websiteurl, d.openairecompatibility.classid as compatibility -from ${hive_source_db_name}.datasource d -WHERE d.datainfo.deletedbyinference=false; +DROP TABLE IF EXISTS ${stats_db_name}.datasource_tmp; +CREATE TABLE ${stats_db_name}.datasource_tmp(`id` string, `name` STRING, `type` STRING, `dateofvalidation` STRING, `yearofvalidation` string, `harvested` BOOLEAN, `piwik_id` INT, `latitude` STRING, `longitude`STRING, `websiteurl` STRING, `compatibility` STRING) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties('transactional'='true'); --- Updating temporary table with everything that is not based on results -> This is done with the following "dual" table. To see if default values are there +INSERT INTO ${stats_db_name}.datasource_tmp SELECT substr(d.id, 4) AS id, officialname.value AS name, datasourcetype.classname AS type, dateofvalidation.value AS dateofvalidation, date_format(d.dateofvalidation.value,'yyyy') AS yearofvalidation, FALSE AS harvested, 0 AS piwik_id, d.latitude.value AS latitude, d.longitude.value AS longitude, d.websiteurl.value AS websiteurl, d.openairecompatibility.classid AS compatibility +FROM ${openaire_db_name}.datasource d +WHERE d.datainfo.deletedbyinference=FALSE; + +-- Updating temporary table with everything that is not based on results -> This is done with the following "dual" table. -- Creating a temporary dual table that will be removed after the following insert -CREATE TABLE ${hive_db_name}.dual(dummy char(1)); -INSERT INTO ${hive_db_name}.dual values('X'); -INSERT INTO ${hive_db_name}.datasource_tmp (`id`, `name`, `type`, `dateofvalidation`, `yearofvalidation`, `harvested`, `piwik_id`, `latitude`, `longitude`, `websiteurl`, `compatibility`) -SELECT 'other', 'Other', 'Repository', NULL, NULL, false, 0, NULL, NULL, NULL, 'unknown' FROM ${hive_db_name}.dual WHERE 'other' not in (SELECT id FROM ${hive_db_name}.datasource_tmp WHERE name='Unknown Repository'); -DROP TABLE ${hive_db_name}.dual; +CREATE TABLE ${stats_db_name}.dual(dummy CHAR(1)); +INSERT INTO ${stats_db_name}.dual VALUES('X'); +INSERT INTO ${stats_db_name}.datasource_tmp (`id`, `name`, `type`, `dateofvalidation`, `yearofvalidation`, `harvested`, `piwik_id`, `latitude`, `longitude`, `websiteurl`, `compatibility`) +SELECT 'other', 'Other', 'Repository', NULL, NULL, false, 0, NULL, NULL, NULL, 'unknown' FROM ${stats_db_name}.dual WHERE 'other' not in (SELECT id FROM ${stats_db_name}.datasource_tmp WHERE name='Unknown Repository'); +DROP TABLE ${stats_db_name}.dual; -UPDATE ${hive_db_name}.datasource_tmp SET name='Other' where name='Unknown Repository'; -UPDATE ${hive_db_name}.datasource_tmp SET yearofvalidation=null WHERE yearofvalidation='-1'; +UPDATE ${stats_db_name}.datasource_tmp SET name='Other' WHERE name='Unknown Repository'; +UPDATE ${stats_db_name}.datasource_tmp SET yearofvalidation=null WHERE yearofvalidation='-1'; -DROP TABLE IF EXISTS ${hive_db_name}.datasource_languages; -CREATE TABLE ${hive_db_name}.datasource_languages AS SELECT substr(d.id, 4) as id, langs.languages as language from ${hive_source_db_name}.datasource d LATERAL VIEW explode(d.odlanguages.value) langs as languages; -DROP TABLE IF EXISTS ${hive_db_name}.datasource_oids; -CREATE TABLE ${hive_db_name}.datasource_oids AS SELECT substr(d.id, 4) as id, oids.ids as oid from ${hive_source_db_name}.datasource d LATERAL VIEW explode(d.originalid) oids as ids; -DROP TABLE IF EXISTS ${hive_db_name}.datasource_organizations; -CREATE TABLE ${hive_db_name}.datasource_organizations AS select substr(r.target, 4) as id, substr(r.source, 4) as organization from ${hive_source_db_name}.relation r where r.reltype='datasourceOrganization'; +DROP TABLE IF EXISTS ${stats_db_name}.datasource_languages; +CREATE TABLE ${stats_db_name}.datasource_languages AS SELECT substr(d.id, 4) AS id, langs.languages AS language FROM ${openaire_db_name}.datasource d LATERAL VIEW explode(d.odlanguages.value) langs AS languages; -CREATE OR REPLACE VIEW ${hive_db_name}.datasource_results AS SELECT datasource AS id, id AS result FROM ${hive_db_name}.result_datasources; +DROP TABLE IF EXISTS ${stats_db_name}.datasource_oids; +CREATE TABLE ${stats_db_name}.datasource_oids AS SELECT substr(d.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.datasource d LATERAL VIEW explode(d.originalid) oids AS ids; + +DROP TABLE IF EXISTS ${stats_db_name}.datasource_organizations; +CREATE TABLE ${stats_db_name}.datasource_organizations AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS organization FROM ${openaire_db_name}.relation r WHERE r.reltype='datasourceOrganization'; + +CREATE OR REPLACE VIEW ${stats_db_name}.datasource_results AS SELECT datasource AS id, id AS result FROM ${stats_db_name}.result_datasources; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_6.sql deleted file mode 100644 index 461f48bfc..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9_6.sql +++ /dev/null @@ -1,2 +0,0 @@ -DROP TABLE IF EXISTS ${stats_db_name}.datasource_languages; -CREATE TABLE ${stats_db_name}.datasource_languages AS SELECT substr(d.id, 4) as id, langs.languages as language from ${openaire_db_name}.datasource d LATERAL VIEW explode(d.odlanguages.value) langs as languages; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index 269525c0f..b5f3158f7 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml @@ -1,12 +1,25 @@ - + - hive_db_name - the target hive database name + stats_db_name + the target stats database name - hive_source_db_name - the source hive database name + openaire_db_name + the original graph database name + + + external_stats_db_name + stats_ext + the external stats that should be added since they are not included in the graph database + + + hive_metastore_uris + hive server metastore URIs + + + hive_jdbc_url + hive server jdbc url hive_timeout @@ -20,7 +33,7 @@ hive.metastore.uris - ${hiveMetastoreUris} + ${hive_metastore_uris} hive.txn.timeout @@ -29,271 +42,170 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - + - ${jobTracker} - ${nameNode} - - - hive.metastore.uris - ${hive_metastore_uris} - - - ${hive_jdbc_url}/${hive_db_name} + ${hive_jdbc_url} - hive_db_name=${hive_db_name} - hive_source_db_name=${hive_source_db_name} + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + - ${jobTracker} - ${nameNode} - - - hive.metastore.uris - ${hive_metastore_uris} - - - ${hive_jdbc_url}/${hive_db_name} + ${hive_jdbc_url} - hive_db_name=${hive_db_name} - hive_source_db_name=${hive_source_db_name} + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + - ${jobTracker} - ${nameNode} - - - hive.metastore.uris - ${hive_metastore_uris} - - - ${hive_jdbc_url}/${hive_db_name} + ${hive_jdbc_url} - hive_db_name=${hive_db_name} - hive_source_db_name=${hive_source_db_name} + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} - + - ${jobTracker} - ${nameNode} - - - hive.metastore.uris - ${hive_metastore_uris} - - - hive.variable.substitute - true - - - ${hive_jdbc_url}/${hive_db_name} + ${hive_jdbc_url} - hive_db_name=${hive_db_name} - hive_source_db_name=${hive_source_db_name} + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + - - ${jobTracker} - ${nameNode} - - - hive.metastore.uris - ${hive_metastore_uris} - - - ${hive_jdbc_url}/${hive_db_name} + + ${hive_jdbc_url} - hive_db_name=${hive_db_name} - hive_source_db_name=${hive_source_db_name} + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + - - ${jobTracker} - ${nameNode} - - - hive.metastore.uris - ${hive_metastore_uris} - - - ${hive_jdbc_url}/${hive_db_name} + + ${hive_jdbc_url} - hive_db_name=${hive_db_name} - hive_source_db_name=${hive_source_db_name} + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + - - ${jobTracker} - ${nameNode} - - - hive.metastore.uris - ${hive_metastore_uris} - - - ${hive_jdbc_url}/${hive_db_name} + + ${hive_jdbc_url} - hive_db_name=${hive_db_name} - hive_source_db_name=${hive_source_db_name} + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + - - ${jobTracker} - ${nameNode} - - - hive.metastore.uris - ${hive_metastore_uris} - - - ${hive_jdbc_url}/${hive_db_name} + + ${hive_jdbc_url} - hive_db_name=${hive_db_name} - hive_source_db_name=${hive_source_db_name} + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + - - ${jobTracker} - ${nameNode} - - - hive.metastore.uris - ${hive_metastore_uris} - - - ${hive_jdbc_url}/${hive_db_name} + + ${hive_jdbc_url} - hive_db_name=${hive_db_name} - hive_source_db_name=${hive_source_db_name} + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + - - ${jobTracker} - ${nameNode} - - - hive.metastore.uris - ${hive_metastore_uris} - - - ${hive_jdbc_url}/${hive_db_name} + + ${hive_jdbc_url} - hive_db_name=${hive_db_name} - hive_source_db_name=${hive_source_db_name} + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + - - ${jobTracker} - ${nameNode} - - - hive.metastore.uris - ${hive_metastore_uris} - - - ${hive_jdbc_url}/${hive_db_name} + + ${hive_jdbc_url} - hive_db_name=${hive_db_name} - hive_source_db_name=${hive_source_db_name} + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + external_stats_db_name=${external_stats_db_name} - + + - - ${jobTracker} - ${nameNode} - - - hive.metastore.uris - ${hive_metastore_uris} - - - ${hive_jdbc_url}/${hive_db_name} + + ${hive_jdbc_url} - hive_db_name=${hive_db_name} - hive_source_db_name=${hive_source_db_name} + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + external_stats_db_name=${external_stats_db_name} - +
+ - ${jobTracker} - ${nameNode} - - - hive.metastore.uris - ${hive_metastore_uris} - - - ${hive_jdbc_url}/${hive_db_name} + ${hive_jdbc_url} - hive_db_name=${hive_db_name} - hive_source_db_name=${hive_source_db_name} + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + ${jobTracker} ${nameNode} - impala-shell.sh - ${hive_db_name} - impala-stats.sql - /user/giorgos.alexiou/graph/oozie_app/impala-stats.sql + ${stats_db_name} + step14.sql + /user/${wf:user()}/oa/graph/stats/oozie_app/scripts/step14.sql impala-shell.sh - - + From adf8a025d23a4c6450f845654e9567e1459e458b Mon Sep 17 00:00:00 2001 From: Spyros Zoupanos Date: Mon, 15 Jun 2020 19:57:40 +0300 Subject: [PATCH 081/100] Adding more relations (Sources, Licences, Additional) and shadow schema as provided and discussed with Antonis Lempesis --- .../graph/stats/oozie_app/scripts/step1.sql | 13 +- .../graph/stats/oozie_app/scripts/step10.sql | 21 ++- .../graph/stats/oozie_app/scripts/step11.sql | 54 ++++-- .../graph/stats/oozie_app/scripts/step12.sql | 66 ++++---- .../graph/stats/oozie_app/scripts/step13.sql | 95 +++++++---- .../graph/stats/oozie_app/scripts/step14.sql | 96 ++++++----- .../graph/stats/oozie_app/scripts/step15.sql | 35 ++++ .../graph/stats/oozie_app/scripts/step16.sql | 160 ++++++++++++++++++ .../graph/stats/oozie_app/scripts/step17.sql | 54 ++++++ .../graph/stats/oozie_app/scripts/step2.sql | 5 +- .../graph/stats/oozie_app/scripts/step3.sql | 38 ++++- .../graph/stats/oozie_app/scripts/step4.sql | 50 +++--- .../graph/stats/oozie_app/scripts/step5.sql | 53 +++--- .../graph/stats/oozie_app/scripts/step6.sql | 53 +++--- .../graph/stats/oozie_app/scripts/step7.sql | 47 ++--- .../graph/stats/oozie_app/scripts/step8.sql | 52 +++--- .../graph/stats/oozie_app/scripts/step9.sql | 44 ++--- .../dhp/oa/graph/stats/oozie_app/workflow.xml | 55 +++++- 18 files changed, 670 insertions(+), 321 deletions(-) create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step17.sql diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql index 92e3be7a5..9697a1dc8 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step1.sql @@ -1,11 +1,8 @@ --- DROP database if EXISTS ${hive_db_name} cascade; --- CREATE database ${hive_db_name}; --- --- CREATE TABLE ${hive_db_name}.Persons ( --- PersonID int, --- LastName varchar(255)); --- --- INSERT INTO ${hive_db_name}.Persons VALUES (1, "test_db_spyros_rec_111"); +-------------------------------------------------------------- +-------------------------------------------------------------- +-- Stats database creation +-------------------------------------------------------------- +-------------------------------------------------------------- DROP database IF EXISTS ${stats_db_name} CASCADE; CREATE database ${stats_db_name}; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql index 792c8cbdd..145be77b0 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql @@ -1,11 +1,10 @@ ----------------------------------------------------------------- ----------------------------------------------------------------- --- Organization table/view and Organization related tables/views ----------------------------------------------------------------- ----------------------------------------------------------------- -DROP TABLE IF EXISTS ${stats_db_name}.organization; -CREATE TABLE ${stats_db_name}.organization AS SELECT substr(o.id, 4) as id, o.legalname.value AS name, o.country.classid AS country FROM ${openaire_db_name}.organization o WHERE o.datainfo.deletedbyinference=false; - -CREATE OR REPLACE VIEW ${stats_db_name}.organization_datasources AS SELECT organization AS id, id AS datasource FROM ${stats_db_name}.datasource_organizations; - -CREATE OR REPLACE VIEW ${stats_db_name}.organization_projects AS SELECT id AS project, organization as id FROM ${stats_db_name}.project_organizations; +------------------------------------------------------------------------------------------------ +------------------------------------------------------------------------------------------------ +-- Tables/views from external tables/views (Fundref, Country, CountyGDP, roarmap, rndexpediture) +------------------------------------------------------------------------------------------------ +------------------------------------------------------------------------------------------------ +CREATE OR REPLACE VIEW ${stats_db_name}.fundref AS SELECT * FROM ${external_stats_db_name}.fundref; +CREATE OR REPLACE VIEW ${stats_db_name}.country AS SELECT * FROM ${external_stats_db_name}.country; +CREATE OR REPLACE VIEW ${stats_db_name}.countrygdp AS SELECT * FROM ${external_stats_db_name}.countrygdp; +CREATE OR REPLACE VIEW ${stats_db_name}.roarmap AS SELECT * FROM ${external_stats_db_name}.roarmap; +CREATE OR REPLACE VIEW ${stats_db_name}.rndexpediture AS SELECT * FROM ${external_stats_db_name}.rndexpediture; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql index 145be77b0..13e141459 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql @@ -1,10 +1,44 @@ ------------------------------------------------------------------------------------------------- ------------------------------------------------------------------------------------------------- --- Tables/views from external tables/views (Fundref, Country, CountyGDP, roarmap, rndexpediture) ------------------------------------------------------------------------------------------------- ------------------------------------------------------------------------------------------------- -CREATE OR REPLACE VIEW ${stats_db_name}.fundref AS SELECT * FROM ${external_stats_db_name}.fundref; -CREATE OR REPLACE VIEW ${stats_db_name}.country AS SELECT * FROM ${external_stats_db_name}.country; -CREATE OR REPLACE VIEW ${stats_db_name}.countrygdp AS SELECT * FROM ${external_stats_db_name}.countrygdp; -CREATE OR REPLACE VIEW ${stats_db_name}.roarmap AS SELECT * FROM ${external_stats_db_name}.roarmap; -CREATE OR REPLACE VIEW ${stats_db_name}.rndexpediture AS SELECT * FROM ${external_stats_db_name}.rndexpediture; +---------------------------------------------------------------- +---------------------------------------------------------------- +-- Post processing - Updates on main tables +---------------------------------------------------------------- +---------------------------------------------------------------- + +--Datasource temporary table updates +UPDATE ${stats_db_name}.datasource_tmp SET harvested='true' WHERE datasource_tmp.id IN (SELECT DISTINCT d.id FROM ${stats_db_name}.datasource_tmp d, ${stats_db_name}.result_datasources rd WHERE d.id=rd.datasource); + +-- Project temporary table update and final project table creation with final updates that can not be applied to ORC tables +UPDATE ${stats_db_name}.project_tmp SET haspubs='yes' WHERE project_tmp.id IN (SELECT pr.id FROM ${stats_db_name}.project_results pr, ${stats_db_name}.result r WHERE pr.result=r.id AND r.type='publication'); + +DROP TABLE IF EXISTS ${stats_db_name}.project; +CREATE TABLE ${stats_db_name}.project stored as parquet as +SELECT p.id , p.acronym, p.title, p.funder, p.funding_lvl0, p.funding_lvl1, p.funding_lvl2, p.ec39, p.type, p.startdate, p.enddate, p.start_year, p.end_year, p.duration, +CASE WHEN prr1.id IS NULL THEN 'no' ELSE 'yes' END AS haspubs, +CASE WHEN prr1.id IS NULL THEN 0 ELSE prr1.np END AS numpubs, +CASE WHEN prr2.id IS NULL THEN 0 ELSE prr2.daysForlastPub END AS daysforlastpub, +CASE WHEN prr2.id IS NULL THEN 0 ELSE prr2.dp END AS delayedpubs, +p.callidentifier, p.code +FROM ${stats_db_name}.project_tmp p +LEFT JOIN (SELECT pr.id, count(distinct pr.result) AS np + FROM ${stats_db_name}.project_results pr INNER JOIN ${stats_db_name}.result r ON pr.result=r.id + WHERE r.type='publication' + GROUP BY pr.id) AS prr1 on prr1.id = p.id +LEFT JOIN (SELECT pp.id, max(datediff(to_date(r.date), to_date(pp.enddate)) ) AS daysForlastPub , count(distinct r.id) AS dp + FROM ${stats_db_name}.project_tmp pp, ${stats_db_name}.project_results pr, ${stats_db_name}.result r + WHERE pp.id=pr.id AND pr.result=r.id AND r.type='publication' AND datediff(to_date(r.date), to_date(pp.enddate)) > 0 + GROUP BY pp.id) AS prr2 + ON prr2.id = p.id; + +-- Publication temporary table updates +UPDATE ${stats_db_name}.publication_tmp SET delayed = 'yes' WHERE publication_tmp.id IN (SELECT distinct r.id FROM stats_wf_db_obs.result r, ${stats_db_name}.project_results pr, ${stats_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); + +-- Dataset temporary table updates +UPDATE ${stats_db_name}.dataset_tmp SET delayed = 'yes' WHERE dataset_tmp.id IN (SELECT distinct r.id FROM stats_wf_db_obs.result r, ${stats_db_name}.project_results pr, ${stats_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); + +-- Software temporary table updates +UPDATE ${stats_db_name}.software_tmp SET delayed = 'yes' WHERE software_tmp.id IN (SELECT distinct r.id FROM ${stats_db_name}.result r, ${stats_db_name}.project_results pr, ${stats_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); + +-- Oherresearchproduct temporary table updates +UPDATE ${stats_db_name}.otherresearchproduct_tmp SET delayed = 'yes' WHERE otherresearchproduct_tmp.id IN (SELECT distinct r.id FROM ${stats_db_name}.result r, ${stats_db_name}.project_results pr, ${stats_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); + +CREATE OR REPLACE VIEW ${stats_db_name}.project_results_publication AS SELECT result_projects.id AS result, result_projects.project AS project_results, result.date as resultdate, project.enddate as projectenddate, result_projects.daysfromend AS daysfromend FROM ${stats_db_name}.result_projects, ${stats_db_name}.result, ${stats_db_name}.project WHERE result_projects.id=result.id AND result.type='publication' AND project.id=result_projects.project; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql index c18e2e1f8..25439852e 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql @@ -1,42 +1,38 @@ ----------------------------- --- Post processing - Updates ----------------------------- +------------------------------------------------------------------------------------------------------ +-- Creating parquet tables from the updated temporary tables and removing unnecessary temporary tables +------------------------------------------------------------------------------------------------------ ---Datasource temporary table updates -UPDATE ${stats_db_name}.datasource_tmp SET harvested='true' WHERE datasource_tmp.id IN (SELECT DISTINCT d.id FROM ${stats_db_name}.datasource_tmp d, ${stats_db_name}.result_datasources rd WHERE d.id=rd.datasource); +DROP TABLE IF EXISTS ${stats_db_name}.datasource; +CREATE TABLE ${stats_db_name}.datasource stored AS parquet AS SELECT * FROM ${stats_db_name}.datasource_tmp; --- Project temporary table update and final project table creation with final updates that can not be applied to ORC tables -UPDATE ${stats_db_name}.project_tmp SET haspubs='yes' WHERE project_tmp.id IN (SELECT pr.id FROM ${stats_db_name}.project_results pr, ${stats_db_name}.result r WHERE pr.result=r.id AND r.type='publication'); +DROP TABLE IF EXISTS ${stats_db_name}.publication; +CREATE TABLE ${stats_db_name}.publication stored AS parquet AS SELECT * FROM ${stats_db_name}.publication_tmp; -DROP TABLE IF EXISTS ${stats_db_name}.project; -CREATE TABLE ${stats_db_name}.project stored as parquet as -SELECT p.id , p.acronym, p.title, p.funder, p.funding_lvl0, p.funding_lvl1, p.funding_lvl2, p.ec39, p.type, p.startdate, p.enddate, p.start_year, p.end_year, p.duration, -CASE WHEN prr1.id IS NULL THEN 'no' ELSE 'yes' END AS haspubs, -CASE WHEN prr1.id IS NULL THEN 0 ELSE prr1.np END AS numpubs, -CASE WHEN prr2.id IS NULL THEN 0 ELSE prr2.daysForlastPub END AS daysforlastpub, -CASE WHEN prr2.id IS NULL THEN 0 ELSE prr2.dp END AS delayedpubs, -p.callidentifier, p.code -FROM ${stats_db_name}.project_tmp p -LEFT JOIN (SELECT pr.id, count(distinct pr.result) AS np - FROM ${stats_db_name}.project_results pr INNER JOIN ${stats_db_name}.result r ON pr.result=r.id - WHERE r.type='publication' - GROUP BY pr.id) AS prr1 on prr1.id = p.id -LEFT JOIN (SELECT pp.id, max(datediff(to_date(r.date), to_date(pp.enddate)) ) AS daysForlastPub , count(distinct r.id) AS dp - FROM ${stats_db_name}.project_tmp pp, ${stats_db_name}.project_results pr, ${stats_db_name}.result r - WHERE pp.id=pr.id AND pr.result=r.id AND r.type='publication' AND datediff(to_date(r.date), to_date(pp.enddate)) > 0 - GROUP BY pp.id) AS prr2 - ON prr2.id = p.id; - --- Publication temporary table updates -UPDATE ${stats_db_name}.publication_tmp SET delayed = 'yes' WHERE publication_tmp.id IN (SELECT distinct r.id FROM stats_wf_db_obs.result r, ${stats_db_name}.project_results pr, ${stats_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); +DROP TABLE IF EXISTS ${stats_db_name}.dataset; +CREATE TABLE ${stats_db_name}.dataset stored AS parquet AS SELECT * FROM ${stats_db_name}.dataset_tmp; --- Dataset temporary table updates -UPDATE ${stats_db_name}.dataset_tmp SET delayed = 'yes' WHERE dataset_tmp.id IN (SELECT distinct r.id FROM stats_wf_db_obs.result r, ${stats_db_name}.project_results pr, ${stats_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); +DROP TABLE IF EXISTS ${stats_db_name}.software; +CREATE TABLE ${stats_db_name}.software stored AS parquet AS SELECT * FROM ${stats_db_name}.software_tmp; --- Software temporary table updates -UPDATE ${stats_db_name}.software_tmp SET delayed = 'yes' WHERE software_tmp.id IN (SELECT distinct r.id FROM ${stats_db_name}.result r, ${stats_db_name}.project_results pr, ${stats_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct; +CREATE TABLE ${stats_db_name}.otherresearchproduct stored AS parquet AS SELECT * FROM ${stats_db_name}.otherresearchproduct_tmp; --- Oherresearchproduct temporary table updates -UPDATE ${stats_db_name}.otherresearchproduct_tmp SET delayed = 'yes' WHERE otherresearchproduct_tmp.id IN (SELECT distinct r.id FROM ${stats_db_name}.result r, ${stats_db_name}.project_results pr, ${stats_db_name}.project_tmp p WHERE r.id=pr.result AND pr.id=p.id AND to_date(r.date)-to_date(p.enddate) > 0); +DROP TABLE ${stats_db_name}.project_tmp; +DROP TABLE ${stats_db_name}.datasource_tmp; +DROP TABLE ${stats_db_name}.publication_tmp; +DROP TABLE ${stats_db_name}.dataset_tmp; +DROP TABLE ${stats_db_name}.software_tmp; +DROP TABLE ${stats_db_name}.otherresearchproduct_tmp; -CREATE OR REPLACE VIEW ${stats_db_name}.project_results_publication AS SELECT result_projects.id AS result, result_projects.project AS project_results, result.date as resultdate, project.enddate as projectenddate, result_projects.daysfromend AS daysfromend FROM ${stats_db_name}.result_projects, ${stats_db_name}.result, ${stats_db_name}.project WHERE result_projects.id=result.id AND result.type='publication' AND project.id=result_projects.project; +---------------------------------------------- +-- Re-creating views from final parquet tables +--------------------------------------------- + +-- Result +CREATE OR REPLACE VIEW ${stats_db_name}.result AS SELECT *, bestlicence AS access_mode FROM ${stats_db_name}.publication UNION ALL SELECT *, bestlicence as access_mode FROM ${stats_db_name}.software UNION ALL SELECT *, bestlicence AS access_mode FROM ${stats_db_name}.dataset UNION ALL SELECT *, bestlicence AS access_mode FROM ${stats_db_name}.otherresearchproduct; + + +------------------------------------------------------------------------------- +-- To see with Antonis if the following is needed and where it should be placed +------------------------------------------------------------------------------- +CREATE TABLE ${stats_db_name}.numbers_country AS SELECT org.country AS country, count(distinct rd.datasource) AS datasources, count(distinct r.id) AS publications FROM ${stats_db_name}.result r, ${stats_db_name}.result_datasources rd, ${stats_db_name}.datasource d, ${stats_db_name}.datasource_organizations dor, ${stats_db_name}.organization org WHERE r.id=rd.id AND rd.datasource=d.id AND d.id=dor.id AND dor.organization=org.id AND r.type='publication' AND r.bestlicence='Open Access' GROUP BY org.country; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql index 25439852e..795770313 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql @@ -1,38 +1,59 @@ ------------------------------------------------------------------------------------------------------- --- Creating parquet tables from the updated temporary tables and removing unnecessary temporary tables ------------------------------------------------------------------------------------------------------- +------------------------------------------------------ +------------------------------------------------------ +-- Additional relations +-- +-- Sources related tables/views +------------------------------------------------------ +------------------------------------------------------ +CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_sources as +SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource +FROM ( + SELECT substr(p.id, 4) as id, substr(datasource, 4) as datasource +from ${openaire_db_name}.publication p lateral view explode(p.collectedfrom.key) c as datasource) p +LEFT OUTER JOIN +( + SELECT substr(d.id, 4) id + from ${openaire_db_name}.datasource d + WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; -DROP TABLE IF EXISTS ${stats_db_name}.datasource; -CREATE TABLE ${stats_db_name}.datasource stored AS parquet AS SELECT * FROM ${stats_db_name}.datasource_tmp; - -DROP TABLE IF EXISTS ${stats_db_name}.publication; -CREATE TABLE ${stats_db_name}.publication stored AS parquet AS SELECT * FROM ${stats_db_name}.publication_tmp; - -DROP TABLE IF EXISTS ${stats_db_name}.dataset; -CREATE TABLE ${stats_db_name}.dataset stored AS parquet AS SELECT * FROM ${stats_db_name}.dataset_tmp; - -DROP TABLE IF EXISTS ${stats_db_name}.software; -CREATE TABLE ${stats_db_name}.software stored AS parquet AS SELECT * FROM ${stats_db_name}.software_tmp; - -DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct; -CREATE TABLE ${stats_db_name}.otherresearchproduct stored AS parquet AS SELECT * FROM ${stats_db_name}.otherresearchproduct_tmp; - -DROP TABLE ${stats_db_name}.project_tmp; -DROP TABLE ${stats_db_name}.datasource_tmp; -DROP TABLE ${stats_db_name}.publication_tmp; -DROP TABLE ${stats_db_name}.dataset_tmp; -DROP TABLE ${stats_db_name}.software_tmp; -DROP TABLE ${stats_db_name}.otherresearchproduct_tmp; - ----------------------------------------------- --- Re-creating views from final parquet tables ---------------------------------------------- - --- Result -CREATE OR REPLACE VIEW ${stats_db_name}.result AS SELECT *, bestlicence AS access_mode FROM ${stats_db_name}.publication UNION ALL SELECT *, bestlicence as access_mode FROM ${stats_db_name}.software UNION ALL SELECT *, bestlicence AS access_mode FROM ${stats_db_name}.dataset UNION ALL SELECT *, bestlicence AS access_mode FROM ${stats_db_name}.otherresearchproduct; - - -------------------------------------------------------------------------------- --- To see with Antonis if the following is needed and where it should be placed -------------------------------------------------------------------------------- -CREATE TABLE ${stats_db_name}.numbers_country AS SELECT org.country AS country, count(distinct rd.datasource) AS datasources, count(distinct r.id) AS publications FROM ${stats_db_name}.result r, ${stats_db_name}.result_datasources rd, ${stats_db_name}.datasource d, ${stats_db_name}.datasource_organizations dor, ${stats_db_name}.organization org WHERE r.id=rd.id AND rd.datasource=d.id AND d.id=dor.id AND dor.organization=org.id AND r.type='publication' AND r.bestlicence='Open Access' GROUP BY org.country; +CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_sources as +SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource +FROM ( + SELECT substr(p.id, 4) as id, substr(datasource, 4) as datasource +from ${openaire_db_name}.dataset p lateral view explode(p.collectedfrom.key) c as datasource) p +LEFT OUTER JOIN +( + SELECT substr(d.id, 4) id + from ${openaire_db_name}.datasource d + WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; + +CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_sources as +SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource +FROM ( + SELECT substr(p.id, 4) as id, substr(datasource, 4) as datasource +from ${openaire_db_name}.software p lateral view explode(p.collectedfrom.key) c as datasource) p +LEFT OUTER JOIN +( + SELECT substr(d.id, 4) id + from ${openaire_db_name}.datasource d + WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; + +CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_sources as +SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource +FROM ( + SELECT substr(p.id, 4) as id, substr(datasource, 4) as datasource +from ${openaire_db_name}.otherresearchproduct p lateral view explode(p.collectedfrom.key) c as datasource) p +LEFT OUTER JOIN +( + SELECT substr(d.id, 4) id + from ${openaire_db_name}.datasource d + WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; + +CREATE VIEW IF NOT EXISTS ${stats_db_name}.result_sources AS +SELECT * FROM ${stats_db_name}.publication_sources +UNION ALL +SELECT * FROM ${stats_db_name}.dataset_sources +UNION ALL +SELECT * FROM ${stats_db_name}.software_sources +UNION ALL +SELECT * FROM ${stats_db_name}.otherresearchproduct_sources; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql index 06d58fd58..4a56b5d68 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql @@ -1,47 +1,49 @@ -COMPUTE STATS dataset; -COMPUTE STATS dataset_citations; -COMPUTE STATS dataset_classifications; -COMPUTE STATS dataset_concepts; -COMPUTE STATS dataset_datasources; -COMPUTE STATS dataset_languages; -COMPUTE STATS dataset_oids; -COMPUTE STATS dataset_pids; -COMPUTE STATS dataset_topics; -COMPUTE STATS datasource; -COMPUTE STATS datasource_languages; -COMPUTE STATS datasource_oids; -COMPUTE STATS datasource_organizations; -COMPUTE STATS numbers_country; -COMPUTE STATS organization; -COMPUTE STATS otherresearchproduct; -COMPUTE STATS otherresearchproduct_citations; -COMPUTE STATS otherresearchproduct_classifications; -COMPUTE STATS otherresearchproduct_concepts; -COMPUTE STATS otherresearchproduct_datasources; -COMPUTE STATS otherresearchproduct_languages; -COMPUTE STATS otherresearchproduct_oids; -COMPUTE STATS otherresearchproduct_pids; -COMPUTE STATS otherresearchproduct_topics; -COMPUTE STATS project; -COMPUTE STATS project_oids; -COMPUTE STATS project_organizations; -COMPUTE STATS project_results; -COMPUTE STATS publication; -COMPUTE STATS publication_citations; -COMPUTE STATS publication_classifications; -COMPUTE STATS publication_concepts; -COMPUTE STATS publication_datasources; -COMPUTE STATS publication_languages; -COMPUTE STATS publication_oids; -COMPUTE STATS publication_pids; -COMPUTE STATS publication_topics; -COMPUTE STATS result_organization; -COMPUTE STATS result_projects; -COMPUTE STATS software; -COMPUTE STATS software_citations; -COMPUTE STATS software_classifications; -COMPUTE STATS software_concepts; -COMPUTE STATS software_datasources; -COMPUTE STATS software_languages; -COMPUTE STATS software_oids; -COMPUTE STATS software_pids; +------------------------------------------------------ +------------------------------------------------------ +-- Additional relations +-- +-- Licences related tables/views +------------------------------------------------------ +------------------------------------------------------ +CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_licenses AS +SELECT substr(p.id, 4) as id, licenses.value as type +from ${openaire_db_name}.publication p LATERAL VIEW explode(p.instance.license) instances as licenses +where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false; + +CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_licenses AS +SELECT substr(p.id, 4) as id, licenses.value as type +from ${openaire_db_name}.dataset p LATERAL VIEW explode(p.instance.license) instances as licenses +where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false; + +CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_licenses AS +SELECT substr(p.id, 4) as id, licenses.value as type +from ${openaire_db_name}.software p LATERAL VIEW explode(p.instance.license) instances as licenses +where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false; + +CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_licenses AS +SELECT substr(p.id, 4) as id, licenses.value as type +from ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.instance.license) instances as licenses +where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false; + +CREATE VIEW IF NOT EXISTS ${stats_db_name}.result_licenses AS +SELECT * FROM ${stats_db_name}.publication_licenses +UNION ALL +SELECT * FROM ${stats_db_name}.dataset_licenses +UNION ALL +SELECT * FROM ${stats_db_name}.software_licenses +UNION ALL +SELECT * FROM ${stats_db_name}.otherresearchproduct_licenses; + +CREATE TABLE IF NOT EXISTS ${stats_db_name}.organization_pids AS +select substr(o.id, 4) as id, ppid.qualifier.classname as type, ppid.value as pid +from ${openaire_db_name}.organization o lateral view explode(o.pid) pids as ppid; + +CREATE TABLE IF NOT EXISTS ${stats_db_name}.organization_sources as +SELECT o.id, case when d.id is null then 'other' else o.datasource end as datasource +FROM ( + SELECT substr(o.id, 4) as id, substr(instances.instance.key, 4) as datasource + from ${openaire_db_name}.organization o lateral view explode(o.collectedfrom) instances as instance) o + LEFT OUTER JOIN ( + SELECT substr(d.id, 4) id + from ${openaire_db_name}.datasource d + WHERE d.datainfo.deletedbyinference=false) d on o.datasource = d.id; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql new file mode 100644 index 000000000..9dacb92ce --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql @@ -0,0 +1,35 @@ +------------------------------------------------------ +------------------------------------------------------ +-- Additional relations +-- +-- Refereed related tables/views +------------------------------------------------------ +------------------------------------------------------ +CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_refereed as +select substr(r.id, 4) as id, inst.refereed.value as refereed +from ${openaire_db_name}.publication r lateral view explode(r.instance) instances as inst +where r.datainfo.deletedbyinference=false; + +CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_refereed as +select substr(r.id, 4) as id, inst.refereed.value as refereed +from ${openaire_db_name}.dataset r lateral view explode(r.instance) instances as inst +where r.datainfo.deletedbyinference=false; + +CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_refereed as +select substr(r.id, 4) as id, inst.refereed.value as refereed +from ${openaire_db_name}.software r lateral view explode(r.instance) instances as inst +where r.datainfo.deletedbyinference=false; + +CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_refereed as +select substr(r.id, 4) as id, inst.refereed.value as refereed +from ${openaire_db_name}.otherresearchproduct r lateral view explode(r.instance) instances as inst +where r.datainfo.deletedbyinference=false; + +CREATE VIEW IF NOT EXISTS ${stats_db_name}.result_refereed as +select * from ${stats_db_name}.publication_refereed +union all +select * from ${stats_db_name}.dataset_refereed +union all +select * from ${stats_db_name}.software_refereed +union all +select * from ${stats_db_name}.otherresearchproduct_refereed; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql new file mode 100644 index 000000000..9e04bb13b --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql @@ -0,0 +1,160 @@ +------------------------------------------------------ +------------------------------------------------------ +-- Shadow schema table exchange +------------------------------------------------------ +------------------------------------------------------ + +-- Dropping old views +DROP VIEW IF EXISTS ${stats_db_shadow_name}.country; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.countrygdp; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_citations; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_classifications; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_concepts; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_datasources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_languages; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_licenses; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_oids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_pids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_sources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_topics; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.datasource; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.datasource_languages; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.datasource_oids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.datasource_organizations; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.datasource_results; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.fundref; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.numbers_country; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.organization; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.organization_datasources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.organization_projects; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_citations; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_classifications; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_concepts; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_datasources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_languages; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_licenses; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_oids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_pids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_sources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_topics; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.project; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.project_oids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.project_organizations; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.project_results; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_citations; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_classifications; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_concepts; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_datasources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_languages; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_licenses; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_oids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_pids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_sources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_topics; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_citations; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_classifications; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_concepts; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_datasources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_languages; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_licenses; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_oids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_organization; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_pids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_projects; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_sources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_topics; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.rndexpediture; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.roarmap; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_citations; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_classifications; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_concepts; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_datasources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_languages; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_licenses; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_oids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_pids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_sources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_topics; + + +-- Creating new views +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.country AS SELECT * FROM ${stats_db_name}2.country; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.countrygdp AS SELECT * FROM ${stats_db_name}2.countrygdp; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset AS SELECT * FROM ${stats_db_name}2.dataset; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_citations AS SELECT * FROM ${stats_db_name}2.dataset_citations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_classifications AS SELECT * FROM ${stats_db_name}2.dataset_classifications; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_concepts AS SELECT * FROM ${stats_db_name}2.dataset_concepts; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_datasources AS SELECT * FROM ${stats_db_name}2.dataset_datasources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_languages AS SELECT * FROM ${stats_db_name}2.dataset_languages; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_licenses AS SELECT * FROM ${stats_db_name}2.dataset_licenses; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_oids AS SELECT * FROM ${stats_db_name}2.dataset_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_pids AS SELECT * FROM ${stats_db_name}2.dataset_pids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_sources AS SELECT * FROM ${stats_db_name}2.dataset_sources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_topics AS SELECT * FROM ${stats_db_name}2.dataset_topics; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource AS SELECT * FROM ${stats_db_name}2.datasource; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_languages AS SELECT * FROM ${stats_db_name}2.datasource_languages; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_oids AS SELECT * FROM ${stats_db_name}2.datasource_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_organizations AS SELECT * FROM ${stats_db_name}2.datasource_organizations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_results AS SELECT * FROM ${stats_db_name}2.datasource_results; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.fundref AS SELECT * FROM ${stats_db_name}2.fundref; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.numbers_country AS SELECT * FROM ${stats_db_name}2.numbers_country; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.organization AS SELECT * FROM ${stats_db_name}2.organization; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.organization_datasources AS SELECT * FROM ${stats_db_name}2.organization_datasources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.organization_projects AS SELECT * FROM ${stats_db_name}2.organization_projects; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct AS SELECT * FROM ${stats_db_name}2.otherresearchproduct; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_citations AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_citations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_classifications AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_classifications; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_concepts AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_concepts; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_datasources AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_datasources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_languages AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_languages; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_licenses AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_licenses; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_oids AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_pids AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_pids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_sources AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_sources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_topics AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_topics; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project AS SELECT * FROM ${stats_db_name}2.project; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project_oids AS SELECT * FROM ${stats_db_name}2.project_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project_organizations AS SELECT * FROM ${stats_db_name}2.project_organizations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project_results AS SELECT * FROM ${stats_db_name}2.project_results; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication AS SELECT * FROM ${stats_db_name}2.publication; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_citations AS SELECT * FROM ${stats_db_name}2.publication_citations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_classifications AS SELECT * FROM ${stats_db_name}2.publication_classifications; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_concepts AS SELECT * FROM ${stats_db_name}2.publication_concepts; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_datasources AS SELECT * FROM ${stats_db_name}2.publication_datasources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_languages AS SELECT * FROM ${stats_db_name}2.publication_languages; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_licenses AS SELECT * FROM ${stats_db_name}2.publication_licenses; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_oids AS SELECT * FROM ${stats_db_name}2.publication_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_pids AS SELECT * FROM ${stats_db_name}2.publication_pids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_sources AS SELECT * FROM ${stats_db_name}2.publication_sources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_topics AS SELECT * FROM ${stats_db_name}2.publication_topics; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result AS SELECT * FROM ${stats_db_name}2.result; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_citations AS SELECT * FROM ${stats_db_name}2.result_citations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_classifications AS SELECT * FROM ${stats_db_name}2.result_classifications; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_concepts AS SELECT * FROM ${stats_db_name}2.result_concepts; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_datasources AS SELECT * FROM ${stats_db_name}2.result_datasources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_languages AS SELECT * FROM ${stats_db_name}2.result_languages; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_licenses AS SELECT * FROM ${stats_db_name}2.result_licenses; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_oids AS SELECT * FROM ${stats_db_name}2.result_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_organization AS SELECT * FROM ${stats_db_name}2.result_organization; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_pids AS SELECT * FROM ${stats_db_name}2.result_pids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_projects AS SELECT * FROM ${stats_db_name}2.result_projects; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_sources AS SELECT * FROM ${stats_db_name}2.result_sources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_topics AS SELECT * FROM ${stats_db_name}2.result_topics; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.rndexpediture AS SELECT * FROM ${stats_db_name}2.rndexpediture; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.roarmap AS SELECT * FROM ${stats_db_name}2.roarmap; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software AS SELECT * FROM ${stats_db_name}2.software; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_citations AS SELECT * FROM ${stats_db_name}2.software_citations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_classifications AS SELECT * FROM ${stats_db_name}2.software_classifications; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_concepts AS SELECT * FROM ${stats_db_name}2.software_concepts; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_datasources AS SELECT * FROM ${stats_db_name}2.software_datasources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_languages AS SELECT * FROM ${stats_db_name}2.software_languages; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_licenses AS SELECT * FROM ${stats_db_name}2.software_licenses; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_oids AS SELECT * FROM ${stats_db_name}2.software_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_pids AS SELECT * FROM ${stats_db_name}2.software_pids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_sources AS SELECT * FROM ${stats_db_name}2.software_sources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_topics AS SELECT * FROM ${stats_db_name}2.software_topics; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step17.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step17.sql new file mode 100644 index 000000000..bfa7fdf5c --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step17.sql @@ -0,0 +1,54 @@ +------------------------------------------------------ +------------------------------------------------------ +-- Impala table statistics - Needed to make the tables +-- visible for impala +------------------------------------------------------ +------------------------------------------------------ + +COMPUTE STATS dataset; +COMPUTE STATS dataset_citations; +COMPUTE STATS dataset_classifications; +COMPUTE STATS dataset_concepts; +COMPUTE STATS dataset_datasources; +COMPUTE STATS dataset_languages; +COMPUTE STATS dataset_oids; +COMPUTE STATS dataset_pids; +COMPUTE STATS dataset_topics; +COMPUTE STATS datasource; +COMPUTE STATS datasource_languages; +COMPUTE STATS datasource_oids; +COMPUTE STATS datasource_organizations; +COMPUTE STATS numbers_country; +COMPUTE STATS organization; +COMPUTE STATS otherresearchproduct; +COMPUTE STATS otherresearchproduct_citations; +COMPUTE STATS otherresearchproduct_classifications; +COMPUTE STATS otherresearchproduct_concepts; +COMPUTE STATS otherresearchproduct_datasources; +COMPUTE STATS otherresearchproduct_languages; +COMPUTE STATS otherresearchproduct_oids; +COMPUTE STATS otherresearchproduct_pids; +COMPUTE STATS otherresearchproduct_topics; +COMPUTE STATS project; +COMPUTE STATS project_oids; +COMPUTE STATS project_organizations; +COMPUTE STATS project_results; +COMPUTE STATS publication; +COMPUTE STATS publication_citations; +COMPUTE STATS publication_classifications; +COMPUTE STATS publication_concepts; +COMPUTE STATS publication_datasources; +COMPUTE STATS publication_languages; +COMPUTE STATS publication_oids; +COMPUTE STATS publication_pids; +COMPUTE STATS publication_topics; +COMPUTE STATS result_organization; +COMPUTE STATS result_projects; +COMPUTE STATS software; +COMPUTE STATS software_citations; +COMPUTE STATS software_classifications; +COMPUTE STATS software_concepts; +COMPUTE STATS software_datasources; +COMPUTE STATS software_languages; +COMPUTE STATS software_oids; +COMPUTE STATS software_pids; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql index 7e0a07184..312a8b82e 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql @@ -1,6 +1,6 @@ -------------------------------------------------------------- -------------------------------------------------------------- --- 2. Publication table/view and Publication related tables/views +-- Publication table/view and Publication related tables/views -------------------------------------------------------------- -------------------------------------------------------------- @@ -30,3 +30,6 @@ CREATE TABLE ${stats_db_name}.publication_oids AS SELECT substr(p.id, 4) AS id, CREATE TABLE ${stats_db_name}.publication_pids AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value as pid FROM ${openaire_db_name}.publication p LATERAL VIEW explode(p.pid) pids AS ppid; CREATE TABLE ${stats_db_name}.publication_topics as select substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS TYPE, subjects.subject.value AS topic FROM ${openaire_db_name}.publication p LATERAL VIEW explode(p.subject) subjects AS subject; + +-- Publication_citations +CREATE TABLE ${stats_db_name}.publication_citations AS SELECT substr(p.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS result FROM ${openaire_db_name}.publication p lateral view explode(p.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql index c1345d1ba..47a102525 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql @@ -1,2 +1,36 @@ --- 3. Publication_citations -CREATE TABLE ${stats_db_name}.publication_citations AS SELECT substr(p.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS result FROM ${openaire_db_name}.publication p lateral view explode(p.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; \ No newline at end of file +------------------------------------------------------ +------------------------------------------------------ +-- Dataset table/view and Dataset related tables/views +------------------------------------------------------ +------------------------------------------------------ + +-- Dataset temporary table supporting updates +DROP TABLE IF EXISTS ${stats_db_name}.dataset_tmp; +CREATE TABLE ${stats_db_name}.dataset_tmp (id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) into 100 buckets stored AS orc tblproperties('transactional'='true'); + +INSERT INTO ${stats_db_name}.dataset_tmp SELECT substr(d.id, 4) AS id, d.title[0].value AS title, d.publisher.value AS publisher, cast(null AS string) AS journal, +d.dateofacceptance.value as date, date_format(d.dateofacceptance.value,'yyyy') AS year, d.bestaccessright.classname AS bestlicence, +d.embargoenddate.value AS embargo_end_date, false AS delayed, size(d.author) AS authors , concat_ws('\u003B',d.source.value) AS source, + CASE WHEN SIZE(d.description) > 0 THEN TRUE ELSE FALSE end AS abstract, +'dataset' AS type +FROM ${openaire_db_name}.dataset d +WHERE d.datainfo.deletedbyinference=FALSE; + +-- Dataset_citations +CREATE TABLE ${stats_db_name}.dataset_citations AS SELECT substr(d.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS result FROM ${openaire_db_name}.dataset d LATERAL VIEW explode(d.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; + +CREATE TABLE ${stats_db_name}.dataset_classifications AS SELECT substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype; + +CREATE TABLE ${stats_db_name}.dataset_concepts AS SELECT substr(p.id, 4) as id, contexts.context.id as concept from ${openaire_db_name}.dataset p LATERAL VIEW explode(p.context) contexts as context; + +CREATE TABLE ${stats_db_name}.dataset_datasources AS SELECT p.id, case when d.id IS NULL THEN 'other' ELSE p.datasource END AS datasource FROM (SELECT substr(p.id, 4) as id, substr(instances.instance.hostedby.key, 4) AS datasource +FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.instance) instances AS instance) p LEFT OUTER JOIN +(SELECT substr(d.id, 4) id FROM ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d ON p.datasource = d.id; + +CREATE TABLE ${stats_db_name}.dataset_languages AS SELECT substr(p.id, 4) AS id, p.language.classname AS language FROM ${openaire_db_name}.dataset p; + +CREATE TABLE ${stats_db_name}.dataset_oids AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.originalid) oids AS ids; + +CREATE TABLE ${stats_db_name}.dataset_pids AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.pid) pids AS ppid; + +CREATE TABLE ${stats_db_name}.dataset_topics AS SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.subject) subjects AS subject; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql index d7202c2de..ca1059cc8 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql @@ -1,36 +1,36 @@ ------------------------------------------------------- ------------------------------------------------------- --- 4. Dataset table/view and Dataset related tables/views ------------------------------------------------------- ------------------------------------------------------- +-------------------------------------------------------- +-------------------------------------------------------- +-- Software table/view and Software related tables/views +-------------------------------------------------------- +-------------------------------------------------------- --- Dataset temporary table supporting updates -DROP TABLE IF EXISTS ${stats_db_name}.dataset_tmp; -CREATE TABLE ${stats_db_name}.dataset_tmp (id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) into 100 buckets stored AS orc tblproperties('transactional'='true'); +-- Software temporary table supporting updates +DROP TABLE IF EXISTS ${stats_db_name}.software_tmp; +CREATE TABLE ${stats_db_name}.software_tmp (id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) INTO 100 buckets stored AS orc tblproperties('transactional'='true'); -INSERT INTO ${stats_db_name}.dataset_tmp SELECT substr(d.id, 4) AS id, d.title[0].value AS title, d.publisher.value AS publisher, cast(null AS string) AS journal, -d.dateofacceptance.value as date, date_format(d.dateofacceptance.value,'yyyy') AS year, d.bestaccessright.classname AS bestlicence, -d.embargoenddate.value AS embargo_end_date, false AS delayed, size(d.author) AS authors , concat_ws('\u003B',d.source.value) AS source, - CASE WHEN SIZE(d.description) > 0 THEN TRUE ELSE FALSE end AS abstract, -'dataset' AS type -FROM ${openaire_db_name}.dataset d -WHERE d.datainfo.deletedbyinference=FALSE; +INSERT INTO ${stats_db_name}.software_tmp SELECT substr(s.id, 4) as id, s.title[0].value AS title, s.publisher.value AS publisher, CAST(NULL AS string) AS journal, +s.dateofacceptance.value AS DATE, date_format(s.dateofacceptance.value,'yyyy') AS YEAR, s.bestaccessright.classname AS bestlicence, +s.embargoenddate.value AS embargo_end_date, FALSE AS delayed, SIZE(s.author) AS authors , concat_ws('\u003B',s.source.value) AS source, + CASE WHEN SIZE(s.description) > 0 THEN TRUE ELSE FALSE END AS abstract, +'software' as type +from ${openaire_db_name}.software s +where s.datainfo.deletedbyinference=false; --- Dataset_citations -CREATE TABLE ${stats_db_name}.dataset_citations AS SELECT substr(d.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS result FROM ${openaire_db_name}.dataset d LATERAL VIEW explode(d.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; +-- Software_citations +CREATE TABLE ${stats_db_name}.software_citations AS SELECT substr(s.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS RESULT FROM ${openaire_db_name}.software s LATERAL VIEW explode(s.extrainfo) citations as citation where xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; -CREATE TABLE ${stats_db_name}.dataset_classifications AS SELECT substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype; +CREATE TABLE ${stats_db_name}.software_classifications AS SELECT substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype; -CREATE TABLE ${stats_db_name}.dataset_concepts AS SELECT substr(p.id, 4) as id, contexts.context.id as concept from ${openaire_db_name}.dataset p LATERAL VIEW explode(p.context) contexts as context; +CREATE TABLE ${stats_db_name}.software_concepts AS SELECT substr(p.id, 4) AS id, contexts.context.id AS concept FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.context) contexts AS context; -CREATE TABLE ${stats_db_name}.dataset_datasources AS SELECT p.id, case when d.id IS NULL THEN 'other' ELSE p.datasource END AS datasource FROM (SELECT substr(p.id, 4) as id, substr(instances.instance.hostedby.key, 4) AS datasource -FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.instance) instances AS instance) p LEFT OUTER JOIN +CREATE TABLE ${stats_db_name}.software_datasources AS SELECT p.id, CASE WHEN d.id IS NULL THEN 'other' ELSE p.datasource end as datasource FROM (SELECT substr(p.id, 4) AS id, substr(instances.instance.hostedby.key, 4) AS datasource +FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.instance) instances AS instance) p LEFT OUTER JOIN (SELECT substr(d.id, 4) id FROM ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d ON p.datasource = d.id; -CREATE TABLE ${stats_db_name}.dataset_languages AS SELECT substr(p.id, 4) AS id, p.language.classname AS language FROM ${openaire_db_name}.dataset p; +CREATE TABLE ${stats_db_name}.software_languages AS select substr(p.id, 4) AS id, p.language.classname AS language FROM ${openaire_db_name}.software p; -CREATE TABLE ${stats_db_name}.dataset_oids AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.originalid) oids AS ids; +CREATE TABLE ${stats_db_name}.software_oids AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.originalid) oids AS ids; -CREATE TABLE ${stats_db_name}.dataset_pids AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.pid) pids AS ppid; +CREATE TABLE ${stats_db_name}.software_pids AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.pid) pids AS ppid; -CREATE TABLE ${stats_db_name}.dataset_topics AS SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.subject) subjects AS subject; +CREATE TABLE ${stats_db_name}.software_topics AS SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.subject) subjects AS subject; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql index b2b3c4c75..b4fb5aec6 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql @@ -1,36 +1,37 @@ --------------------------------------------------------- --------------------------------------------------------- --- 5. Software table/view and Software related tables/views --------------------------------------------------------- --------------------------------------------------------- +-------------------------------------------------------------------------------- +-------------------------------------------------------------------------------- +-- Otherresearchproduct table/view and Otherresearchproduct related tables/views +-------------------------------------------------------------------------------- +-------------------------------------------------------------------------------- --- Software temporary table supporting updates -DROP TABLE IF EXISTS ${stats_db_name}.software_tmp; -CREATE TABLE ${stats_db_name}.software_tmp (id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) clustered by (id) INTO 100 buckets stored AS orc tblproperties('transactional'='true'); +-- Otherresearchproduct temporary table supporting updates +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_tmp; +CREATE TABLE ${stats_db_name}.otherresearchproduct_tmp ( id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties('transactional'='true'); -INSERT INTO ${stats_db_name}.software_tmp SELECT substr(s.id, 4) as id, s.title[0].value AS title, s.publisher.value AS publisher, CAST(NULL AS string) AS journal, -s.dateofacceptance.value AS DATE, date_format(s.dateofacceptance.value,'yyyy') AS YEAR, s.bestaccessright.classname AS bestlicence, -s.embargoenddate.value AS embargo_end_date, FALSE AS delayed, SIZE(s.author) AS authors , concat_ws('\u003B',s.source.value) AS source, - CASE WHEN SIZE(s.description) > 0 THEN TRUE ELSE FALSE END AS abstract, -'software' as type -from ${openaire_db_name}.software s -where s.datainfo.deletedbyinference=false; +INSERT INTO ${stats_db_name}.otherresearchproduct_tmp SELECT substr(o.id, 4) AS id, o.title[0].value AS title, o.publisher.value AS publisher, CAST(NULL AS string) AS journal, +o.dateofacceptance.value AS DATE, date_format(o.dateofacceptance.value,'yyyy') AS year, o.bestaccessright.classname AS bestlicence, +o.embargoenddate.value as embargo_end_date, FALSE AS delayed, SIZE(o.author) AS authors , concat_ws('\u003B',o.source.value) AS source, +CASE WHEN SIZE(o.description) > 0 THEN TRUE ELSE FALSE END AS abstract, +'other' AS type +FROM ${openaire_db_name}.otherresearchproduct o +WHERE o.datainfo.deletedbyinference=FALSE; --- Software_citations -CREATE TABLE ${stats_db_name}.software_citations AS SELECT substr(s.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS RESULT FROM ${openaire_db_name}.software s LATERAL VIEW explode(s.extrainfo) citations as citation where xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; +-- Otherresearchproduct_citations +CREATE TABLE ${stats_db_name}.otherresearchproduct_citations AS SELECT substr(o.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS RESULT FROM ${openaire_db_name}.otherresearchproduct o LATERAL VIEW explode(o.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; -CREATE TABLE ${stats_db_name}.software_classifications AS SELECT substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype; +CREATE TABLE ${stats_db_name}.otherresearchproduct_classifications AS SELECT substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype; -CREATE TABLE ${stats_db_name}.software_concepts AS SELECT substr(p.id, 4) AS id, contexts.context.id AS concept FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.context) contexts AS context; +CREATE TABLE ${stats_db_name}.otherresearchproduct_concepts AS SELECT substr(p.id, 4) AS id, contexts.context.id AS concept FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.context) contexts AS context; -CREATE TABLE ${stats_db_name}.software_datasources AS SELECT p.id, CASE WHEN d.id IS NULL THEN 'other' ELSE p.datasource end as datasource FROM (SELECT substr(p.id, 4) AS id, substr(instances.instance.hostedby.key, 4) AS datasource -FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.instance) instances AS instance) p LEFT OUTER JOIN -(SELECT substr(d.id, 4) id FROM ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d ON p.datasource = d.id; -CREATE TABLE ${stats_db_name}.software_languages AS select substr(p.id, 4) AS id, p.language.classname AS language FROM ${openaire_db_name}.software p; +CREATE TABLE ${stats_db_name}.otherresearchproduct_datasources AS SELECT p.id, CASE WHEN d.id IS NULL THEN 'other' ELSE p.datasource END AS datasource FROM (SELECT substr(p.id, 4) AS id, substr(instances.instance.hostedby.key, 4) AS datasource +from ${openaire_db_name}.otherresearchproduct p lateral view explode(p.instance) instances as instance) p LEFT OUTER JOIN +(SELECT substr(d.id, 4) id from ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; -CREATE TABLE ${stats_db_name}.software_oids AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.originalid) oids AS ids; +CREATE TABLE ${stats_db_name}.otherresearchproduct_languages AS SELECT substr(p.id, 4) AS id, p.language.classname AS language FROM ${openaire_db_name}.otherresearchproduct p; -CREATE TABLE ${stats_db_name}.software_pids AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.pid) pids AS ppid; +CREATE TABLE ${stats_db_name}.otherresearchproduct_oids AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.originalid) oids AS ids; -CREATE TABLE ${stats_db_name}.software_topics AS SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.subject) subjects AS subject; +CREATE TABLE ${stats_db_name}.otherresearchproduct_pids AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.pid) pids AS ppid; + +CREATE TABLE ${stats_db_name}.otherresearchproduct_topics AS SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.subject) subjects AS subject; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql index cb2c67c9e..aa6eb00b4 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql @@ -1,37 +1,24 @@ --------------------------------------------------------------------------------- --------------------------------------------------------------------------------- --- 6. Otherresearchproduct table/view and Otherresearchproduct related tables/views --------------------------------------------------------------------------------- --------------------------------------------------------------------------------- +------------------------------------------------------ +------------------------------------------------------ +-- Project table/view and Project related tables/views +------------------------------------------------------ +------------------------------------------------------ +-- Project_oids Table +DROP TABLE IF EXISTS ${stats_db_name}.project_oids; +CREATE TABLE ${stats_db_name}.project_oids AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.project p LATERAL VIEW explode(p.originalid) oids AS ids; --- Otherresearchproduct temporary table supporting updates -DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_tmp; -CREATE TABLE ${stats_db_name}.otherresearchproduct_tmp ( id STRING, title STRING, publisher STRING, journal STRING, date STRING, year STRING, bestlicence STRING, embargo_end_date STRING, delayed BOOLEAN, authors INT, source STRING, abstract BOOLEAN, type STRING ) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties('transactional'='true'); +-- Project_organizations Table +DROP TABLE IF EXISTS ${stats_db_name}.project_organizations; +CREATE TABLE ${stats_db_name}.project_organizations AS SELECT substr(r.source, 4) AS id, substr(r.target, 4) AS organization from ${openaire_db_name}.relation r WHERE r.reltype='projectOrganization'; -INSERT INTO ${stats_db_name}.otherresearchproduct_tmp SELECT substr(o.id, 4) AS id, o.title[0].value AS title, o.publisher.value AS publisher, CAST(NULL AS string) AS journal, -o.dateofacceptance.value AS DATE, date_format(o.dateofacceptance.value,'yyyy') AS year, o.bestaccessright.classname AS bestlicence, -o.embargoenddate.value as embargo_end_date, FALSE AS delayed, SIZE(o.author) AS authors , concat_ws('\u003B',o.source.value) AS source, -CASE WHEN SIZE(o.description) > 0 THEN TRUE ELSE FALSE END AS abstract, -'other' AS type -FROM ${openaire_db_name}.otherresearchproduct o -WHERE o.datainfo.deletedbyinference=FALSE; +-- Project_results Table +DROP TABLE IF EXISTS ${stats_db_name}.project_results; +CREATE TABLE ${stats_db_name}.project_results AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS result FROM ${openaire_db_name}.relation r WHERE r.reltype='resultProject'; --- Otherresearchproduct_citations -CREATE TABLE ${stats_db_name}.otherresearchproduct_citations AS SELECT substr(o.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS RESULT FROM ${openaire_db_name}.otherresearchproduct o LATERAL VIEW explode(o.extrainfo) citations AS citation WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") !=""; +-- Project table +---------------- +-- Creating and populating temporary Project table +DROP TABLE IF EXISTS ${stats_db_name}.project_tmp; +CREATE TABLE ${stats_db_name}.project_tmp (id STRING, acronym STRING, title STRING, funder STRING, funding_lvl0 STRING, funding_lvl1 STRING, funding_lvl2 STRING, ec39 STRING, type STRING, startdate STRING, enddate STRING, start_year STRING, end_year STRING, duration INT, haspubs STRING, numpubs INT, daysforlastpub INT, delayedpubs INT, callidentifier STRING, code STRING) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties('transactional'='true'); -CREATE TABLE ${stats_db_name}.otherresearchproduct_classifications AS SELECT substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype; - -CREATE TABLE ${stats_db_name}.otherresearchproduct_concepts AS SELECT substr(p.id, 4) AS id, contexts.context.id AS concept FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.context) contexts AS context; - - -CREATE TABLE ${stats_db_name}.otherresearchproduct_datasources AS SELECT p.id, CASE WHEN d.id IS NULL THEN 'other' ELSE p.datasource END AS datasource FROM (SELECT substr(p.id, 4) AS id, substr(instances.instance.hostedby.key, 4) AS datasource -from ${openaire_db_name}.otherresearchproduct p lateral view explode(p.instance) instances as instance) p LEFT OUTER JOIN -(SELECT substr(d.id, 4) id from ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false) d on p.datasource = d.id; - -CREATE TABLE ${stats_db_name}.otherresearchproduct_languages AS SELECT substr(p.id, 4) AS id, p.language.classname AS language FROM ${openaire_db_name}.otherresearchproduct p; - -CREATE TABLE ${stats_db_name}.otherresearchproduct_oids AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.originalid) oids AS ids; - -CREATE TABLE ${stats_db_name}.otherresearchproduct_pids AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.pid) pids AS ppid; - -CREATE TABLE ${stats_db_name}.otherresearchproduct_topics AS SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.subject) subjects AS subject; +INSERT INTO ${stats_db_name}.project_tmp SELECT substr(p.id, 4) AS id, p.acronym.value AS acronym, p.title.value AS title, xpath_string(p.fundingtree[0].value, '//funder/name') AS funder, xpath_string(p.fundingtree[0].value, '//funding_level_0/name') AS funding_lvl0, xpath_string(p.fundingtree[0].value, '//funding_level_1/name') AS funding_lvl1, xpath_string(p.fundingtree[0].value, '//funding_level_2/name') AS funding_lvl2, p.ecsc39.value AS ec39, p.contracttype.classname AS type, p.startdate.value AS startdate, p.enddate.value AS enddate, date_format(p.startdate.value, 'yyyy') AS start_year, date_format(p.enddate.value, 'yyyy') AS end_year, 0 AS duration, 'no' AS haspubs, 0 AS numpubs, 0 AS daysforlastpub, 0 AS delayedpubs, p.callidentifier.value AS callidentifier, p.code.value AS code FROM ${openaire_db_name}.project p WHERE p.datainfo.deletedbyinference=false; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql index 927543f5a..7acabf1dd 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql @@ -1,24 +1,31 @@ ------------------------------------------------------- ------------------------------------------------------- --- 7. Project table/view and Project related tables/views ------------------------------------------------------- ------------------------------------------------------- --- Project_oids Table -DROP TABLE IF EXISTS ${stats_db_name}.project_oids; -CREATE TABLE ${stats_db_name}.project_oids AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.project p LATERAL VIEW explode(p.originalid) oids AS ids; +---------------------------------------------------- +---------------------------------------------------- +-- Result table/view and Result related tables/views +---------------------------------------------------- +---------------------------------------------------- --- Project_organizations Table -DROP TABLE IF EXISTS ${stats_db_name}.project_organizations; -CREATE TABLE ${stats_db_name}.project_organizations AS SELECT substr(r.source, 4) AS id, substr(r.target, 4) AS organization from ${openaire_db_name}.relation r WHERE r.reltype='projectOrganization'; +-- Views on temporary tables that should be re-created in the end +CREATE OR REPLACE VIEW ${stats_db_name}.result as SELECT *, bestlicence AS access_mode FROM ${stats_db_name}.publication_tmp UNION ALL SELECT *,bestlicence AS access_mode FROM ${stats_db_name}.software_tmp UNION ALL SELECT *,bestlicence AS access_mode FROM ${stats_db_name}.dataset_tmp UNION ALL SELECT *,bestlicence AS access_mode FROM ${stats_db_name}.otherresearchproduct_tmp; --- Project_results Table -DROP TABLE IF EXISTS ${stats_db_name}.project_results; -CREATE TABLE ${stats_db_name}.project_results AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS result FROM ${openaire_db_name}.relation r WHERE r.reltype='resultProject'; +-- Views on final tables +CREATE OR REPLACE VIEW ${stats_db_name}.result_datasources AS SELECT * FROM ${stats_db_name}.publication_datasources UNION ALL SELECT * FROM ${stats_db_name}.software_datasources UNION ALL SELECT * FROM ${stats_db_name}.dataset_datasources UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_datasources; --- Project table ----------------- --- Creating and populating temporary Project table -DROP TABLE IF EXISTS ${stats_db_name}.project_tmp; -CREATE TABLE ${stats_db_name}.project_tmp (id STRING, acronym STRING, title STRING, funder STRING, funding_lvl0 STRING, funding_lvl1 STRING, funding_lvl2 STRING, ec39 STRING, type STRING, startdate STRING, enddate STRING, start_year STRING, end_year STRING, duration INT, haspubs STRING, numpubs INT, daysforlastpub INT, delayedpubs INT, callidentifier STRING, code STRING) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties('transactional'='true'); +CREATE OR REPLACE VIEW ${stats_db_name}.result_citations AS SELECT * FROM ${stats_db_name}.publication_citations UNION ALL SELECT * FROM ${stats_db_name}.software_citations UNION ALL SELECT * FROM ${stats_db_name}.dataset_citations UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_citations; -INSERT INTO ${stats_db_name}.project_tmp SELECT substr(p.id, 4) AS id, p.acronym.value AS acronym, p.title.value AS title, xpath_string(p.fundingtree[0].value, '//funder/name') AS funder, xpath_string(p.fundingtree[0].value, '//funding_level_0/name') AS funding_lvl0, xpath_string(p.fundingtree[0].value, '//funding_level_1/name') AS funding_lvl1, xpath_string(p.fundingtree[0].value, '//funding_level_2/name') AS funding_lvl2, p.ecsc39.value AS ec39, p.contracttype.classname AS type, p.startdate.value AS startdate, p.enddate.value AS enddate, date_format(p.startdate.value, 'yyyy') AS start_year, date_format(p.enddate.value, 'yyyy') AS end_year, 0 AS duration, 'no' AS haspubs, 0 AS numpubs, 0 AS daysforlastpub, 0 AS delayedpubs, p.callidentifier.value AS callidentifier, p.code.value AS code FROM ${openaire_db_name}.project p WHERE p.datainfo.deletedbyinference=false; +CREATE OR REPLACE VIEW ${stats_db_name}.result_classifications AS SELECT * FROM ${stats_db_name}.publication_classifications UNION ALL SELECT * FROM ${stats_db_name}.software_classifications UNION ALL SELECT * FROM ${stats_db_name}.dataset_classifications UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_classifications; + +CREATE OR REPLACE VIEW ${stats_db_name}.result_concepts AS SELECT * FROM ${stats_db_name}.publication_concepts UNION ALL SELECT * FROM ${stats_db_name}.software_concepts UNION ALL SELECT * FROM ${stats_db_name}.dataset_concepts UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_concepts; + +CREATE OR REPLACE VIEW ${stats_db_name}.result_languages AS SELECT * FROM ${stats_db_name}.publication_languages UNION ALL SELECT * FROM ${stats_db_name}.software_languages UNION ALL SELECT * FROM ${stats_db_name}.dataset_languages UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_languages; + +CREATE OR REPLACE VIEW ${stats_db_name}.result_oids AS SELECT * FROM ${stats_db_name}.publication_oids UNION ALL SELECT * FROM ${stats_db_name}.software_oids UNION ALL SELECT * FROM ${stats_db_name}.dataset_oids UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_oids; + +CREATE OR REPLACE VIEW ${stats_db_name}.result_pids AS SELECT * FROM ${stats_db_name}.publication_pids UNION ALL SELECT * FROM ${stats_db_name}.software_pids UNION ALL SELECT * FROM ${stats_db_name}.dataset_pids UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_pids; + +CREATE OR REPLACE VIEW ${stats_db_name}.result_topics AS SELECT * FROM ${stats_db_name}.publication_topics UNION ALL SELECT * FROM ${stats_db_name}.software_topics UNION ALL SELECT * FROM ${stats_db_name}.dataset_topics UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_topics; + +DROP TABLE IF EXISTS ${stats_db_name}.result_organization; +CREATE TABLE ${stats_db_name}.result_organization AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS organization FROM ${openaire_db_name}.relation r WHERE r.reltype='resultOrganization'; + +DROP TABLE IF EXISTS ${stats_db_name}.result_projects; +CREATE TABLE ${stats_db_name}.result_projects AS select pr.result AS id, pr.id AS project, datediff(p.enddate, p.startdate) AS daysfromend FROM ${stats_db_name}.result r JOIN ${stats_db_name}.project_results pr ON r.id=pr.result JOIN ${stats_db_name}.project_tmp p ON p.id=pr.id; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql index de87b0073..3bbe582c1 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql @@ -1,31 +1,37 @@ ----------------------------------------------------- ----------------------------------------------------- --- 8. Result table/view and Result related tables/views ----------------------------------------------------- ----------------------------------------------------- +------------------------------------------------------------ +------------------------------------------------------------ +-- Datasource table/view and Datasource related tables/views +------------------------------------------------------------ +------------------------------------------------------------ --- Views on temporary tables that should be re-created in the end -CREATE OR REPLACE VIEW ${stats_db_name}.result as SELECT *, bestlicence AS access_mode FROM ${stats_db_name}.publication_tmp UNION ALL SELECT *,bestlicence AS access_mode FROM ${stats_db_name}.software_tmp UNION ALL SELECT *,bestlicence AS access_mode FROM ${stats_db_name}.dataset_tmp UNION ALL SELECT *,bestlicence AS access_mode FROM ${stats_db_name}.otherresearchproduct_tmp; +-- Datasource table creation & update +------------------------------------- +-- Creating and populating temporary datasource table +DROP TABLE IF EXISTS ${stats_db_name}.datasource_tmp; +CREATE TABLE ${stats_db_name}.datasource_tmp(`id` string, `name` STRING, `type` STRING, `dateofvalidation` STRING, `yearofvalidation` string, `harvested` BOOLEAN, `piwik_id` INT, `latitude` STRING, `longitude`STRING, `websiteurl` STRING, `compatibility` STRING) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties('transactional'='true'); --- Views on final tables -CREATE OR REPLACE VIEW ${stats_db_name}.result_datasources AS SELECT * FROM ${stats_db_name}.publication_datasources UNION ALL SELECT * FROM ${stats_db_name}.software_datasources UNION ALL SELECT * FROM ${stats_db_name}.dataset_datasources UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_datasources; +INSERT INTO ${stats_db_name}.datasource_tmp SELECT substr(d.id, 4) AS id, officialname.value AS name, datasourcetype.classname AS type, dateofvalidation.value AS dateofvalidation, date_format(d.dateofvalidation.value,'yyyy') AS yearofvalidation, FALSE AS harvested, 0 AS piwik_id, d.latitude.value AS latitude, d.longitude.value AS longitude, d.websiteurl.value AS websiteurl, d.openairecompatibility.classid AS compatibility +FROM ${openaire_db_name}.datasource d +WHERE d.datainfo.deletedbyinference=FALSE; -CREATE OR REPLACE VIEW ${stats_db_name}.result_citations AS SELECT * FROM ${stats_db_name}.publication_citations UNION ALL SELECT * FROM ${stats_db_name}.software_citations UNION ALL SELECT * FROM ${stats_db_name}.dataset_citations UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_citations; +-- Updating temporary table with everything that is not based on results -> This is done with the following "dual" table. +-- Creating a temporary dual table that will be removed after the following insert +CREATE TABLE ${stats_db_name}.dual(dummy CHAR(1)); +INSERT INTO ${stats_db_name}.dual VALUES('X'); +INSERT INTO ${stats_db_name}.datasource_tmp (`id`, `name`, `type`, `dateofvalidation`, `yearofvalidation`, `harvested`, `piwik_id`, `latitude`, `longitude`, `websiteurl`, `compatibility`) +SELECT 'other', 'Other', 'Repository', NULL, NULL, false, 0, NULL, NULL, NULL, 'unknown' FROM ${stats_db_name}.dual WHERE 'other' not in (SELECT id FROM ${stats_db_name}.datasource_tmp WHERE name='Unknown Repository'); +DROP TABLE ${stats_db_name}.dual; -CREATE OR REPLACE VIEW ${stats_db_name}.result_classifications AS SELECT * FROM ${stats_db_name}.publication_classifications UNION ALL SELECT * FROM ${stats_db_name}.software_classifications UNION ALL SELECT * FROM ${stats_db_name}.dataset_classifications UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_classifications; +UPDATE ${stats_db_name}.datasource_tmp SET name='Other' WHERE name='Unknown Repository'; +UPDATE ${stats_db_name}.datasource_tmp SET yearofvalidation=null WHERE yearofvalidation='-1'; -CREATE OR REPLACE VIEW ${stats_db_name}.result_concepts AS SELECT * FROM ${stats_db_name}.publication_concepts UNION ALL SELECT * FROM ${stats_db_name}.software_concepts UNION ALL SELECT * FROM ${stats_db_name}.dataset_concepts UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_concepts; +DROP TABLE IF EXISTS ${stats_db_name}.datasource_languages; +CREATE TABLE ${stats_db_name}.datasource_languages AS SELECT substr(d.id, 4) AS id, langs.languages AS language FROM ${openaire_db_name}.datasource d LATERAL VIEW explode(d.odlanguages.value) langs AS languages; -CREATE OR REPLACE VIEW ${stats_db_name}.result_languages AS SELECT * FROM ${stats_db_name}.publication_languages UNION ALL SELECT * FROM ${stats_db_name}.software_languages UNION ALL SELECT * FROM ${stats_db_name}.dataset_languages UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_languages; +DROP TABLE IF EXISTS ${stats_db_name}.datasource_oids; +CREATE TABLE ${stats_db_name}.datasource_oids AS SELECT substr(d.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.datasource d LATERAL VIEW explode(d.originalid) oids AS ids; -CREATE OR REPLACE VIEW ${stats_db_name}.result_oids AS SELECT * FROM ${stats_db_name}.publication_oids UNION ALL SELECT * FROM ${stats_db_name}.software_oids UNION ALL SELECT * FROM ${stats_db_name}.dataset_oids UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_oids; +DROP TABLE IF EXISTS ${stats_db_name}.datasource_organizations; +CREATE TABLE ${stats_db_name}.datasource_organizations AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS organization FROM ${openaire_db_name}.relation r WHERE r.reltype='datasourceOrganization'; -CREATE OR REPLACE VIEW ${stats_db_name}.result_pids AS SELECT * FROM ${stats_db_name}.publication_pids UNION ALL SELECT * FROM ${stats_db_name}.software_pids UNION ALL SELECT * FROM ${stats_db_name}.dataset_pids UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_pids; - -CREATE OR REPLACE VIEW ${stats_db_name}.result_topics AS SELECT * FROM ${stats_db_name}.publication_topics UNION ALL SELECT * FROM ${stats_db_name}.software_topics UNION ALL SELECT * FROM ${stats_db_name}.dataset_topics UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_topics; - -DROP TABLE IF EXISTS ${stats_db_name}.result_organization; -CREATE TABLE ${stats_db_name}.result_organization AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS organization FROM ${openaire_db_name}.relation r WHERE r.reltype='resultOrganization'; - -DROP TABLE IF EXISTS ${stats_db_name}.result_projects; -CREATE TABLE ${stats_db_name}.result_projects AS select pr.result AS id, pr.id AS project, datediff(p.enddate, p.startdate) AS daysfromend FROM ${stats_db_name}.result r JOIN ${stats_db_name}.project_results pr ON r.id=pr.result JOIN ${stats_db_name}.project_tmp p ON p.id=pr.id; +CREATE OR REPLACE VIEW ${stats_db_name}.datasource_results AS SELECT datasource AS id, id AS result FROM ${stats_db_name}.result_datasources; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql index 5a120d920..462a6b674 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql @@ -1,36 +1,12 @@ ------------------------------------------------------------- ------------------------------------------------------------- --- 9. Datasource table/view and Datasource related tables/views ------------------------------------------------------------- ------------------------------------------------------------- --- Datasource table creation & update -------------------------------------- --- Creating and populating temporary datasource table -DROP TABLE IF EXISTS ${stats_db_name}.datasource_tmp; -CREATE TABLE ${stats_db_name}.datasource_tmp(`id` string, `name` STRING, `type` STRING, `dateofvalidation` STRING, `yearofvalidation` string, `harvested` BOOLEAN, `piwik_id` INT, `latitude` STRING, `longitude`STRING, `websiteurl` STRING, `compatibility` STRING) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties('transactional'='true'); +---------------------------------------------------------------- +---------------------------------------------------------------- +-- Organization table/view and Organization related tables/views +---------------------------------------------------------------- +---------------------------------------------------------------- +DROP TABLE IF EXISTS ${stats_db_name}.organization; +CREATE TABLE IF NOT EXISTS ${stats_db_name}.organization AS SELECT substr(o.id, 4) as id, o.legalname.value as name, o.legalshortname as legalshortname, o.country.classid as country +FROM openaire.organization o WHERE o.datainfo.deletedbyinference=FALSE; -INSERT INTO ${stats_db_name}.datasource_tmp SELECT substr(d.id, 4) AS id, officialname.value AS name, datasourcetype.classname AS type, dateofvalidation.value AS dateofvalidation, date_format(d.dateofvalidation.value,'yyyy') AS yearofvalidation, FALSE AS harvested, 0 AS piwik_id, d.latitude.value AS latitude, d.longitude.value AS longitude, d.websiteurl.value AS websiteurl, d.openairecompatibility.classid AS compatibility -FROM ${openaire_db_name}.datasource d -WHERE d.datainfo.deletedbyinference=FALSE; +CREATE OR REPLACE VIEW ${stats_db_name}.organization_datasources AS SELECT organization AS id, id AS datasource FROM ${stats_db_name}.datasource_organizations; --- Updating temporary table with everything that is not based on results -> This is done with the following "dual" table. --- Creating a temporary dual table that will be removed after the following insert -CREATE TABLE ${stats_db_name}.dual(dummy CHAR(1)); -INSERT INTO ${stats_db_name}.dual VALUES('X'); -INSERT INTO ${stats_db_name}.datasource_tmp (`id`, `name`, `type`, `dateofvalidation`, `yearofvalidation`, `harvested`, `piwik_id`, `latitude`, `longitude`, `websiteurl`, `compatibility`) -SELECT 'other', 'Other', 'Repository', NULL, NULL, false, 0, NULL, NULL, NULL, 'unknown' FROM ${stats_db_name}.dual WHERE 'other' not in (SELECT id FROM ${stats_db_name}.datasource_tmp WHERE name='Unknown Repository'); -DROP TABLE ${stats_db_name}.dual; - -UPDATE ${stats_db_name}.datasource_tmp SET name='Other' WHERE name='Unknown Repository'; -UPDATE ${stats_db_name}.datasource_tmp SET yearofvalidation=null WHERE yearofvalidation='-1'; - -DROP TABLE IF EXISTS ${stats_db_name}.datasource_languages; -CREATE TABLE ${stats_db_name}.datasource_languages AS SELECT substr(d.id, 4) AS id, langs.languages AS language FROM ${openaire_db_name}.datasource d LATERAL VIEW explode(d.odlanguages.value) langs AS languages; - -DROP TABLE IF EXISTS ${stats_db_name}.datasource_oids; -CREATE TABLE ${stats_db_name}.datasource_oids AS SELECT substr(d.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.datasource d LATERAL VIEW explode(d.originalid) oids AS ids; - -DROP TABLE IF EXISTS ${stats_db_name}.datasource_organizations; -CREATE TABLE ${stats_db_name}.datasource_organizations AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS organization FROM ${openaire_db_name}.relation r WHERE r.reltype='datasourceOrganization'; - -CREATE OR REPLACE VIEW ${stats_db_name}.datasource_results AS SELECT datasource AS id, id AS result FROM ${stats_db_name}.result_datasources; +CREATE OR REPLACE VIEW ${stats_db_name}.organization_projects AS SELECT id AS project, organization as id FROM ${stats_db_name}.project_organizations; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index b5f3158f7..f667e1578 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml @@ -13,6 +13,10 @@ stats_ext the external stats that should be added since they are not included in the graph database + + stats_db_shadow_name + the name of the shadow schema + hive_metastore_uris hive server metastore URIs @@ -69,7 +73,7 @@ - + ${hive_jdbc_url} @@ -82,7 +86,7 @@ - + ${hive_jdbc_url} stats_db_name=${stats_db_name} @@ -153,11 +157,12 @@ stats_db_name=${stats_db_name} openaire_db_name=${openaire_db_name} + external_stats_db_name=${external_stats_db_name} - - + + ${hive_jdbc_url} @@ -168,19 +173,18 @@ - - + + - + ${hive_jdbc_url} stats_db_name=${stats_db_name} openaire_db_name=${openaire_db_name} - external_stats_db_name=${external_stats_db_name} - + @@ -194,6 +198,39 @@ + + ${hive_jdbc_url} + + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + + + + + + + + ${hive_jdbc_url} + + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + + + + + + + + ${hive_jdbc_url} + + stats_db_name=${stats_db_name} + stats_db_shadow_name=${stats_db_shadow_name} + + + + + + ${jobTracker} ${nameNode} From 5546f29e63ee454621d2501aaa6226f6e07b346e Mon Sep 17 00:00:00 2001 From: Spyros Zoupanos Date: Tue, 16 Jun 2020 22:30:53 +0300 Subject: [PATCH 082/100] Corrections on the shadow schema and the impala table stats calculation --- .../graph/stats/oozie_app/scripts/step16.sql | 153 +++++++++--------- .../graph/stats/oozie_app/scripts/step17.sql | 27 ++++ .../graph/stats/oozie_app/scripts/step9.sql | 2 +- .../dhp/oa/graph/stats/oozie_app/workflow.xml | 2 +- 4 files changed, 107 insertions(+), 77 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql index 9e04bb13b..8df0072b3 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql @@ -82,79 +82,82 @@ DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_sources; DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_topics; +-- Creating the shadow database, in case it doesn't exist +CREATE database ${stats_db_shadow_name}; + -- Creating new views -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.country AS SELECT * FROM ${stats_db_name}2.country; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.countrygdp AS SELECT * FROM ${stats_db_name}2.countrygdp; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset AS SELECT * FROM ${stats_db_name}2.dataset; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_citations AS SELECT * FROM ${stats_db_name}2.dataset_citations; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_classifications AS SELECT * FROM ${stats_db_name}2.dataset_classifications; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_concepts AS SELECT * FROM ${stats_db_name}2.dataset_concepts; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_datasources AS SELECT * FROM ${stats_db_name}2.dataset_datasources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_languages AS SELECT * FROM ${stats_db_name}2.dataset_languages; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_licenses AS SELECT * FROM ${stats_db_name}2.dataset_licenses; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_oids AS SELECT * FROM ${stats_db_name}2.dataset_oids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_pids AS SELECT * FROM ${stats_db_name}2.dataset_pids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_sources AS SELECT * FROM ${stats_db_name}2.dataset_sources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_topics AS SELECT * FROM ${stats_db_name}2.dataset_topics; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource AS SELECT * FROM ${stats_db_name}2.datasource; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_languages AS SELECT * FROM ${stats_db_name}2.datasource_languages; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_oids AS SELECT * FROM ${stats_db_name}2.datasource_oids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_organizations AS SELECT * FROM ${stats_db_name}2.datasource_organizations; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_results AS SELECT * FROM ${stats_db_name}2.datasource_results; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.fundref AS SELECT * FROM ${stats_db_name}2.fundref; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.numbers_country AS SELECT * FROM ${stats_db_name}2.numbers_country; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.organization AS SELECT * FROM ${stats_db_name}2.organization; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.organization_datasources AS SELECT * FROM ${stats_db_name}2.organization_datasources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.organization_projects AS SELECT * FROM ${stats_db_name}2.organization_projects; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct AS SELECT * FROM ${stats_db_name}2.otherresearchproduct; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_citations AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_citations; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_classifications AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_classifications; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_concepts AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_concepts; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_datasources AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_datasources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_languages AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_languages; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_licenses AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_licenses; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_oids AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_oids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_pids AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_pids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_sources AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_sources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_topics AS SELECT * FROM ${stats_db_name}2.otherresearchproduct_topics; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project AS SELECT * FROM ${stats_db_name}2.project; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project_oids AS SELECT * FROM ${stats_db_name}2.project_oids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project_organizations AS SELECT * FROM ${stats_db_name}2.project_organizations; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project_results AS SELECT * FROM ${stats_db_name}2.project_results; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication AS SELECT * FROM ${stats_db_name}2.publication; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_citations AS SELECT * FROM ${stats_db_name}2.publication_citations; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_classifications AS SELECT * FROM ${stats_db_name}2.publication_classifications; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_concepts AS SELECT * FROM ${stats_db_name}2.publication_concepts; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_datasources AS SELECT * FROM ${stats_db_name}2.publication_datasources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_languages AS SELECT * FROM ${stats_db_name}2.publication_languages; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_licenses AS SELECT * FROM ${stats_db_name}2.publication_licenses; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_oids AS SELECT * FROM ${stats_db_name}2.publication_oids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_pids AS SELECT * FROM ${stats_db_name}2.publication_pids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_sources AS SELECT * FROM ${stats_db_name}2.publication_sources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_topics AS SELECT * FROM ${stats_db_name}2.publication_topics; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result AS SELECT * FROM ${stats_db_name}2.result; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_citations AS SELECT * FROM ${stats_db_name}2.result_citations; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_classifications AS SELECT * FROM ${stats_db_name}2.result_classifications; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_concepts AS SELECT * FROM ${stats_db_name}2.result_concepts; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_datasources AS SELECT * FROM ${stats_db_name}2.result_datasources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_languages AS SELECT * FROM ${stats_db_name}2.result_languages; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_licenses AS SELECT * FROM ${stats_db_name}2.result_licenses; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_oids AS SELECT * FROM ${stats_db_name}2.result_oids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_organization AS SELECT * FROM ${stats_db_name}2.result_organization; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_pids AS SELECT * FROM ${stats_db_name}2.result_pids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_projects AS SELECT * FROM ${stats_db_name}2.result_projects; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_sources AS SELECT * FROM ${stats_db_name}2.result_sources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_topics AS SELECT * FROM ${stats_db_name}2.result_topics; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.rndexpediture AS SELECT * FROM ${stats_db_name}2.rndexpediture; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.roarmap AS SELECT * FROM ${stats_db_name}2.roarmap; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software AS SELECT * FROM ${stats_db_name}2.software; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_citations AS SELECT * FROM ${stats_db_name}2.software_citations; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_classifications AS SELECT * FROM ${stats_db_name}2.software_classifications; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_concepts AS SELECT * FROM ${stats_db_name}2.software_concepts; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_datasources AS SELECT * FROM ${stats_db_name}2.software_datasources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_languages AS SELECT * FROM ${stats_db_name}2.software_languages; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_licenses AS SELECT * FROM ${stats_db_name}2.software_licenses; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_oids AS SELECT * FROM ${stats_db_name}2.software_oids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_pids AS SELECT * FROM ${stats_db_name}2.software_pids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_sources AS SELECT * FROM ${stats_db_name}2.software_sources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_topics AS SELECT * FROM ${stats_db_name}2.software_topics; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.country AS SELECT * FROM ${stats_db_name}.country; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.countrygdp AS SELECT * FROM ${stats_db_name}.countrygdp; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset AS SELECT * FROM ${stats_db_name}.dataset; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_citations AS SELECT * FROM ${stats_db_name}.dataset_citations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_classifications AS SELECT * FROM ${stats_db_name}.dataset_classifications; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_concepts AS SELECT * FROM ${stats_db_name}.dataset_concepts; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_datasources AS SELECT * FROM ${stats_db_name}.dataset_datasources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_languages AS SELECT * FROM ${stats_db_name}.dataset_languages; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_licenses AS SELECT * FROM ${stats_db_name}.dataset_licenses; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_oids AS SELECT * FROM ${stats_db_name}.dataset_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_pids AS SELECT * FROM ${stats_db_name}.dataset_pids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_sources AS SELECT * FROM ${stats_db_name}.dataset_sources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_topics AS SELECT * FROM ${stats_db_name}.dataset_topics; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource AS SELECT * FROM ${stats_db_name}.datasource; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_languages AS SELECT * FROM ${stats_db_name}.datasource_languages; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_oids AS SELECT * FROM ${stats_db_name}.datasource_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_organizations AS SELECT * FROM ${stats_db_name}.datasource_organizations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_results AS SELECT * FROM ${stats_db_name}.datasource_results; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.fundref AS SELECT * FROM ${stats_db_name}.fundref; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.numbers_country AS SELECT * FROM ${stats_db_name}.numbers_country; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.organization AS SELECT * FROM ${stats_db_name}.organization; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.organization_datasources AS SELECT * FROM ${stats_db_name}.organization_datasources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.organization_projects AS SELECT * FROM ${stats_db_name}.organization_projects; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct AS SELECT * FROM ${stats_db_name}.otherresearchproduct; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_citations AS SELECT * FROM ${stats_db_name}.otherresearchproduct_citations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_classifications AS SELECT * FROM ${stats_db_name}.otherresearchproduct_classifications; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_concepts AS SELECT * FROM ${stats_db_name}.otherresearchproduct_concepts; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_datasources AS SELECT * FROM ${stats_db_name}.otherresearchproduct_datasources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_languages AS SELECT * FROM ${stats_db_name}.otherresearchproduct_languages; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_licenses AS SELECT * FROM ${stats_db_name}.otherresearchproduct_licenses; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_oids AS SELECT * FROM ${stats_db_name}.otherresearchproduct_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_pids AS SELECT * FROM ${stats_db_name}.otherresearchproduct_pids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_sources AS SELECT * FROM ${stats_db_name}.otherresearchproduct_sources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_topics AS SELECT * FROM ${stats_db_name}.otherresearchproduct_topics; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project AS SELECT * FROM ${stats_db_name}.project; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project_oids AS SELECT * FROM ${stats_db_name}.project_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project_organizations AS SELECT * FROM ${stats_db_name}.project_organizations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project_results AS SELECT * FROM ${stats_db_name}.project_results; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication AS SELECT * FROM ${stats_db_name}.publication; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_citations AS SELECT * FROM ${stats_db_name}.publication_citations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_classifications AS SELECT * FROM ${stats_db_name}.publication_classifications; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_concepts AS SELECT * FROM ${stats_db_name}.publication_concepts; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_datasources AS SELECT * FROM ${stats_db_name}.publication_datasources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_languages AS SELECT * FROM ${stats_db_name}.publication_languages; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_licenses AS SELECT * FROM ${stats_db_name}.publication_licenses; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_oids AS SELECT * FROM ${stats_db_name}.publication_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_pids AS SELECT * FROM ${stats_db_name}.publication_pids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_sources AS SELECT * FROM ${stats_db_name}.publication_sources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_topics AS SELECT * FROM ${stats_db_name}.publication_topics; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result AS SELECT * FROM ${stats_db_name}.result; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_citations AS SELECT * FROM ${stats_db_name}.result_citations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_classifications AS SELECT * FROM ${stats_db_name}.result_classifications; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_concepts AS SELECT * FROM ${stats_db_name}.result_concepts; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_datasources AS SELECT * FROM ${stats_db_name}.result_datasources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_languages AS SELECT * FROM ${stats_db_name}.result_languages; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_licenses AS SELECT * FROM ${stats_db_name}.result_licenses; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_oids AS SELECT * FROM ${stats_db_name}.result_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_organization AS SELECT * FROM ${stats_db_name}.result_organization; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_pids AS SELECT * FROM ${stats_db_name}.result_pids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_projects AS SELECT * FROM ${stats_db_name}.result_projects; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_sources AS SELECT * FROM ${stats_db_name}.result_sources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_topics AS SELECT * FROM ${stats_db_name}.result_topics; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.rndexpediture AS SELECT * FROM ${stats_db_name}.rndexpediture; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.roarmap AS SELECT * FROM ${stats_db_name}.roarmap; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software AS SELECT * FROM ${stats_db_name}.software; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_citations AS SELECT * FROM ${stats_db_name}.software_citations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_classifications AS SELECT * FROM ${stats_db_name}.software_classifications; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_concepts AS SELECT * FROM ${stats_db_name}.software_concepts; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_datasources AS SELECT * FROM ${stats_db_name}.software_datasources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_languages AS SELECT * FROM ${stats_db_name}.software_languages; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_licenses AS SELECT * FROM ${stats_db_name}.software_licenses; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_oids AS SELECT * FROM ${stats_db_name}.software_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_pids AS SELECT * FROM ${stats_db_name}.software_pids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_sources AS SELECT * FROM ${stats_db_name}.software_sources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_topics AS SELECT * FROM ${stats_db_name}.software_topics; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step17.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step17.sql index bfa7fdf5c..5645db309 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step17.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step17.sql @@ -5,6 +5,8 @@ ------------------------------------------------------ ------------------------------------------------------ +COMPUTE STATS country; +COMPUTE STATS countrygdp; COMPUTE STATS dataset; COMPUTE STATS dataset_citations; COMPUTE STATS dataset_classifications; @@ -13,21 +15,28 @@ COMPUTE STATS dataset_datasources; COMPUTE STATS dataset_languages; COMPUTE STATS dataset_oids; COMPUTE STATS dataset_pids; +COMPUTE STATS dataset_sources; COMPUTE STATS dataset_topics; COMPUTE STATS datasource; COMPUTE STATS datasource_languages; COMPUTE STATS datasource_oids; COMPUTE STATS datasource_organizations; +COMPUTE STATS datasource_results; +COMPUTE STATS fundref; COMPUTE STATS numbers_country; COMPUTE STATS organization; +COMPUTE STATS organization_datasources; +COMPUTE STATS organization_projects; COMPUTE STATS otherresearchproduct; COMPUTE STATS otherresearchproduct_citations; COMPUTE STATS otherresearchproduct_classifications; COMPUTE STATS otherresearchproduct_concepts; COMPUTE STATS otherresearchproduct_datasources; COMPUTE STATS otherresearchproduct_languages; +COMPUTE STATS otherresearchproduct_licenses; COMPUTE STATS otherresearchproduct_oids; COMPUTE STATS otherresearchproduct_pids; +COMPUTE STATS otherresearchproduct_sources; COMPUTE STATS otherresearchproduct_topics; COMPUTE STATS project; COMPUTE STATS project_oids; @@ -39,16 +48,34 @@ COMPUTE STATS publication_classifications; COMPUTE STATS publication_concepts; COMPUTE STATS publication_datasources; COMPUTE STATS publication_languages; +COMPUTE STATS publication_licenses; COMPUTE STATS publication_oids; COMPUTE STATS publication_pids; +COMPUTE STATS publication_sources; COMPUTE STATS publication_topics; +COMPUTE STATS result; +COMPUTE STATS result_citations; +COMPUTE STATS result_classifications; +COMPUTE STATS result_concepts; +COMPUTE STATS result_datasources; +COMPUTE STATS result_languages; +COMPUTE STATS result_licenses; +COMPUTE STATS result_oids; COMPUTE STATS result_organization; +COMPUTE STATS result_pids; COMPUTE STATS result_projects; +COMPUTE STATS result_sources; +COMPUTE STATS result_topics; +COMPUTE STATS rndexpediture; +COMPUTE STATS roarmap; COMPUTE STATS software; COMPUTE STATS software_citations; COMPUTE STATS software_classifications; COMPUTE STATS software_concepts; COMPUTE STATS software_datasources; COMPUTE STATS software_languages; +COMPUTE STATS software_licenses; COMPUTE STATS software_oids; COMPUTE STATS software_pids; +COMPUTE STATS software_sources; +COMPUTE STATS software_topics; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql index 462a6b674..9bbf07e15 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql @@ -5,7 +5,7 @@ ---------------------------------------------------------------- DROP TABLE IF EXISTS ${stats_db_name}.organization; CREATE TABLE IF NOT EXISTS ${stats_db_name}.organization AS SELECT substr(o.id, 4) as id, o.legalname.value as name, o.legalshortname as legalshortname, o.country.classid as country -FROM openaire.organization o WHERE o.datainfo.deletedbyinference=FALSE; +FROM ${openaire_db_name}.organization o WHERE o.datainfo.deletedbyinference=FALSE; CREATE OR REPLACE VIEW ${stats_db_name}.organization_datasources AS SELECT organization AS id, id AS datasource FROM ${stats_db_name}.datasource_organizations; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index f667e1578..6c9197d18 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml @@ -1,4 +1,4 @@ - + stats_db_name From 3630794cefbc3027837f52d02227b55c842d7286 Mon Sep 17 00:00:00 2001 From: Spyros Zoupanos Date: Tue, 16 Jun 2020 22:32:28 +0300 Subject: [PATCH 083/100] Fix to consider the relationships that have been 'virtually deleted' for project_results - defect #5607 --- .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql index aa6eb00b4..e77964513 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql @@ -13,7 +13,7 @@ CREATE TABLE ${stats_db_name}.project_organizations AS SELECT substr(r.source, 4 -- Project_results Table DROP TABLE IF EXISTS ${stats_db_name}.project_results; -CREATE TABLE ${stats_db_name}.project_results AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS result FROM ${openaire_db_name}.relation r WHERE r.reltype='resultProject'; +CREATE TABLE ${stats_db_name}.project_results AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS result FROM ${openaire_db_name}.relation r WHERE r.reltype='resultProject' and r.datainfo.deletedbyinference=false; -- Project table ---------------- From 138c6ddffa665f9772ac0c6ecf1d77b400341cf2 Mon Sep 17 00:00:00 2001 From: Spyros Zoupanos Date: Wed, 17 Jun 2020 23:48:01 +0300 Subject: [PATCH 084/100] Insert statement to datasource table that takes into account the piwik_id of the openAIRE graph --- .../graph/stats/oozie_app/scripts/step8.sql | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql index 3bbe582c1..609132781 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql @@ -10,9 +10,22 @@ DROP TABLE IF EXISTS ${stats_db_name}.datasource_tmp; CREATE TABLE ${stats_db_name}.datasource_tmp(`id` string, `name` STRING, `type` STRING, `dateofvalidation` STRING, `yearofvalidation` string, `harvested` BOOLEAN, `piwik_id` INT, `latitude` STRING, `longitude`STRING, `websiteurl` STRING, `compatibility` STRING) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties('transactional'='true'); -INSERT INTO ${stats_db_name}.datasource_tmp SELECT substr(d.id, 4) AS id, officialname.value AS name, datasourcetype.classname AS type, dateofvalidation.value AS dateofvalidation, date_format(d.dateofvalidation.value,'yyyy') AS yearofvalidation, FALSE AS harvested, 0 AS piwik_id, d.latitude.value AS latitude, d.longitude.value AS longitude, d.websiteurl.value AS websiteurl, d.openairecompatibility.classid AS compatibility -FROM ${openaire_db_name}.datasource d -WHERE d.datainfo.deletedbyinference=FALSE; +-- Insert statement that takes into account the piwik_id of the openAIRE graph +INSERT INTO ${stats_db_name}.datasource_tmp +SELECT substr(d1.id, 4) AS id, officialname.value AS name, +datasourcetype.classname AS type, dateofvalidation.value AS dateofvalidation, date_format(d1.dateofvalidation.value,'yyyy') AS yearofvalidation, +FALSE AS harvested, +CASE WHEN d2.piwik_id IS NULL THEN 0 ELSE d2.piwik_id END AS piwik_id, +d1.latitude.value AS latitude, d1.longitude.value AS longitude, +d1.websiteurl.value AS websiteurl, d1.openairecompatibility.classid AS compatibility +FROM ${openaire_db_name}.datasource d1 +LEFT OUTER JOIN +(SELECT id, split(originalidd, '\\:')[1] as piwik_id +FROM ${openaire_db_name}.datasource +LATERAL VIEW EXPLODE(originalid) temp AS originalidd +WHERE originalidd like "piwik:%") AS d2 +ON d1.id = d2.id +WHERE d1.datainfo.deletedbyinference=FALSE; -- Updating temporary table with everything that is not based on results -> This is done with the following "dual" table. -- Creating a temporary dual table that will be removed after the following insert From 63cd797aba6cb86f33b2d9f35cf8d368d78f2672 Mon Sep 17 00:00:00 2001 From: Spyros Zoupanos Date: Thu, 25 Jun 2020 19:45:57 +0300 Subject: [PATCH 085/100] Comment out step 15 to make it work with the new schema of Claudio --- .../graph/stats/oozie_app/scripts/step15.sql | 58 ++++++++++--------- 1 file changed, 32 insertions(+), 26 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql index 9dacb92ce..a638d7163 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql @@ -5,31 +5,37 @@ -- Refereed related tables/views ------------------------------------------------------ ------------------------------------------------------ -CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_refereed as -select substr(r.id, 4) as id, inst.refereed.value as refereed -from ${openaire_db_name}.publication r lateral view explode(r.instance) instances as inst -where r.datainfo.deletedbyinference=false; -CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_refereed as -select substr(r.id, 4) as id, inst.refereed.value as refereed -from ${openaire_db_name}.dataset r lateral view explode(r.instance) instances as inst -where r.datainfo.deletedbyinference=false; +-- To be commented out after the following queries are adapted to +-- the new openAIRE graph schema. +-- When it is ready they should be added to the to statistic recalculation +-- for the impala tables and the shadow views -CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_refereed as -select substr(r.id, 4) as id, inst.refereed.value as refereed -from ${openaire_db_name}.software r lateral view explode(r.instance) instances as inst -where r.datainfo.deletedbyinference=false; - -CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_refereed as -select substr(r.id, 4) as id, inst.refereed.value as refereed -from ${openaire_db_name}.otherresearchproduct r lateral view explode(r.instance) instances as inst -where r.datainfo.deletedbyinference=false; - -CREATE VIEW IF NOT EXISTS ${stats_db_name}.result_refereed as -select * from ${stats_db_name}.publication_refereed -union all -select * from ${stats_db_name}.dataset_refereed -union all -select * from ${stats_db_name}.software_refereed -union all -select * from ${stats_db_name}.otherresearchproduct_refereed; +-- CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_refereed as +-- select substr(r.id, 4) as id, inst.refereed.value as refereed +-- from ${openaire_db_name}.publication r lateral view explode(r.instance) instances as inst +-- where r.datainfo.deletedbyinference=false; +-- +-- CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_refereed as +-- select substr(r.id, 4) as id, inst.refereed.value as refereed +-- from ${openaire_db_name}.dataset r lateral view explode(r.instance) instances as inst +-- where r.datainfo.deletedbyinference=false; +-- +-- CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_refereed as +-- select substr(r.id, 4) as id, inst.refereed.value as refereed +-- from ${openaire_db_name}.software r lateral view explode(r.instance) instances as inst +-- where r.datainfo.deletedbyinference=false; +-- +-- CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_refereed as +-- select substr(r.id, 4) as id, inst.refereed.value as refereed +-- from ${openaire_db_name}.otherresearchproduct r lateral view explode(r.instance) instances as inst +-- where r.datainfo.deletedbyinference=false; +-- +-- CREATE VIEW IF NOT EXISTS ${stats_db_name}.result_refereed as +-- select * from ${stats_db_name}.publication_refereed +-- union all +-- select * from ${stats_db_name}.dataset_refereed +-- union all +-- select * from ${stats_db_name}.software_refereed +-- union all +-- select * from ${stats_db_name}.otherresearchproduct_refereed; From ec52141f1a4d864083025abfb2ea0279f617fc43 Mon Sep 17 00:00:00 2001 From: antleb <50_David> Date: Tue, 30 Jun 2020 01:49:56 +0300 Subject: [PATCH 086/100] changed refereed type from value to clssname --- .../graph/stats/oozie_app/scripts/step15.sql | 56 +++++++++---------- 1 file changed, 28 insertions(+), 28 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql index a638d7163..41d80405d 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql @@ -11,31 +11,31 @@ -- When it is ready they should be added to the to statistic recalculation -- for the impala tables and the shadow views --- CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_refereed as --- select substr(r.id, 4) as id, inst.refereed.value as refereed --- from ${openaire_db_name}.publication r lateral view explode(r.instance) instances as inst --- where r.datainfo.deletedbyinference=false; --- --- CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_refereed as --- select substr(r.id, 4) as id, inst.refereed.value as refereed --- from ${openaire_db_name}.dataset r lateral view explode(r.instance) instances as inst --- where r.datainfo.deletedbyinference=false; --- --- CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_refereed as --- select substr(r.id, 4) as id, inst.refereed.value as refereed --- from ${openaire_db_name}.software r lateral view explode(r.instance) instances as inst --- where r.datainfo.deletedbyinference=false; --- --- CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_refereed as --- select substr(r.id, 4) as id, inst.refereed.value as refereed --- from ${openaire_db_name}.otherresearchproduct r lateral view explode(r.instance) instances as inst --- where r.datainfo.deletedbyinference=false; --- --- CREATE VIEW IF NOT EXISTS ${stats_db_name}.result_refereed as --- select * from ${stats_db_name}.publication_refereed --- union all --- select * from ${stats_db_name}.dataset_refereed --- union all --- select * from ${stats_db_name}.software_refereed --- union all --- select * from ${stats_db_name}.otherresearchproduct_refereed; +CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_refereed as +select substr(r.id, 4) as id, inst.refereed.classsname as refereed +from ${openaire_db_name}.publication r lateral view explode(r.instance) instances as inst +where r.datainfo.deletedbyinference=false; + +CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_refereed as +select substr(r.id, 4) as id, inst.refereed.classsname as refereed +from ${openaire_db_name}.dataset r lateral view explode(r.instance) instances as inst +where r.datainfo.deletedbyinference=false; + +CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_refereed as +select substr(r.id, 4) as id, inst.refereed.classsname as refereed +from ${openaire_db_name}.software r lateral view explode(r.instance) instances as inst +where r.datainfo.deletedbyinference=false; + +CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_refereed as +select substr(r.id, 4) as id, inst.refereed.classsname as refereed +from ${openaire_db_name}.otherresearchproduct r lateral view explode(r.instance) instances as inst +where r.datainfo.deletedbyinference=false; + +CREATE VIEW IF NOT EXISTS ${stats_db_name}.result_refereed as +select * from ${stats_db_name}.publication_refereed +union all +select * from ${stats_db_name}.dataset_refereed +union all +select * from ${stats_db_name}.software_refereed +union all +select * from ${stats_db_name}.otherresearchproduct_refereed; From 68389d0125969a09104f644cf6cfe28b6a726358 Mon Sep 17 00:00:00 2001 From: antleb <50_David> Date: Tue, 30 Jun 2020 01:50:05 +0300 Subject: [PATCH 087/100] Corrected the script used by the last step of the wf --- .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index 6c9197d18..13b4bfaf7 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml @@ -236,8 +236,8 @@ ${nameNode} impala-shell.sh ${stats_db_name} - step14.sql - /user/${wf:user()}/oa/graph/stats/oozie_app/scripts/step14.sql + step17.sql + /user/${wf:user()}/oa/graph/stats/oozie_app/scripts/step17.sql impala-shell.sh From 391cf80fb8c11cf62d310bbf4527403b6ad9576b Mon Sep 17 00:00:00 2001 From: antleb <50_David> Date: Tue, 30 Jun 2020 02:13:08 +0300 Subject: [PATCH 088/100] Added peer-reviewed, green, gold tables and fields in result. Added shortcuts from result-country --- .../graph/stats/oozie_app/scripts/step16.sql | 290 ++++++++---------- .../graph/stats/oozie_app/scripts/step17.sql | 234 +++++++++----- .../graph/stats/oozie_app/scripts/step18.sql | 81 +++++ .../dhp/oa/graph/stats/oozie_app/workflow.xml | 19 +- 4 files changed, 384 insertions(+), 240 deletions(-) create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step18.sql diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql index 8df0072b3..8e361c1ad 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql @@ -1,163 +1,133 @@ ------------------------------------------------------- ------------------------------------------------------- --- Shadow schema table exchange ------------------------------------------------------- ------------------------------------------------------- - --- Dropping old views -DROP VIEW IF EXISTS ${stats_db_shadow_name}.country; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.countrygdp; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_citations; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_classifications; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_concepts; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_datasources; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_languages; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_licenses; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_oids; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_pids; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_sources; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_topics; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.datasource; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.datasource_languages; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.datasource_oids; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.datasource_organizations; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.datasource_results; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.fundref; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.numbers_country; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.organization; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.organization_datasources; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.organization_projects; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_citations; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_classifications; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_concepts; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_datasources; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_languages; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_licenses; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_oids; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_pids; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_sources; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_topics; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.project; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.project_oids; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.project_organizations; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.project_results; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_citations; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_classifications; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_concepts; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_datasources; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_languages; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_licenses; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_oids; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_pids; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_sources; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_topics; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.result; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_citations; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_classifications; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_concepts; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_datasources; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_languages; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_licenses; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_oids; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_organization; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_pids; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_projects; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_sources; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_topics; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.rndexpediture; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.roarmap; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.software; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_citations; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_classifications; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_concepts; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_datasources; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_languages; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_licenses; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_oids; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_pids; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_sources; -DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_topics; +---------------------------------------------------- +-- Shortcuts for various definitions in stats db --- +---------------------------------------------------- --- Creating the shadow database, in case it doesn't exist -CREATE database ${stats_db_shadow_name}; +-- Peer reviewed: +-- Results that have been collected from Crossref +create table ${stats_db_name}.result_peerreviewed as +with peer_reviewed as ( + select distinct r.id as id + from ${stats_db_name}.result r + join ${stats_db_name}.result_sources rs on rs.id=r.id + join ${stats_db_name}.datasource d on d.id=rs.datasource + where d.name='Crossref') +select distinct peer_reviewed.id as id, true as peer_reviewed +from peer_reviewed +union all +select distinct r.id as id, false as peer_reviewed +from ${stats_db_name}.result r +where r.id not in (select id from peer_reviewed); --- Creating new views -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.country AS SELECT * FROM ${stats_db_name}.country; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.countrygdp AS SELECT * FROM ${stats_db_name}.countrygdp; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset AS SELECT * FROM ${stats_db_name}.dataset; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_citations AS SELECT * FROM ${stats_db_name}.dataset_citations; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_classifications AS SELECT * FROM ${stats_db_name}.dataset_classifications; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_concepts AS SELECT * FROM ${stats_db_name}.dataset_concepts; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_datasources AS SELECT * FROM ${stats_db_name}.dataset_datasources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_languages AS SELECT * FROM ${stats_db_name}.dataset_languages; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_licenses AS SELECT * FROM ${stats_db_name}.dataset_licenses; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_oids AS SELECT * FROM ${stats_db_name}.dataset_oids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_pids AS SELECT * FROM ${stats_db_name}.dataset_pids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_sources AS SELECT * FROM ${stats_db_name}.dataset_sources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_topics AS SELECT * FROM ${stats_db_name}.dataset_topics; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource AS SELECT * FROM ${stats_db_name}.datasource; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_languages AS SELECT * FROM ${stats_db_name}.datasource_languages; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_oids AS SELECT * FROM ${stats_db_name}.datasource_oids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_organizations AS SELECT * FROM ${stats_db_name}.datasource_organizations; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_results AS SELECT * FROM ${stats_db_name}.datasource_results; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.fundref AS SELECT * FROM ${stats_db_name}.fundref; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.numbers_country AS SELECT * FROM ${stats_db_name}.numbers_country; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.organization AS SELECT * FROM ${stats_db_name}.organization; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.organization_datasources AS SELECT * FROM ${stats_db_name}.organization_datasources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.organization_projects AS SELECT * FROM ${stats_db_name}.organization_projects; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct AS SELECT * FROM ${stats_db_name}.otherresearchproduct; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_citations AS SELECT * FROM ${stats_db_name}.otherresearchproduct_citations; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_classifications AS SELECT * FROM ${stats_db_name}.otherresearchproduct_classifications; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_concepts AS SELECT * FROM ${stats_db_name}.otherresearchproduct_concepts; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_datasources AS SELECT * FROM ${stats_db_name}.otherresearchproduct_datasources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_languages AS SELECT * FROM ${stats_db_name}.otherresearchproduct_languages; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_licenses AS SELECT * FROM ${stats_db_name}.otherresearchproduct_licenses; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_oids AS SELECT * FROM ${stats_db_name}.otherresearchproduct_oids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_pids AS SELECT * FROM ${stats_db_name}.otherresearchproduct_pids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_sources AS SELECT * FROM ${stats_db_name}.otherresearchproduct_sources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_topics AS SELECT * FROM ${stats_db_name}.otherresearchproduct_topics; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project AS SELECT * FROM ${stats_db_name}.project; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project_oids AS SELECT * FROM ${stats_db_name}.project_oids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project_organizations AS SELECT * FROM ${stats_db_name}.project_organizations; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project_results AS SELECT * FROM ${stats_db_name}.project_results; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication AS SELECT * FROM ${stats_db_name}.publication; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_citations AS SELECT * FROM ${stats_db_name}.publication_citations; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_classifications AS SELECT * FROM ${stats_db_name}.publication_classifications; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_concepts AS SELECT * FROM ${stats_db_name}.publication_concepts; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_datasources AS SELECT * FROM ${stats_db_name}.publication_datasources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_languages AS SELECT * FROM ${stats_db_name}.publication_languages; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_licenses AS SELECT * FROM ${stats_db_name}.publication_licenses; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_oids AS SELECT * FROM ${stats_db_name}.publication_oids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_pids AS SELECT * FROM ${stats_db_name}.publication_pids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_sources AS SELECT * FROM ${stats_db_name}.publication_sources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_topics AS SELECT * FROM ${stats_db_name}.publication_topics; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result AS SELECT * FROM ${stats_db_name}.result; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_citations AS SELECT * FROM ${stats_db_name}.result_citations; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_classifications AS SELECT * FROM ${stats_db_name}.result_classifications; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_concepts AS SELECT * FROM ${stats_db_name}.result_concepts; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_datasources AS SELECT * FROM ${stats_db_name}.result_datasources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_languages AS SELECT * FROM ${stats_db_name}.result_languages; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_licenses AS SELECT * FROM ${stats_db_name}.result_licenses; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_oids AS SELECT * FROM ${stats_db_name}.result_oids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_organization AS SELECT * FROM ${stats_db_name}.result_organization; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_pids AS SELECT * FROM ${stats_db_name}.result_pids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_projects AS SELECT * FROM ${stats_db_name}.result_projects; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_sources AS SELECT * FROM ${stats_db_name}.result_sources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_topics AS SELECT * FROM ${stats_db_name}.result_topics; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.rndexpediture AS SELECT * FROM ${stats_db_name}.rndexpediture; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.roarmap AS SELECT * FROM ${stats_db_name}.roarmap; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software AS SELECT * FROM ${stats_db_name}.software; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_citations AS SELECT * FROM ${stats_db_name}.software_citations; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_classifications AS SELECT * FROM ${stats_db_name}.software_classifications; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_concepts AS SELECT * FROM ${stats_db_name}.software_concepts; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_datasources AS SELECT * FROM ${stats_db_name}.software_datasources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_languages AS SELECT * FROM ${stats_db_name}.software_languages; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_licenses AS SELECT * FROM ${stats_db_name}.software_licenses; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_oids AS SELECT * FROM ${stats_db_name}.software_oids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_pids AS SELECT * FROM ${stats_db_name}.software_pids; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_sources AS SELECT * FROM ${stats_db_name}.software_sources; -CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_topics AS SELECT * FROM ${stats_db_name}.software_topics; +-- Green OA: +-- OA results that are hosted by an Institutional repository and have NOT been harvested from a DOAJ journal. +create table ${stats_db_name}.result_greenoa as +with result_green as ( + select distinct r.id as id + from ${stats_db_name}.result r + join ${stats_db_name}.result_datasources rd on rd.id=r.id + join ${stats_db_name}.datasource d on d.id=rd.datasource + where r.bestlicence in ('Open Access', 'Open Source') and d.type='Institutional Repository' and not exists ( + select 1 from ${stats_db_name}.result_datasources rd + join ${stats_db_name}.datasource d on rd.datasource=d.id + join ${stats_db_name}.datasource_sources sds on sds.id=d.id + join ${stats_db_name}.datasource sd on sd.id=sds.datasource + where sd.name='DOAJ-ARTICLES' and rd.id=r.id)) +select distinct result_green.id, true as green +from result_green +union all +select distinct r.id as id, false as green +from ${stats_db_name}.result r +where r.id not in (select id from result_green); + +-- GOLD OA: +-- OA results that have been harvested from a DOAJ journal. +create table ${stats_db_name}.result_gold as +with result_gold as ( + select distinct r.id as id + from ${stats_db_name}.result r + join ${stats_db_name}.result_datasources rd on rd.id=r.id + join ${stats_db_name}.datasource d on d.id=rd.datasource + join ${stats_db_name}.datasource_sources sds on sds.id=d.id + join ${stats_db_name}.datasource sd on sd.id=sds.datasource + where r.type='publication' and r.bestlicence='Open Access' and sd.name='DOAJ-Articles') +select distinct result_gold.id, true as gold +from result_gold +union all +select distinct r.id, false as gold +from ${stats_db_name}.result r +where r.id not in (select id from result_gold); + +-- shortcut result-country through the organization affiliation +create table ${stats_db_name}.result_affiliated_country as +select r.id as id, o.country as country +from ${stats_db_name}.result r +join ${stats_db_name}.result_organization ro on ro.id=r.id +join ${stats_db_name}.organization o on o.id=ro.organization +where o.country is not null and o.country!=''; + +-- shortcut result-country through datasource of deposition +create table ${stats_db_name}.result_deposited_country as +select r.id as id, o.country as country +from ${stats_db_name}.result r +join ${stats_db_name}.result_datasources rd on rd.id=r.id +join ${stats_db_name}.datasource d on d.id=rd.datasource +join ${stats_db_name}.datasource_organizations dor on dor.id=d.id +join ${stats_db_name}.organization o on o.id=dor.organization +where o.country is not null and o.country!=''; + +-- replace the creation of the result view to include the boolean fields from the previous tables (green, gold, +-- peer reviewed) +drop table if exists ${stats_db_name}.result_tmp; +CREATE TABLE result_tmp ( + id STRING, + title STRING, + publisher STRING, + journal STRING, + `date` STRING, + `year` INT, + bestlicence STRING, + access_mode STRING, + embargo_end_date STRING, + delayed BOOLEAN, + authors INT, + source STRING, + abstract BOOLEAN, + type STRING , + peer_reviewed BOOLEAN, + green BOOLEAN, + gold BOOLEAN) +clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); + +insert into ${stats_db_name}.result_tmp +select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold +FROM ${stats_db_name}.publication r +LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id +LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id +LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; + +insert into ${stats_db_name}.result_tmp +select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold +FROM ${stats_db_name}.dataset r +LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id +LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id +LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; + +insert into ${stats_db_name}.result_tmp +select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold +FROM ${stats_db_name}.software r +LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id +LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id +LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; + +insert into ${stats_db_name}.result_tmp +select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold +FROM ${stats_db_name}.otherresearchproduct r +LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id +LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id +LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; + +drop table if exists ${stats_db_name}.result; +drop view if exists ${stats_db_name}.result; +create table ${stats_db_name}.result stored as parquet as select * from ${stats_db_name}.result_tmp; +drop table ${stats_db_name}.result_tmp; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step17.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step17.sql index 5645db309..8df0072b3 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step17.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step17.sql @@ -1,81 +1,163 @@ ------------------------------------------------------ ------------------------------------------------------ --- Impala table statistics - Needed to make the tables --- visible for impala +-- Shadow schema table exchange ------------------------------------------------------ ------------------------------------------------------ -COMPUTE STATS country; -COMPUTE STATS countrygdp; -COMPUTE STATS dataset; -COMPUTE STATS dataset_citations; -COMPUTE STATS dataset_classifications; -COMPUTE STATS dataset_concepts; -COMPUTE STATS dataset_datasources; -COMPUTE STATS dataset_languages; -COMPUTE STATS dataset_oids; -COMPUTE STATS dataset_pids; -COMPUTE STATS dataset_sources; -COMPUTE STATS dataset_topics; -COMPUTE STATS datasource; -COMPUTE STATS datasource_languages; -COMPUTE STATS datasource_oids; -COMPUTE STATS datasource_organizations; -COMPUTE STATS datasource_results; -COMPUTE STATS fundref; -COMPUTE STATS numbers_country; -COMPUTE STATS organization; -COMPUTE STATS organization_datasources; -COMPUTE STATS organization_projects; -COMPUTE STATS otherresearchproduct; -COMPUTE STATS otherresearchproduct_citations; -COMPUTE STATS otherresearchproduct_classifications; -COMPUTE STATS otherresearchproduct_concepts; -COMPUTE STATS otherresearchproduct_datasources; -COMPUTE STATS otherresearchproduct_languages; -COMPUTE STATS otherresearchproduct_licenses; -COMPUTE STATS otherresearchproduct_oids; -COMPUTE STATS otherresearchproduct_pids; -COMPUTE STATS otherresearchproduct_sources; -COMPUTE STATS otherresearchproduct_topics; -COMPUTE STATS project; -COMPUTE STATS project_oids; -COMPUTE STATS project_organizations; -COMPUTE STATS project_results; -COMPUTE STATS publication; -COMPUTE STATS publication_citations; -COMPUTE STATS publication_classifications; -COMPUTE STATS publication_concepts; -COMPUTE STATS publication_datasources; -COMPUTE STATS publication_languages; -COMPUTE STATS publication_licenses; -COMPUTE STATS publication_oids; -COMPUTE STATS publication_pids; -COMPUTE STATS publication_sources; -COMPUTE STATS publication_topics; -COMPUTE STATS result; -COMPUTE STATS result_citations; -COMPUTE STATS result_classifications; -COMPUTE STATS result_concepts; -COMPUTE STATS result_datasources; -COMPUTE STATS result_languages; -COMPUTE STATS result_licenses; -COMPUTE STATS result_oids; -COMPUTE STATS result_organization; -COMPUTE STATS result_pids; -COMPUTE STATS result_projects; -COMPUTE STATS result_sources; -COMPUTE STATS result_topics; -COMPUTE STATS rndexpediture; -COMPUTE STATS roarmap; -COMPUTE STATS software; -COMPUTE STATS software_citations; -COMPUTE STATS software_classifications; -COMPUTE STATS software_concepts; -COMPUTE STATS software_datasources; -COMPUTE STATS software_languages; -COMPUTE STATS software_licenses; -COMPUTE STATS software_oids; -COMPUTE STATS software_pids; -COMPUTE STATS software_sources; -COMPUTE STATS software_topics; +-- Dropping old views +DROP VIEW IF EXISTS ${stats_db_shadow_name}.country; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.countrygdp; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_citations; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_classifications; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_concepts; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_datasources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_languages; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_licenses; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_oids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_pids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_sources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.dataset_topics; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.datasource; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.datasource_languages; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.datasource_oids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.datasource_organizations; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.datasource_results; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.fundref; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.numbers_country; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.organization; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.organization_datasources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.organization_projects; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_citations; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_classifications; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_concepts; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_datasources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_languages; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_licenses; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_oids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_pids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_sources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.otherresearchproduct_topics; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.project; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.project_oids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.project_organizations; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.project_results; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_citations; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_classifications; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_concepts; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_datasources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_languages; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_licenses; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_oids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_pids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_sources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.publication_topics; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_citations; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_classifications; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_concepts; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_datasources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_languages; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_licenses; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_oids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_organization; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_pids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_projects; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_sources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.result_topics; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.rndexpediture; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.roarmap; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_citations; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_classifications; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_concepts; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_datasources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_languages; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_licenses; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_oids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_pids; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_sources; +DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_topics; + + +-- Creating the shadow database, in case it doesn't exist +CREATE database ${stats_db_shadow_name}; + +-- Creating new views +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.country AS SELECT * FROM ${stats_db_name}.country; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.countrygdp AS SELECT * FROM ${stats_db_name}.countrygdp; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset AS SELECT * FROM ${stats_db_name}.dataset; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_citations AS SELECT * FROM ${stats_db_name}.dataset_citations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_classifications AS SELECT * FROM ${stats_db_name}.dataset_classifications; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_concepts AS SELECT * FROM ${stats_db_name}.dataset_concepts; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_datasources AS SELECT * FROM ${stats_db_name}.dataset_datasources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_languages AS SELECT * FROM ${stats_db_name}.dataset_languages; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_licenses AS SELECT * FROM ${stats_db_name}.dataset_licenses; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_oids AS SELECT * FROM ${stats_db_name}.dataset_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_pids AS SELECT * FROM ${stats_db_name}.dataset_pids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_sources AS SELECT * FROM ${stats_db_name}.dataset_sources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.dataset_topics AS SELECT * FROM ${stats_db_name}.dataset_topics; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource AS SELECT * FROM ${stats_db_name}.datasource; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_languages AS SELECT * FROM ${stats_db_name}.datasource_languages; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_oids AS SELECT * FROM ${stats_db_name}.datasource_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_organizations AS SELECT * FROM ${stats_db_name}.datasource_organizations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.datasource_results AS SELECT * FROM ${stats_db_name}.datasource_results; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.fundref AS SELECT * FROM ${stats_db_name}.fundref; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.numbers_country AS SELECT * FROM ${stats_db_name}.numbers_country; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.organization AS SELECT * FROM ${stats_db_name}.organization; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.organization_datasources AS SELECT * FROM ${stats_db_name}.organization_datasources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.organization_projects AS SELECT * FROM ${stats_db_name}.organization_projects; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct AS SELECT * FROM ${stats_db_name}.otherresearchproduct; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_citations AS SELECT * FROM ${stats_db_name}.otherresearchproduct_citations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_classifications AS SELECT * FROM ${stats_db_name}.otherresearchproduct_classifications; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_concepts AS SELECT * FROM ${stats_db_name}.otherresearchproduct_concepts; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_datasources AS SELECT * FROM ${stats_db_name}.otherresearchproduct_datasources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_languages AS SELECT * FROM ${stats_db_name}.otherresearchproduct_languages; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_licenses AS SELECT * FROM ${stats_db_name}.otherresearchproduct_licenses; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_oids AS SELECT * FROM ${stats_db_name}.otherresearchproduct_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_pids AS SELECT * FROM ${stats_db_name}.otherresearchproduct_pids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_sources AS SELECT * FROM ${stats_db_name}.otherresearchproduct_sources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.otherresearchproduct_topics AS SELECT * FROM ${stats_db_name}.otherresearchproduct_topics; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project AS SELECT * FROM ${stats_db_name}.project; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project_oids AS SELECT * FROM ${stats_db_name}.project_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project_organizations AS SELECT * FROM ${stats_db_name}.project_organizations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.project_results AS SELECT * FROM ${stats_db_name}.project_results; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication AS SELECT * FROM ${stats_db_name}.publication; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_citations AS SELECT * FROM ${stats_db_name}.publication_citations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_classifications AS SELECT * FROM ${stats_db_name}.publication_classifications; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_concepts AS SELECT * FROM ${stats_db_name}.publication_concepts; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_datasources AS SELECT * FROM ${stats_db_name}.publication_datasources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_languages AS SELECT * FROM ${stats_db_name}.publication_languages; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_licenses AS SELECT * FROM ${stats_db_name}.publication_licenses; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_oids AS SELECT * FROM ${stats_db_name}.publication_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_pids AS SELECT * FROM ${stats_db_name}.publication_pids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_sources AS SELECT * FROM ${stats_db_name}.publication_sources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.publication_topics AS SELECT * FROM ${stats_db_name}.publication_topics; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result AS SELECT * FROM ${stats_db_name}.result; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_citations AS SELECT * FROM ${stats_db_name}.result_citations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_classifications AS SELECT * FROM ${stats_db_name}.result_classifications; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_concepts AS SELECT * FROM ${stats_db_name}.result_concepts; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_datasources AS SELECT * FROM ${stats_db_name}.result_datasources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_languages AS SELECT * FROM ${stats_db_name}.result_languages; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_licenses AS SELECT * FROM ${stats_db_name}.result_licenses; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_oids AS SELECT * FROM ${stats_db_name}.result_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_organization AS SELECT * FROM ${stats_db_name}.result_organization; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_pids AS SELECT * FROM ${stats_db_name}.result_pids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_projects AS SELECT * FROM ${stats_db_name}.result_projects; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_sources AS SELECT * FROM ${stats_db_name}.result_sources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.result_topics AS SELECT * FROM ${stats_db_name}.result_topics; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.rndexpediture AS SELECT * FROM ${stats_db_name}.rndexpediture; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.roarmap AS SELECT * FROM ${stats_db_name}.roarmap; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software AS SELECT * FROM ${stats_db_name}.software; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_citations AS SELECT * FROM ${stats_db_name}.software_citations; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_classifications AS SELECT * FROM ${stats_db_name}.software_classifications; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_concepts AS SELECT * FROM ${stats_db_name}.software_concepts; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_datasources AS SELECT * FROM ${stats_db_name}.software_datasources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_languages AS SELECT * FROM ${stats_db_name}.software_languages; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_licenses AS SELECT * FROM ${stats_db_name}.software_licenses; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_oids AS SELECT * FROM ${stats_db_name}.software_oids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_pids AS SELECT * FROM ${stats_db_name}.software_pids; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_sources AS SELECT * FROM ${stats_db_name}.software_sources; +CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.software_topics AS SELECT * FROM ${stats_db_name}.software_topics; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step18.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step18.sql new file mode 100644 index 000000000..5645db309 --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step18.sql @@ -0,0 +1,81 @@ +------------------------------------------------------ +------------------------------------------------------ +-- Impala table statistics - Needed to make the tables +-- visible for impala +------------------------------------------------------ +------------------------------------------------------ + +COMPUTE STATS country; +COMPUTE STATS countrygdp; +COMPUTE STATS dataset; +COMPUTE STATS dataset_citations; +COMPUTE STATS dataset_classifications; +COMPUTE STATS dataset_concepts; +COMPUTE STATS dataset_datasources; +COMPUTE STATS dataset_languages; +COMPUTE STATS dataset_oids; +COMPUTE STATS dataset_pids; +COMPUTE STATS dataset_sources; +COMPUTE STATS dataset_topics; +COMPUTE STATS datasource; +COMPUTE STATS datasource_languages; +COMPUTE STATS datasource_oids; +COMPUTE STATS datasource_organizations; +COMPUTE STATS datasource_results; +COMPUTE STATS fundref; +COMPUTE STATS numbers_country; +COMPUTE STATS organization; +COMPUTE STATS organization_datasources; +COMPUTE STATS organization_projects; +COMPUTE STATS otherresearchproduct; +COMPUTE STATS otherresearchproduct_citations; +COMPUTE STATS otherresearchproduct_classifications; +COMPUTE STATS otherresearchproduct_concepts; +COMPUTE STATS otherresearchproduct_datasources; +COMPUTE STATS otherresearchproduct_languages; +COMPUTE STATS otherresearchproduct_licenses; +COMPUTE STATS otherresearchproduct_oids; +COMPUTE STATS otherresearchproduct_pids; +COMPUTE STATS otherresearchproduct_sources; +COMPUTE STATS otherresearchproduct_topics; +COMPUTE STATS project; +COMPUTE STATS project_oids; +COMPUTE STATS project_organizations; +COMPUTE STATS project_results; +COMPUTE STATS publication; +COMPUTE STATS publication_citations; +COMPUTE STATS publication_classifications; +COMPUTE STATS publication_concepts; +COMPUTE STATS publication_datasources; +COMPUTE STATS publication_languages; +COMPUTE STATS publication_licenses; +COMPUTE STATS publication_oids; +COMPUTE STATS publication_pids; +COMPUTE STATS publication_sources; +COMPUTE STATS publication_topics; +COMPUTE STATS result; +COMPUTE STATS result_citations; +COMPUTE STATS result_classifications; +COMPUTE STATS result_concepts; +COMPUTE STATS result_datasources; +COMPUTE STATS result_languages; +COMPUTE STATS result_licenses; +COMPUTE STATS result_oids; +COMPUTE STATS result_organization; +COMPUTE STATS result_pids; +COMPUTE STATS result_projects; +COMPUTE STATS result_sources; +COMPUTE STATS result_topics; +COMPUTE STATS rndexpediture; +COMPUTE STATS roarmap; +COMPUTE STATS software; +COMPUTE STATS software_citations; +COMPUTE STATS software_classifications; +COMPUTE STATS software_concepts; +COMPUTE STATS software_datasources; +COMPUTE STATS software_languages; +COMPUTE STATS software_licenses; +COMPUTE STATS software_oids; +COMPUTE STATS software_pids; +COMPUTE STATS software_sources; +COMPUTE STATS software_topics; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index 13b4bfaf7..9b0345a32 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml @@ -218,11 +218,22 @@ - + ${hive_jdbc_url} + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + + + + + + + + ${hive_jdbc_url} + stats_db_name=${stats_db_name} stats_db_shadow_name=${stats_db_shadow_name} @@ -230,14 +241,14 @@ - + ${jobTracker} ${nameNode} impala-shell.sh ${stats_db_name} - step17.sql - /user/${wf:user()}/oa/graph/stats/oozie_app/scripts/step17.sql + step18.sql + /user/${wf:user()}/oa/graph/stats/oozie_app/scripts/step18.sql impala-shell.sh From de49173420dc973a421e1655188b578b7f30b61a Mon Sep 17 00:00:00 2001 From: antleb Date: Wed, 1 Jul 2020 13:23:59 +0300 Subject: [PATCH 089/100] fixed a typo in queries --- .../dhp/oa/graph/stats/oozie_app/scripts/step15.sql | 13 ++++--------- .../dhp/oa/graph/stats/oozie_app/workflow.xml | 2 +- 2 files changed, 5 insertions(+), 10 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql index 41d80405d..60b37048b 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql @@ -6,28 +6,23 @@ ------------------------------------------------------ ------------------------------------------------------ --- To be commented out after the following queries are adapted to --- the new openAIRE graph schema. --- When it is ready they should be added to the to statistic recalculation --- for the impala tables and the shadow views - CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_refereed as -select substr(r.id, 4) as id, inst.refereed.classsname as refereed +select substr(r.id, 4) as id, inst.refereed.classname as refereed from ${openaire_db_name}.publication r lateral view explode(r.instance) instances as inst where r.datainfo.deletedbyinference=false; CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_refereed as -select substr(r.id, 4) as id, inst.refereed.classsname as refereed +select substr(r.id, 4) as id, inst.refereed.classname as refereed from ${openaire_db_name}.dataset r lateral view explode(r.instance) instances as inst where r.datainfo.deletedbyinference=false; CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_refereed as -select substr(r.id, 4) as id, inst.refereed.classsname as refereed +select substr(r.id, 4) as id, inst.refereed.classname as refereed from ${openaire_db_name}.software r lateral view explode(r.instance) instances as inst where r.datainfo.deletedbyinference=false; CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_refereed as -select substr(r.id, 4) as id, inst.refereed.classsname as refereed +select substr(r.id, 4) as id, inst.refereed.classname as refereed from ${openaire_db_name}.otherresearchproduct r lateral view explode(r.instance) instances as inst where r.datainfo.deletedbyinference=false; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index 9b0345a32..d7fc0564b 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml @@ -237,7 +237,7 @@ stats_db_name=${stats_db_name} stats_db_shadow_name=${stats_db_shadow_name} - + From e6f50de6ef649cd56df04099e4b0a4a810494975 Mon Sep 17 00:00:00 2001 From: antleb Date: Thu, 2 Jul 2020 02:42:30 +0300 Subject: [PATCH 090/100] Separated impala from hive steps --- .../stats/oozie_app/scripts/step15_5.sql | 6 ++ .../graph/stats/oozie_app/scripts/step16.sql | 59 +------------------ .../stats/oozie_app/scripts/step16_5.sql | 55 +++++++++++++++++ .../graph/stats/oozie_app/scripts/step9.sql | 2 +- .../dhp/oa/graph/stats/oozie_app/workflow.xml | 27 ++++++++- 5 files changed, 89 insertions(+), 60 deletions(-) create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql new file mode 100644 index 000000000..30abfeb71 --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql @@ -0,0 +1,6 @@ +-- datasource sources: +-- where the datasource info have been collected from. +create table if not exists ${stats_db_name}.datasource_sources AS +select substr(d.id,4) as id, substr(cf.key, 4) as datasource +from ${openaire_db_name}.datasource d lateral view explode(d.collectedfrom) cfrom as cf +where d.datainfo.deletedbyinference=false; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql index 8e361c1ad..bc10831cc 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql @@ -2,7 +2,6 @@ -- Shortcuts for various definitions in stats db --- ---------------------------------------------------- - -- Peer reviewed: -- Results that have been collected from Crossref create table ${stats_db_name}.result_peerreviewed as @@ -74,60 +73,4 @@ join ${stats_db_name}.result_datasources rd on rd.id=r.id join ${stats_db_name}.datasource d on d.id=rd.datasource join ${stats_db_name}.datasource_organizations dor on dor.id=d.id join ${stats_db_name}.organization o on o.id=dor.organization -where o.country is not null and o.country!=''; - --- replace the creation of the result view to include the boolean fields from the previous tables (green, gold, --- peer reviewed) -drop table if exists ${stats_db_name}.result_tmp; -CREATE TABLE result_tmp ( - id STRING, - title STRING, - publisher STRING, - journal STRING, - `date` STRING, - `year` INT, - bestlicence STRING, - access_mode STRING, - embargo_end_date STRING, - delayed BOOLEAN, - authors INT, - source STRING, - abstract BOOLEAN, - type STRING , - peer_reviewed BOOLEAN, - green BOOLEAN, - gold BOOLEAN) -clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); - -insert into ${stats_db_name}.result_tmp -select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold -FROM ${stats_db_name}.publication r -LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; - -insert into ${stats_db_name}.result_tmp -select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold -FROM ${stats_db_name}.dataset r -LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; - -insert into ${stats_db_name}.result_tmp -select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold -FROM ${stats_db_name}.software r -LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; - -insert into ${stats_db_name}.result_tmp -select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold -FROM ${stats_db_name}.otherresearchproduct r -LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id -LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; - -drop table if exists ${stats_db_name}.result; -drop view if exists ${stats_db_name}.result; -create table ${stats_db_name}.result stored as parquet as select * from ${stats_db_name}.result_tmp; -drop table ${stats_db_name}.result_tmp; \ No newline at end of file +where o.country is not null and o.country!=''; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql new file mode 100644 index 000000000..2bd242cd7 --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql @@ -0,0 +1,55 @@ +-- replace the creation of the result view to include the boolean fields from the previous tables (green, gold, +-- peer reviewed) +drop table if exists ${stats_db_name}.result_tmp; +CREATE TABLE result_tmp ( + id STRING, + title STRING, + publisher STRING, + journal STRING, + `date` STRING, + `year` INT, + bestlicence STRING, + access_mode STRING, + embargo_end_date STRING, + delayed BOOLEAN, + authors INT, + source STRING, + abstract BOOLEAN, + type STRING , + peer_reviewed BOOLEAN, + green BOOLEAN, + gold BOOLEAN) +clustered by (id) into 100 buckets stored as orc tblproperties('transactional'='true'); + +insert into ${stats_db_name}.result_tmp +select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold +FROM ${stats_db_name}.publication r +LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id +LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id +LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; + +insert into ${stats_db_name}.result_tmp +select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold +FROM ${stats_db_name}.dataset r +LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id +LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id +LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; + +insert into ${stats_db_name}.result_tmp +select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold +FROM ${stats_db_name}.software r +LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id +LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id +LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; + +insert into ${stats_db_name}.result_tmp +select r.id, r.title, r.publisher, r.journal, r.`date`, date_format(r.`date`, 'yyyy'), r.bestlicence, r.bestlicence, r.embargo_end_date, r.delayed, r.authors, r.source, r.abstract, r.type, pr.peer_reviewed, green.green, gold.gold +FROM ${stats_db_name}.otherresearchproduct r +LEFT OUTER JOIN ${stats_db_name}.result_peerreviewed pr on pr.id=r.id +LEFT OUTER JOIN ${stats_db_name}.result_greenoa green on green.id=r.id +LEFT OUTER JOIN ${stats_db_name}.result_gold gold on gold.id=r.id; + +drop table if exists ${stats_db_name}.result; +drop view if exists ${stats_db_name}.result; +create table ${stats_db_name}.result stored as parquet as select * from ${stats_db_name}.result_tmp; +drop table ${stats_db_name}.result_tmp; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql index 9bbf07e15..a918e4de4 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql @@ -4,7 +4,7 @@ ---------------------------------------------------------------- ---------------------------------------------------------------- DROP TABLE IF EXISTS ${stats_db_name}.organization; -CREATE TABLE IF NOT EXISTS ${stats_db_name}.organization AS SELECT substr(o.id, 4) as id, o.legalname.value as name, o.legalshortname as legalshortname, o.country.classid as country +CREATE TABLE IF NOT EXISTS ${stats_db_name}.organization AS SELECT substr(o.id, 4) as id, o.legalname.value as name, o.legalshortname.value as legalshortname, o.country.classid as country FROM ${openaire_db_name}.organization o WHERE o.datainfo.deletedbyinference=FALSE; CREATE OR REPLACE VIEW ${stats_db_name}.organization_datasources AS SELECT organization AS id, id AS datasource FROM ${stats_db_name}.datasource_organizations; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index d7fc0564b..296241569 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml @@ -215,14 +215,39 @@ stats_db_name=${stats_db_name} openaire_db_name=${openaire_db_name} + + + + + + + ${hive_jdbc_url} + + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + + + ${jobTracker} + ${nameNode} + impala-shell.sh + ${stats_db_name} + step16.sql + /user/${wf:user()}/oa/graph/stats/oozie_app/scripts/step16.sql + impala-shell.sh + + + + + + ${hive_jdbc_url} - + stats_db_name=${stats_db_name} openaire_db_name=${openaire_db_name} From dd3d6a6e154be3d270a290f09b3526a544d5189e Mon Sep 17 00:00:00 2001 From: antleb Date: Fri, 3 Jul 2020 01:14:34 +0300 Subject: [PATCH 091/100] compute stats for the used and new impala tables --- .../graph/stats/oozie_app/scripts/step16.sql | 24 ++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql index bc10831cc..3d21a9d68 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql @@ -1,7 +1,19 @@ ---------------------------------------------------- -- Shortcuts for various definitions in stats db --- +-- since these statements are executed using Impala, +-- we'll have to compute the stats for the tables we use ---------------------------------------------------- +COMPUTE STATS result; +COMPUTE STATS result_sources; +COMPUTE STATS datasource; +COMPUTE STATS result_datasources; +COMPUTE STATS datasource_sources; +COMPUTE STATS country; +COMPUTE STATS result_organization; +COMPUTE STATS organization; +COMPUTE STATS datasource_organizations; + -- Peer reviewed: -- Results that have been collected from Crossref create table ${stats_db_name}.result_peerreviewed as @@ -18,6 +30,8 @@ select distinct r.id as id, false as peer_reviewed from ${stats_db_name}.result r where r.id not in (select id from peer_reviewed); +COMPUTE STATS result_peerreviewed; + -- Green OA: -- OA results that are hosted by an Institutional repository and have NOT been harvested from a DOAJ journal. create table ${stats_db_name}.result_greenoa as @@ -39,6 +53,8 @@ select distinct r.id as id, false as green from ${stats_db_name}.result r where r.id not in (select id from result_green); +COMPUTE STATS result_greenoa; + -- GOLD OA: -- OA results that have been harvested from a DOAJ journal. create table ${stats_db_name}.result_gold as @@ -57,6 +73,8 @@ select distinct r.id, false as gold from ${stats_db_name}.result r where r.id not in (select id from result_gold); +COMPUTE STATS result_gold; + -- shortcut result-country through the organization affiliation create table ${stats_db_name}.result_affiliated_country as select r.id as id, o.country as country @@ -65,6 +83,8 @@ join ${stats_db_name}.result_organization ro on ro.id=r.id join ${stats_db_name}.organization o on o.id=ro.organization where o.country is not null and o.country!=''; +COMPUTE STATS result_affiliated_country; + -- shortcut result-country through datasource of deposition create table ${stats_db_name}.result_deposited_country as select r.id as id, o.country as country @@ -73,4 +93,6 @@ join ${stats_db_name}.result_datasources rd on rd.id=r.id join ${stats_db_name}.datasource d on d.id=rd.datasource join ${stats_db_name}.datasource_organizations dor on dor.id=d.id join ${stats_db_name}.organization o on o.id=dor.organization -where o.country is not null and o.country!=''; \ No newline at end of file +where o.country is not null and o.country!=''; + +COMPUTE STATS result_deposited_country; \ No newline at end of file From c9cfc165d9f299f2db92b1d0c3b34772754af50c Mon Sep 17 00:00:00 2001 From: antleb Date: Fri, 3 Jul 2020 15:24:57 +0300 Subject: [PATCH 092/100] not using impala since the resulting tables are not visible --- .../graph/stats/oozie_app/scripts/step16.sql | 38 +++++-------------- .../dhp/oa/graph/stats/oozie_app/workflow.xml | 26 +++---------- 2 files changed, 16 insertions(+), 48 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql index 3d21a9d68..33849b960 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16.sql @@ -1,19 +1,7 @@ ---------------------------------------------------- -- Shortcuts for various definitions in stats db --- --- since these statements are executed using Impala, --- we'll have to compute the stats for the tables we use ---------------------------------------------------- -COMPUTE STATS result; -COMPUTE STATS result_sources; -COMPUTE STATS datasource; -COMPUTE STATS result_datasources; -COMPUTE STATS datasource_sources; -COMPUTE STATS country; -COMPUTE STATS result_organization; -COMPUTE STATS organization; -COMPUTE STATS datasource_organizations; - -- Peer reviewed: -- Results that have been collected from Crossref create table ${stats_db_name}.result_peerreviewed as @@ -28,9 +16,8 @@ from peer_reviewed union all select distinct r.id as id, false as peer_reviewed from ${stats_db_name}.result r -where r.id not in (select id from peer_reviewed); - -COMPUTE STATS result_peerreviewed; +left outer join peer_reviewed pr on pr.id=r.id +where pr.id is null; -- Green OA: -- OA results that are hosted by an Institutional repository and have NOT been harvested from a DOAJ journal. @@ -40,20 +27,21 @@ with result_green as ( from ${stats_db_name}.result r join ${stats_db_name}.result_datasources rd on rd.id=r.id join ${stats_db_name}.datasource d on d.id=rd.datasource - where r.bestlicence in ('Open Access', 'Open Source') and d.type='Institutional Repository' and not exists ( - select 1 from ${stats_db_name}.result_datasources rd + left outer join ( + select rd.id from ${stats_db_name}.result_datasources rd join ${stats_db_name}.datasource d on rd.datasource=d.id join ${stats_db_name}.datasource_sources sds on sds.id=d.id join ${stats_db_name}.datasource sd on sd.id=sds.datasource - where sd.name='DOAJ-ARTICLES' and rd.id=r.id)) + where sd.name='DOAJ-ARTICLES' + ) as doaj on doaj.id=r.id + where r.bestlicence in ('Open Access', 'Open Source') and d.type='Institutional Repository' and doaj.id is null) select distinct result_green.id, true as green from result_green union all select distinct r.id as id, false as green from ${stats_db_name}.result r -where r.id not in (select id from result_green); - -COMPUTE STATS result_greenoa; +left outer join result_green rg on rg.id=r.id +where rg.id is null; -- GOLD OA: -- OA results that have been harvested from a DOAJ journal. @@ -73,8 +61,6 @@ select distinct r.id, false as gold from ${stats_db_name}.result r where r.id not in (select id from result_gold); -COMPUTE STATS result_gold; - -- shortcut result-country through the organization affiliation create table ${stats_db_name}.result_affiliated_country as select r.id as id, o.country as country @@ -83,8 +69,6 @@ join ${stats_db_name}.result_organization ro on ro.id=r.id join ${stats_db_name}.organization o on o.id=ro.organization where o.country is not null and o.country!=''; -COMPUTE STATS result_affiliated_country; - -- shortcut result-country through datasource of deposition create table ${stats_db_name}.result_deposited_country as select r.id as id, o.country as country @@ -93,6 +77,4 @@ join ${stats_db_name}.result_datasources rd on rd.id=r.id join ${stats_db_name}.datasource d on d.id=rd.datasource join ${stats_db_name}.datasource_organizations dor on dor.id=d.id join ${stats_db_name}.organization o on o.id=dor.organization -where o.country is not null and o.country!=''; - -COMPUTE STATS result_deposited_country; \ No newline at end of file +where o.country is not null and o.country!=''; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index 296241569..174d78901 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml @@ -215,31 +215,17 @@ stats_db_name=${stats_db_name} openaire_db_name=${openaire_db_name} - - - - - - - ${hive_jdbc_url} - - stats_db_name=${stats_db_name} - openaire_db_name=${openaire_db_name} - - - ${jobTracker} - ${nameNode} - impala-shell.sh - ${stats_db_name} - step16.sql - /user/${wf:user()}/oa/graph/stats/oozie_app/scripts/step16.sql - impala-shell.sh - + + ${hive_jdbc_url} + + stats_db_name=${stats_db_name} + openaire_db_name=${openaire_db_name} + From a8da4ab9c0462e05d42b31e54834ab476010a70f Mon Sep 17 00:00:00 2001 From: antleb Date: Fri, 3 Jul 2020 15:25:43 +0300 Subject: [PATCH 093/100] years in projects are now integers --- .../dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql index e77964513..df2d142d4 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql @@ -1,3 +1,5 @@ +-- noinspection SqlNoDataSourceInspectionForFile + ------------------------------------------------------ ------------------------------------------------------ -- Project table/view and Project related tables/views @@ -19,6 +21,6 @@ CREATE TABLE ${stats_db_name}.project_results AS SELECT substr(r.target, 4) AS i ---------------- -- Creating and populating temporary Project table DROP TABLE IF EXISTS ${stats_db_name}.project_tmp; -CREATE TABLE ${stats_db_name}.project_tmp (id STRING, acronym STRING, title STRING, funder STRING, funding_lvl0 STRING, funding_lvl1 STRING, funding_lvl2 STRING, ec39 STRING, type STRING, startdate STRING, enddate STRING, start_year STRING, end_year STRING, duration INT, haspubs STRING, numpubs INT, daysforlastpub INT, delayedpubs INT, callidentifier STRING, code STRING) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties('transactional'='true'); +CREATE TABLE ${stats_db_name}.project_tmp (id STRING, acronym STRING, title STRING, funder STRING, funding_lvl0 STRING, funding_lvl1 STRING, funding_lvl2 STRING, ec39 STRING, type STRING, startdate STRING, enddate STRING, start_year INT, end_year INT, duration INT, haspubs STRING, numpubs INT, daysforlastpub INT, delayedpubs INT, callidentifier STRING, code STRING) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties('transactional'='true'); -INSERT INTO ${stats_db_name}.project_tmp SELECT substr(p.id, 4) AS id, p.acronym.value AS acronym, p.title.value AS title, xpath_string(p.fundingtree[0].value, '//funder/name') AS funder, xpath_string(p.fundingtree[0].value, '//funding_level_0/name') AS funding_lvl0, xpath_string(p.fundingtree[0].value, '//funding_level_1/name') AS funding_lvl1, xpath_string(p.fundingtree[0].value, '//funding_level_2/name') AS funding_lvl2, p.ecsc39.value AS ec39, p.contracttype.classname AS type, p.startdate.value AS startdate, p.enddate.value AS enddate, date_format(p.startdate.value, 'yyyy') AS start_year, date_format(p.enddate.value, 'yyyy') AS end_year, 0 AS duration, 'no' AS haspubs, 0 AS numpubs, 0 AS daysforlastpub, 0 AS delayedpubs, p.callidentifier.value AS callidentifier, p.code.value AS code FROM ${openaire_db_name}.project p WHERE p.datainfo.deletedbyinference=false; +INSERT INTO ${stats_db_name}.project_tmp SELECT substr(p.id, 4) AS id, p.acronym.value AS acronym, p.title.value AS title, xpath_string(p.fundingtree[0].value, '//funder/name') AS funder, xpath_string(p.fundingtree[0].value, '//funding_level_0/name') AS funding_lvl0, xpath_string(p.fundingtree[0].value, '//funding_level_1/name') AS funding_lvl1, xpath_string(p.fundingtree[0].value, '//funding_level_2/name') AS funding_lvl2, p.ecsc39.value AS ec39, p.contracttype.classname AS type, p.startdate.value AS startdate, p.enddate.value AS enddate, year(p.startdate.value) AS start_year, year(p.enddate.value) AS end_year, 0 AS duration, 'no' AS haspubs, 0 AS numpubs, 0 AS daysforlastpub, 0 AS delayedpubs, p.callidentifier.value AS callidentifier, p.code.value AS code FROM ${openaire_db_name}.project p WHERE p.datainfo.deletedbyinference=false; From 25b7a615f5824975e6e2e8004d9e24f61cfea05d Mon Sep 17 00:00:00 2001 From: antleb Date: Fri, 3 Jul 2020 15:27:02 +0300 Subject: [PATCH 094/100] moved datasource_sources table creating in the datasource section --- .../dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql | 6 ------ .../dhp/oa/graph/stats/oozie_app/scripts/step8.sql | 8 ++++++++ 2 files changed, 8 insertions(+), 6 deletions(-) delete mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql deleted file mode 100644 index 30abfeb71..000000000 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql +++ /dev/null @@ -1,6 +0,0 @@ --- datasource sources: --- where the datasource info have been collected from. -create table if not exists ${stats_db_name}.datasource_sources AS -select substr(d.id,4) as id, substr(cf.key, 4) as datasource -from ${openaire_db_name}.datasource d lateral view explode(d.collectedfrom) cfrom as cf -where d.datainfo.deletedbyinference=false; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql index 609132781..4e13b3dd8 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql @@ -1,3 +1,5 @@ +-- noinspection SqlNoDataSourceInspectionForFile + ------------------------------------------------------------ ------------------------------------------------------------ -- Datasource table/view and Datasource related tables/views @@ -47,4 +49,10 @@ CREATE TABLE ${stats_db_name}.datasource_oids AS SELECT substr(d.id, 4) AS id, o DROP TABLE IF EXISTS ${stats_db_name}.datasource_organizations; CREATE TABLE ${stats_db_name}.datasource_organizations AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS organization FROM ${openaire_db_name}.relation r WHERE r.reltype='datasourceOrganization'; +-- datasource sources: +-- where the datasource info have been collected from. +create table if not exists ${stats_db_name}.datasource_sources AS select substr(d.id,4) as id, substr(cf.key, 4) as datasource from ${openaire_db_name}.datasource d lateral view explode(d.collectedfrom) cfrom as cf where d.datainfo.deletedbyinference=false; + CREATE OR REPLACE VIEW ${stats_db_name}.datasource_results AS SELECT datasource AS id, id AS result FROM ${stats_db_name}.result_datasources; + + From cecbbfa0fc00991357c1d303233173110d5e2a76 Mon Sep 17 00:00:00 2001 From: antleb Date: Mon, 6 Jul 2020 15:16:58 +0300 Subject: [PATCH 095/100] added missing tables and views: contexts, creation_date, funder --- .../dhp/oa/graph/stats/oozie_app/scripts/step10.sql | 11 +++++++++++ .../dhp/oa/graph/stats/oozie_app/scripts/step6.sql | 4 ++++ 2 files changed, 15 insertions(+) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql index 145be77b0..2baaba254 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql @@ -8,3 +8,14 @@ CREATE OR REPLACE VIEW ${stats_db_name}.country AS SELECT * FROM ${external_sta CREATE OR REPLACE VIEW ${stats_db_name}.countrygdp AS SELECT * FROM ${external_stats_db_name}.countrygdp; CREATE OR REPLACE VIEW ${stats_db_name}.roarmap AS SELECT * FROM ${external_stats_db_name}.roarmap; CREATE OR REPLACE VIEW ${stats_db_name}.rndexpediture AS SELECT * FROM ${external_stats_db_name}.rndexpediture; +CREATE OR REPLACE VIEW ${stats_db_name}.context AS SELECT * FROM ${external_stats_db_name}.context; +CREATE OR REPLACE VIEW ${stats_db_name}.category AS SELECT * FROM ${external_stats_db_name}.category; +CREATE OR REPLACE VIEW ${stats_db_name}.concept AS SELECT * FROM ${external_stats_db_name}.concept; + + +------------------------------------------------------------------------------------------------ +------------------------------------------------------------------------------------------------ +-- Creation date of the database +------------------------------------------------------------------------------------------------ +------------------------------------------------------------------------------------------------ +create table creation_date as select date_format(current_date(), 'dd-MM-yyyy') as date; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql index df2d142d4..c16d924ba 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql @@ -24,3 +24,7 @@ DROP TABLE IF EXISTS ${stats_db_name}.project_tmp; CREATE TABLE ${stats_db_name}.project_tmp (id STRING, acronym STRING, title STRING, funder STRING, funding_lvl0 STRING, funding_lvl1 STRING, funding_lvl2 STRING, ec39 STRING, type STRING, startdate STRING, enddate STRING, start_year INT, end_year INT, duration INT, haspubs STRING, numpubs INT, daysforlastpub INT, delayedpubs INT, callidentifier STRING, code STRING) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties('transactional'='true'); INSERT INTO ${stats_db_name}.project_tmp SELECT substr(p.id, 4) AS id, p.acronym.value AS acronym, p.title.value AS title, xpath_string(p.fundingtree[0].value, '//funder/name') AS funder, xpath_string(p.fundingtree[0].value, '//funding_level_0/name') AS funding_lvl0, xpath_string(p.fundingtree[0].value, '//funding_level_1/name') AS funding_lvl1, xpath_string(p.fundingtree[0].value, '//funding_level_2/name') AS funding_lvl2, p.ecsc39.value AS ec39, p.contracttype.classname AS type, p.startdate.value AS startdate, p.enddate.value AS enddate, year(p.startdate.value) AS start_year, year(p.enddate.value) AS end_year, 0 AS duration, 'no' AS haspubs, 0 AS numpubs, 0 AS daysforlastpub, 0 AS delayedpubs, p.callidentifier.value AS callidentifier, p.code.value AS code FROM ${openaire_db_name}.project p WHERE p.datainfo.deletedbyinference=false; + +create table ${stats_db_name}.funder as +select distinct xpath_string(fund, '//funder/id') as id, xpath_string(fund, '//funder/name') as name, xpath_string(fund, '//funder/shortname') as shortname +from project p lateral view explode(p.fundingtree.value) fundingtree as fund From 1ee7eeedf3dfdd58052c445d184d6773fb8e89a2 Mon Sep 17 00:00:00 2001 From: antleb Date: Fri, 10 Jul 2020 00:18:49 +0300 Subject: [PATCH 096/100] added the source db name... --- .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql index c16d924ba..85fa7b500 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql @@ -27,4 +27,4 @@ INSERT INTO ${stats_db_name}.project_tmp SELECT substr(p.id, 4) AS id, p.acronym create table ${stats_db_name}.funder as select distinct xpath_string(fund, '//funder/id') as id, xpath_string(fund, '//funder/name') as name, xpath_string(fund, '//funder/shortname') as shortname -from project p lateral view explode(p.fundingtree.value) fundingtree as fund +from ${openaire_db_name}.project p lateral view explode(p.fundingtree.value) fundingtree as fund From b16bb68b9fa2f6d7c424276842d214654ae4586c Mon Sep 17 00:00:00 2001 From: antleb Date: Fri, 10 Jul 2020 02:52:01 +0300 Subject: [PATCH 097/100] added the target db name... --- .../dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql index 2bd242cd7..f737c1ea6 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql @@ -1,7 +1,7 @@ -- replace the creation of the result view to include the boolean fields from the previous tables (green, gold, -- peer reviewed) drop table if exists ${stats_db_name}.result_tmp; -CREATE TABLE result_tmp ( +CREATE TABLE ${stats_db_name}.result_tmp ( id STRING, title STRING, publisher STRING, From e217d496ab6274c6c7433fa8ac630bcc0afc3752 Mon Sep 17 00:00:00 2001 From: antleb Date: Thu, 16 Jul 2020 17:01:12 +0300 Subject: [PATCH 098/100] added the dest db... --- .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql index 2baaba254..46ff295f4 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql @@ -18,4 +18,4 @@ CREATE OR REPLACE VIEW ${stats_db_name}.concept AS SELECT * FROM ${external_sta -- Creation date of the database ------------------------------------------------------------------------------------------------ ------------------------------------------------------------------------------------------------ -create table creation_date as select date_format(current_date(), 'dd-MM-yyyy') as date; \ No newline at end of file +create table ${stats_db_name}.creation_date as select date_format(current_date(), 'dd-MM-yyyy') as date; \ No newline at end of file From 18d9464b5275a402e758f9e1907dac14531e104e Mon Sep 17 00:00:00 2001 From: antleb Date: Fri, 17 Jul 2020 12:39:35 +0300 Subject: [PATCH 099/100] creating shadow db only if it not exists... --- .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step17.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step17.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step17.sql index 8df0072b3..e002f656e 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step17.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step17.sql @@ -83,7 +83,7 @@ DROP VIEW IF EXISTS ${stats_db_shadow_name}.software_topics; -- Creating the shadow database, in case it doesn't exist -CREATE database ${stats_db_shadow_name}; +CREATE database IF NOT EXISTS ${stats_db_shadow_name}; -- Creating new views CREATE VIEW IF NOT EXISTS ${stats_db_shadow_name}.country AS SELECT * FROM ${stats_db_name}.country; From 4ac8ebe427bf4860641f850db37a368e3dce18d8 Mon Sep 17 00:00:00 2001 From: antleb Date: Wed, 22 Jul 2020 15:22:25 +0300 Subject: [PATCH 100/100] correctly calculating the project duration --- .../eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql index 85fa7b500..21a944164 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql @@ -23,7 +23,7 @@ CREATE TABLE ${stats_db_name}.project_results AS SELECT substr(r.target, 4) AS i DROP TABLE IF EXISTS ${stats_db_name}.project_tmp; CREATE TABLE ${stats_db_name}.project_tmp (id STRING, acronym STRING, title STRING, funder STRING, funding_lvl0 STRING, funding_lvl1 STRING, funding_lvl2 STRING, ec39 STRING, type STRING, startdate STRING, enddate STRING, start_year INT, end_year INT, duration INT, haspubs STRING, numpubs INT, daysforlastpub INT, delayedpubs INT, callidentifier STRING, code STRING) CLUSTERED BY (id) INTO 100 buckets stored AS orc tblproperties('transactional'='true'); -INSERT INTO ${stats_db_name}.project_tmp SELECT substr(p.id, 4) AS id, p.acronym.value AS acronym, p.title.value AS title, xpath_string(p.fundingtree[0].value, '//funder/name') AS funder, xpath_string(p.fundingtree[0].value, '//funding_level_0/name') AS funding_lvl0, xpath_string(p.fundingtree[0].value, '//funding_level_1/name') AS funding_lvl1, xpath_string(p.fundingtree[0].value, '//funding_level_2/name') AS funding_lvl2, p.ecsc39.value AS ec39, p.contracttype.classname AS type, p.startdate.value AS startdate, p.enddate.value AS enddate, year(p.startdate.value) AS start_year, year(p.enddate.value) AS end_year, 0 AS duration, 'no' AS haspubs, 0 AS numpubs, 0 AS daysforlastpub, 0 AS delayedpubs, p.callidentifier.value AS callidentifier, p.code.value AS code FROM ${openaire_db_name}.project p WHERE p.datainfo.deletedbyinference=false; +INSERT INTO ${stats_db_name}.project_tmp SELECT substr(p.id, 4) AS id, p.acronym.value AS acronym, p.title.value AS title, xpath_string(p.fundingtree[0].value, '//funder/name') AS funder, xpath_string(p.fundingtree[0].value, '//funding_level_0/name') AS funding_lvl0, xpath_string(p.fundingtree[0].value, '//funding_level_1/name') AS funding_lvl1, xpath_string(p.fundingtree[0].value, '//funding_level_2/name') AS funding_lvl2, p.ecsc39.value AS ec39, p.contracttype.classname AS type, p.startdate.value AS startdate, p.enddate.value AS enddate, year(p.startdate.value) AS start_year, year(p.enddate.value) AS end_year, CAST(MONTHS_BETWEEN(p.enddate.value, p.startdate.value) AS INT) AS duration, 'no' AS haspubs, 0 AS numpubs, 0 AS daysforlastpub, 0 AS delayedpubs, p.callidentifier.value AS callidentifier, p.code.value AS code FROM ${openaire_db_name}.project p WHERE p.datainfo.deletedbyinference=false; create table ${stats_db_name}.funder as select distinct xpath_string(fund, '//funder/id') as id, xpath_string(fund, '//funder/name') as name, xpath_string(fund, '//funder/shortname') as shortname