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/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-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-broker-events/pom.xml b/dhp-workflows/dhp-broker-events/pom.xml index 424015a3c..f98708c64 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 @@ -51,9 +57,8 @@ - eu.dnetlib + eu.dnetlib.dhp dnet-openaire-broker-common - [3.0.3,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..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 @@ -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 { @@ -32,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); @@ -42,6 +47,7 @@ public class EventFactory { res.setCreationDate(now); res.setExpiryDate(calculateExpiryDate(now)); res.setInstantMessage(false); + return res; } @@ -51,8 +57,11 @@ public class EventFactory { final OaBrokerMainEntity source = updateInfo.getSource(); final OaBrokerMainEntity target = updateInfo.getTarget(); - map.setTargetDatasourceId(target.getCollectedFromId()); - map.setTargetDatasourceName(target.getCollectedFromName()); + final OaBrokerRelatedDatasource targetDs = updateInfo.getTargetDs(); + + map.setTargetDatasourceId(targetDs.getOpenaireId()); + map.setTargetDatasourceName(targetDs.getName()); + map.setTargetDatasourceType(targetDs.getType()); map.setTargetResultId(target.getOpenaireId()); @@ -71,18 +80,29 @@ public class EventFactory { // PROVENANCE INFO map.setTrust(updateInfo.getTrust()); - map.setProvenanceDatasourceId(source.getCollectedFromId()); - map.setProvenanceDatasourceName(source.getCollectedFromName()); 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; } - 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/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/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 dbe2fdd47..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 @@ -3,28 +3,28 @@ 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.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.api.java.function.MapFunction; +import org.apache.spark.SparkContext; 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; -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; 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 { @@ -47,57 +47,54 @@ 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); - final SparkConf conf = new SparkConf(); + final Set dsIdWhitelist = ClusterUtils.parseParamAsList(parser, "datasourceIdWhitelist"); + log.info("datasourceIdWhitelist: {}", StringUtils.join(dsIdWhitelist, ",")); - // TODO UNCOMMENT - // final DedupConfig dedupConfig = loadDedupConfig(isLookupUrl, dedupConfigProfileId); - final DedupConfig dedupConfig = null; + final Set dsTypeWhitelist = ClusterUtils.parseParamAsList(parser, "datasourceTypeWhitelist"); + log.info("datasourceTypeWhitelist: {}", StringUtils.join(dsTypeWhitelist, ",")); + + final Set dsIdBlacklist = ClusterUtils.parseParamAsList(parser, "datasourceIdBlacklist"); + log.info("datasourceIdBlacklist: {}", StringUtils.join(dsIdBlacklist, ",")); + + final SparkConf conf = new SparkConf(); runWithSparkSession(conf, isSparkSessionManaged, spark -> { ClusterUtils.removeDir(spark, eventsPath); + 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 + final Dataset dataset = groups .map( - (MapFunction) g -> EventFinder.generateEvents(g, dedupConfig), - Encoders.bean(EventGroup.class)) - .flatMap(group -> group.getData().iterator(), Encoders.bean(Event.class)); + g -> EventFinder + .generateEvents(g, dsIdWhitelist, dsIdBlacklist, dsTypeWhitelist, accumulators), + Encoders + .bean(EventGroup.class)) + .flatMap(g -> g.getData().iterator(), Encoders.bean(Event.class)); - events.write().mode(SaveMode.Overwrite).json(eventsPath); + ClusterUtils.save(dataset, eventsPath, Event.class, total); }); } - private static DedupConfig loadDedupConfig(final String isLookupUrl, final String profId) throws Exception { + public static Map prepareAccumulators(final SparkContext sc) { - final ISLookUpService isLookUpService = ISLookupClientFactory.getLookUpService(isLookupUrl); + return EventFinder + .getMatchers() + .stream() + .map(UpdateMatcher::accumulatorName) + .distinct() + .collect(Collectors.toMap(s -> s, s -> sc.longAccumulator(s))); - 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/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/IndexOnESJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/IndexOnESJob.java new file mode 100644 index 000000000..806147bdd --- /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; +import eu.dnetlib.dhp.broker.oa.util.ClusterUtils; + +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 = ClusterUtils + .readPath(spark, eventsPath, Event.class) + // .limit(10000) // TODO REMOVE + .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/JoinStep0Job.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/JoinStep0Job.java new file mode 100644 index 000000000..39fa76e43 --- /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.RelatedDatasource; +import eu.dnetlib.dhp.broker.oa.util.aggregators.withRels.RelatedDatasourceAggregator; +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 joinedEntitiesPath = workingPath + "/joinedEntities_step0"; + log.info("joinedEntitiesPath: {}", joinedEntitiesPath); + + final SparkConf conf = new SparkConf(); + + runWithSparkSession(conf, isSparkSessionManaged, spark -> { + + ClusterUtils.removeDir(spark, joinedEntitiesPath); + + final LongAccumulator total = spark.sparkContext().longAccumulator("total_entities"); + + final Dataset sources = ClusterUtils + .readPath(spark, workingPath + "/simpleEntities", OaBrokerMainEntity.class); + + final Dataset typedRels = ClusterUtils + .readPath(spark, workingPath + "/relatedDatasources", RelatedDatasource.class); + + final TypedColumn, OaBrokerMainEntity> aggr = new RelatedDatasourceAggregator() + .toColumn(); + + final Dataset dataset = sources + .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, joinedEntitiesPath, 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 1be782a12..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 @@ -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,8 +52,10 @@ 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); + .readPath(spark, workingPath + "/joinedEntities_step0", OaBrokerMainEntity.class); final Dataset typedRels = ClusterUtils .readPath(spark, workingPath + "/relatedProjects", RelatedProject.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..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,11 +7,10 @@ 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.TypedColumn; +import org.apache.spark.util.LongAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,6 +51,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 +62,13 @@ 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()) + .groupByKey(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/PrepareRelatedDatasourcesJob.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedDatasourcesJob.java new file mode 100644 index 000000000..166372a7f --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/PrepareRelatedDatasourcesJob.java @@ -0,0 +1,107 @@ + +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.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.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 { + + 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 + "/relatedDatasources"; + 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> 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)); + + 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/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 7f82f9a2b..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 @@ -12,11 +12,12 @@ 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.broker.objects.OaBrokerRelatedDatasource; 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 { @@ -34,20 +35,21 @@ 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 DedupConfig dedupConfig) { + 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(), dedupConfig); + getHighlightToStringFunction()); final String s = DigestUtils.md5Hex(info.getHighlightValueAsString()); if (!infoMap.containsKey(s) || infoMap.get(s).getTrust() < info.getTrust()) { @@ -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; } } @@ -80,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() { @@ -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 != 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/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/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 85b2cbe28..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 @@ -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 { @@ -16,27 +18,27 @@ 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<>(); + } + 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/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..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,11 +14,19 @@ 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; 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/ClusterUtils.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/ClusterUtils.java index de9b901d0..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,13 +1,21 @@ 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; +import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; +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 { @@ -44,4 +52,38 @@ 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); + } + + 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 b61d5e7cc..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,11 +121,10 @@ 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.setExternalReferences(mappedList(result.getExternalReference(), ConversionUtils::oafExtRefToBrokerExtRef)); + res + .setExternalReferences(mappedList(result.getExternalReference(), ConversionUtils::oafExtRefToBrokerExtRef)); return res; } @@ -141,6 +142,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; @@ -148,6 +150,11 @@ 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; @@ -216,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; } @@ -245,7 +264,13 @@ public class ConversionUtils { private static List fieldList(final List> fl) { return fl != null - ? fl.stream().map(Field::getValue).filter(StringUtils::isNotBlank).collect(Collectors.toList()) + ? fl + .stream() + .map(Field::getValue) + .map(s -> StringUtils.abbreviate(s, BrokerConstants.MAX_STRING_SIZE)) + .filter(StringUtils::isNotBlank) + .limit(BrokerConstants.MAX_LIST_SIZE) + .collect(Collectors.toList()) : new ArrayList<>(); } @@ -255,6 +280,7 @@ public class ConversionUtils { .stream() .map(StructuredProperty::getValue) .filter(StringUtils::isNotBlank) + .limit(BrokerConstants.MAX_LIST_SIZE) .collect(Collectors.toList()) : new ArrayList<>(); } @@ -280,6 +306,7 @@ public class ConversionUtils { .stream() .map(func::apply) .filter(Objects::nonNull) + .limit(BrokerConstants.MAX_LIST_SIZE) .collect(Collectors.toList()); } @@ -293,6 +320,7 @@ public class ConversionUtils { .map(func::apply) .flatMap(List::stream) .filter(Objects::nonNull) + .limit(BrokerConstants.MAX_LIST_SIZE) .collect(Collectors.toList()); } 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 e142b5904..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 @@ -3,11 +3,31 @@ 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.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; +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; @@ -18,11 +38,12 @@ 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 { - private static List> matchers = new ArrayList<>(); + private static final Logger log = LoggerFactory.getLogger(EventFinder.class); + + private static final List> matchers = new ArrayList<>(); static { matchers.add(new EnrichMissingAbstract()); matchers.add(new EnrichMissingAuthorOrcid()); @@ -34,40 +55,66 @@ 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()); - // 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()); } - public static EventGroup generateEvents(final ResultGroup results, final DedupConfig dedupConfig) { + public static EventGroup generateEvents(final ResultGroup results, + final Set dsIdWhitelist, + final Set dsIdBlacklist, + final Set dsTypeWhitelist, + 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)); + 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)); + } + } } } return asEventGroup(list); } + private static boolean verifyTarget(final OaBrokerRelatedDatasource target, + final Set dsIdWhitelist, + final Set dsIdBlacklist, + final Set dsTypeWhitelist) { + + if (dsIdWhitelist.contains(target.getOpenaireId())) { + return true; + } else if (dsIdBlacklist.contains(target.getOpenaireId())) { + return false; + } else { + return dsTypeWhitelist.contains(target.getType()); + } + } + private static EventGroup asEventGroup(final List> list) { final EventGroup events = new EventGroup(); list.stream().map(EventFactory::newBrokerEvent).forEach(events::addElement); return events; } + public static List> getMatchers() { + return matchers; + } + } 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 048683b50..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 @@ -4,20 +4,12 @@ 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.broker.objects.OaBrokerRelatedDatasource; 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 { @@ -29,26 +21,27 @@ public final class UpdateInfo { private final OaBrokerMainEntity target; + private final OaBrokerRelatedDatasource targetDs; + private final BiConsumer compileHighlight; private final Function highlightToString; 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 OaBrokerRelatedDatasource targetDs, 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.targetDs = targetDs; this.compileHighlight = compileHighlight; this.highlightToString = highlightToString; - this.trust = calculateTrust(dedupConfig, source, target); + this.trust = TrustUtils.calculateTrust(source, target); } public T getHighlightValue() { @@ -63,29 +56,8 @@ 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; - } + public OaBrokerRelatedDatasource getTargetDs() { + return targetDs; } protected Topic getTopic() { @@ -112,7 +84,20 @@ public final class UpdateInfo { compileHighlight.accept(hl, getHighlightValue()); final String provId = getSource().getOpenaireId(); - final String provRepo = getSource().getCollectedFromName(); + 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() @@ -122,7 +107,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/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/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/RelatedDatasourceAggregator.java b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/RelatedDatasourceAggregator.java new file mode 100644 index 000000000..2c0c7917d --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/main/java/eu/dnetlib/dhp/broker/oa/util/aggregators/withRels/RelatedDatasourceAggregator.java @@ -0,0 +1,69 @@ + +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 eu.dnetlib.dhp.broker.oa.util.BrokerConstants; +import scala.Tuple2; + +public class RelatedDatasourceAggregator + extends Aggregator, OaBrokerMainEntity, OaBrokerMainEntity> { + + /** + * + */ + private static final long serialVersionUID = -7212121913834713672L; + + @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 && res.getDatasources().size() < BrokerConstants.MAX_NUMBER_OF_RELS) { + res.getDatasources().add(t._2.getRelDatasource()); + } + return res; + + } + + @Override + public OaBrokerMainEntity merge(final OaBrokerMainEntity g1, final OaBrokerMainEntity g2) { + 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; + } + } + + @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/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 8752200ff..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 @@ -9,15 +9,29 @@ workingPath the path where the the generated data will be stored - - isLookupUrl - the address of the lookUp service + + 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 - dedupConfProfId - the id of a valid Dedup Configuration Profile + esIndexName + the elasticsearch index name + + + esIndexHost + the elasticsearch host - sparkDriverMemory memory for driver process @@ -89,6 +103,7 @@ + @@ -118,6 +133,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} + + + + @@ -216,7 +255,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} + + + + @@ -356,8 +419,58 @@ --conf spark.sql.shuffle.partitions=3840 --workingPath${workingPath} - --isLookupUrl${isLookupUrl} - --dedupConfProfile${dedupConfProfId} + --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 + 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/generate_events.json b/dhp-workflows/dhp-broker-events/src/main/resources/eu/dnetlib/dhp/broker/oa/generate_events.json index 7ae076159..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 @@ -6,15 +6,21 @@ "paramRequired": true }, { - "paramName": "lu", - "paramLongName": "isLookupUrl", - "paramDescription": "the address of the ISLookUpService", + "paramName": "datasourceIdWhitelist", + "paramLongName": "datasourceIdWhitelist", + "paramDescription": "a white list (comma separeted, - for empty list) of datasource ids", "paramRequired": true }, { - "paramName": "d", - "paramLongName": "dedupConfProfile", - "paramDescription": "the id of a valid Dedup Configuration Profile", + "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/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..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 @@ -1,4 +1,4 @@ - + @@ -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 @@ -78,9 +69,8 @@ Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - + + yarn cluster @@ -98,8 +88,58 @@ --conf spark.sql.shuffle.partitions=3840 --workingPath${workingPath} - --isLookupUrl${isLookupUrl} - --dedupConfProfile${dedupConfProfId} + --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 + 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/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..8fa95abe5 --- /dev/null +++ b/dhp-workflows/dhp-broker-events/src/test/java/eu/dnetlib/dhp/broker/oa/matchers/UpdateMatcherTest.java @@ -0,0 +1,133 @@ + +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 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 { + } + + @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, targetDs, Arrays.asList(p1, p2, p3, p4), 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, targetDs, Arrays.asList(p1, p2, p3, p4), 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, targetDs, Arrays.asList(p1, p2, p3, p4), 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, targetDs, Arrays.asList(p1, p2, p3, p4), 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, targetDs, Arrays.asList(p1, p2, p3, p4), 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, targetDs, 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/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()); + } + +} 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); 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/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/Deduper.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/Deduper.java index c72940deb..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 @@ -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 @@ -52,6 +52,7 @@ public class Deduper implements Serializable { .collect(Collectors.toList()) .iterator()) .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); } } 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..6fb7b844b --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/RelationAggregator.java @@ -0,0 +1,57 @@ + +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 mergeRel(b, a); + } + + @Override + public Relation merge(Relation b, Relation a) { + return mergeRel(b, a); + } + + @Override + public Relation finish(Relation r) { + 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); + } + + @Override + public Encoder outputEncoder() { + return Encoders.kryo(Relation.class); + } +} 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..1e13485e5 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkBlockStats.java @@ -0,0 +1,126 @@ + +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; +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; +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; +import eu.dnetlib.pace.config.DedupConfig; +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); + + 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); + + SparkConf conf = new SparkConf(); + + new SparkBlockStats(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } + + public Long computeComparisons(Long blockSize, Long slidingWindowSize) { + + if (slidingWindowSize >= blockSize) + return (slidingWindowSize * (slidingWindowSize - 1)) / 2; + else { + return (blockSize - slidingWindowSize + 1) * (slidingWindowSize * (slidingWindowSize - 1)) / 2; + } + } + + @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"); + final int numPartitions = Optional + .ofNullable(parser.get("numPartitions")) + .map(Integer::valueOf) + .orElse(NUM_PARTITIONS); + + 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)) + .repartition(numPartitions) + .mapToPair( + (PairFunction) s -> { + MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s); + return new Tuple2<>(d.getIdentifier(), d); + }); + + // create blocks for deduplication + JavaRDD blockStats = Deduper + .createSortedBlocks(mapDocuments, dedupConf) + .repartition(numPartitions) + .map(b -> asBlockStats(dedupConf, b)); + + // save the blockstats in the workingdir + spark + .createDataset(blockStats.rdd(), Encoders.bean(BlockStats.class)) + .write() + .mode(SaveMode.Overwrite) + .save(outputPath); + } + } + + 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())); + } + +} 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..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 @@ -5,11 +5,13 @@ 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; 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; @@ -75,7 +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 = 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); @@ -100,8 +106,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(); @@ -109,7 +117,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)) @@ -117,6 +125,7 @@ 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/SparkCreateSimRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateSimRels.java index 2cfe2e080..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 @@ -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; @@ -48,13 +49,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"))); } @@ -68,7 +62,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); @@ -88,6 +87,7 @@ public class SparkCreateSimRels extends AbstractSparkAction { JavaPairRDD mapDocuments = sc .textFile(DedupUtility.createEntityPath(graphBasePath, subEntity)) + .repartition(numPartitions) .mapToPair( (PairFunction) s -> { MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s); @@ -95,19 +95,17 @@ public class SparkCreateSimRels extends AbstractSparkAction { }); // create blocks for deduplication - JavaPairRDD blocks = Deduper.createSortedBlocks(mapDocuments, dedupConf); + JavaPairRDD blocks = Deduper + .createSortedBlocks(mapDocuments, dedupConf) + .repartition(numPartitions); // 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)); - - // save the simrel in the workingdir - spark - .createDataset(relations.rdd(), Encoders.bean(Relation.class)) - .write() - .mode(SaveMode.Append) - .save(outputPath); + .map(t -> createSimRel(t._1(), t._2(), entity)) + .repartition(numPartitions) + .map(r -> OBJECT_MAPPER.writeValueAsString(r)) + .saveAsTextFile(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 516808511..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 @@ -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,7 +97,24 @@ public class SparkPropagateRelation extends AbstractSparkAction { FieldType.TARGET, getDeletedFn()); - save(newRels.union(updated).union(mergeRels), outputRelationPath, SaveMode.Overwrite); + save( + 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()) + .agg(new RelationAggregator().toColumn()) + .map((MapFunction, Relation>) t -> t._2(), Encoders.bean(Relation.class)); } private static Dataset processDataset( @@ -112,6 +131,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 bfd2c25e2..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 @@ -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; @@ -18,12 +19,17 @@ public class ConnectedComponent implements Serializable { private Set docIds; private String ccId; - public ConnectedComponent() { - } - - public ConnectedComponent(Set docIds) { + public ConnectedComponent(Set docIds, final int cut) { this.docIds = docIds; createID(); + 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); + } } 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/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..0ec8c2699 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/BlockStats.java @@ -0,0 +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 + + 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..09f4365d3 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createBlockStats_parameters.json @@ -0,0 +1,32 @@ +[ + { + "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 + }, + { + "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/createCC_parameters.json b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json index 6eedd5432..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 @@ -17,6 +17,12 @@ "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", 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-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/config-default.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/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-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/oozie_app/config-default.xml 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..c0080b028 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/statistics/oozie_app/workflow.xml @@ -0,0 +1,108 @@ + + + + graphBasePath + the raw graph base path + + + isLookUpUrl + the address of the lookUp service + + + actionSetId + id of the actionSet + + + numPartitions + number of partitions for the similarity relations intermediate phases + + + 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 deduplication blocks + 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 + + --graphBasePath${graphBasePath} + --isLookUpUrl${isLookUpUrl} + --actionSetId${actionSetId} + --workingPath${workingDir} + --numPartitions${numPartitions} + + + + + + + \ No newline at end of file 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", 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..3fb9d1751 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,17 @@ 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); + + assertEquals(merged.getBestaccessright().getClassid(), "OPEN SOURCE"); + } + @Test public void publicationMergerTest() throws InstantiationException, IllegalAccessException { 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..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 @@ -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; @@ -71,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()); @@ -152,33 +160,38 @@ 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); 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); @@ -190,6 +203,101 @@ 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); + + 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 + @Order(3) public void createMergeRelsTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -225,8 +333,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() @@ -241,7 +351,7 @@ public class SparkDedupTest implements Serializable { } @Test - @Order(3) + @Order(4) public void createDedupRecordTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -288,7 +398,7 @@ public class SparkDedupTest implements Serializable { } @Test - @Order(4) + @Order(5) public void updateEntityTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -404,7 +514,7 @@ public class SparkDedupTest implements Serializable { } @Test - @Order(5) + @Order(6) public void propagateRelationTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -423,7 +533,7 @@ public class SparkDedupTest implements Serializable { long relations = jsc.textFile(testDedupGraphBasePath + "/relation").count(); - assertEquals(4975, relations); + assertEquals(4866, relations); // check deletedbyinference final Dataset mergeRels = spark @@ -454,7 +564,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); 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..7e76c284b --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkStatsTest.java @@ -0,0 +1,177 @@ + +package eu.dnetlib.dhp.oa.dedup; + +import static java.nio.file.Files.createTempDirectory; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.lenient; + +import 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; +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 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; + + 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); + } +} 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..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" : "2000", + "queueMaxSize" : "100", "groupMaxSize" : "100", "maxChildren" : "100", - "slidingWindowSize" : "200", + "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", 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..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" : "2000", + "queueMaxSize" : "100", "groupMaxSize" : "100", "maxChildren" : "100", - "slidingWindowSize" : "200", + "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", 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..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": "2000", + "queueMaxSize": "100", "groupMaxSize": "100", "maxChildren": "100", - "slidingWindowSize": "200", + "slidingWindowSize": "100", "rootBuilder": [ "result", "resultProject_outcome_isProducedBy", 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..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" : "2000", + "queueMaxSize" : "100", "groupMaxSize" : "100", "maxChildren" : "100", - "slidingWindowSize" : "200", + "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", @@ -19,7 +19,7 @@ "clustering" : [ { "name" : "ngrampairs", "fields" : [ "title" ], "params" : { "max" : "1", "ngramLen" : "3"} }, { "name" : "suffixprefix", "fields" : [ "title" ], "params" : { "max" : "1", "len" : "3" } }, - { "name" : "lowercase", "fields" : [ "doi", "url" ], "params" : { } } + { "name" : "lowercase", "fields" : [ "doi" ], "params" : { } } ], "decisionTree": { "start": { 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 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/oa/graph/clean/CleanGraphSparkJob.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/CleanGraphSparkJob.java index e1c4b53b5..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 @@ -23,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; @@ -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,16 @@ 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/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/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..b723de955 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJob.java @@ -0,0 +1,162 @@ + +package eu.dnetlib.dhp.oa.graph.merge; + +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; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.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; + +/** + * 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(); + + private static final String PRIORITY_DEFAULT = "BETA"; // BETA | PROD + + 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 priority = Optional + .ofNullable(parser.get("priority")) + .orElse(PRIORITY_DEFAULT); + log.info("priority: {}", priority); + + 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, priority, betaInputPath, prodInputPath, entityClazz, entityClazz, outputPath); + }); + } + + private static

void mergeGraphTable( + SparkSession spark, + String priority, + 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); + switch (priority) { + default: + case "BETA": + return mergeWithPriorityToBETA(p, b); + case "PROD": + return mergeWithPriorityToPROD(p, b); + } + }, Encoders.bean(p_clazz)) + .filter((FilterFunction

) Objects::nonNull) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .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) { + + 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/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..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 @@ -1,36 +1,10 @@ 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.dom4j.Document; @@ -40,24 +14,8 @@ import org.dom4j.Node; import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyGroup; import eu.dnetlib.dhp.schema.common.LicenseComparator; -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.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.*; public abstract class AbstractMdRecordToOafMapper { @@ -99,7 +57,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 +75,39 @@ 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() + .map(s -> UNKNOWN.equalsIgnoreCase(s) ? "0000" : s) + .orElse("0000"); // Unknown + 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; + } + 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 +122,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 +132,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 +152,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 +164,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,14 +243,16 @@ 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); 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")); @@ -291,7 +277,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)); } @@ -378,6 +364,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 @@ -425,6 +415,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/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/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/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 new file mode 100644 index 000000000..60857f0fc --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/SparkCreateEBIDataFrame.scala @@ -0,0 +1,87 @@ +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") + + + + 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 0db2b2688..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,6 +150,17 @@ 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..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 @@ -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,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(); + 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 +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")); return st; }) .collect(Collectors.toList())); @@ -282,6 +317,13 @@ 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..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,6 +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")); return st; }) .collect(Collectors.toList())); 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", 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..07a125fb6 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge/oozie_app/workflow.xml @@ -0,0 +1,293 @@ + + + + + betaInputGgraphPath + the beta graph root path + + + prodInputGgraphPath + the production graph root path + + + graphOutputPath + the output merged graph root path + + + priority + decides from which infrastructure the content must win in case of ID clash + + + + 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 + --priority${priority} + + + + + + + + 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 + --priority${priority} + + + + + + + + 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 + --priority${priority} + + + + + + + + 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 + --priority${priority} + + + + + + + + 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 + --priority${priority} + + + + + + + + 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 + --priority${priority} + + + + + + + + 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 + --priority${priority} + + + + + + + + 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 + --priority${priority} + + + + + + + + + \ 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..1a612807b --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/merge_graphs_parameters.json @@ -0,0 +1,38 @@ +[ + { + "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 + }, + { + "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 diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_claims/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/raw_claims/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/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..66eaeeb26 --- /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,157 @@ + + + + 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 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 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..cac3cc2bb --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/oozie_app/config-default.xml @@ -0,0 +1,68 @@ + + + + + + + + + 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 + + + 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..a5035c56c --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/sx/ebi/oozie_app/workflow.xml @@ -0,0 +1,97 @@ + + + + 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 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-graph-mapper-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.sql.shuffle.partitions=1000 + ${sparkExtraOPT} + + --workingPath${workingPath} + --masteryarn + + + + + + + \ No newline at end of file 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/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..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 @@ -6,6 +6,7 @@ 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; @@ -20,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; @@ -31,24 +33,25 @@ 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 { + @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 @@ -68,9 +71,14 @@ 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()); + assertTrue(p.getSource().size() == 1); assertTrue(p.getAuthor().size() > 0); final Optional author = p @@ -79,6 +87,7 @@ public class MappersTest { .filter(a -> a.getPid() != null && !a.getPid().isEmpty()) .findFirst(); assertTrue(author.isPresent()); + final StructuredProperty pid = author .get() .getPid() @@ -169,6 +178,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); @@ -261,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/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..92e14895d --- /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/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", 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..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 @@ -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 @@ -47,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 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 637362acf..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, 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._ @@ -99,6 +100,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() @@ -412,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-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]) } 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..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 @@ -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; @@ -118,10 +119,7 @@ public class CreateRelatedEntitiesJob_phase1 { Dataset> entities = readPathEntity(spark, inputEntityPath, clazz) .filter("dataInfo.invisible == false") .map( - (MapFunction) value -> asRelatedEntity(value, clazz), - 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(); @@ -165,13 +163,24 @@ 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()); + 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")); 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..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 @@ -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; @@ -64,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 @@ -249,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); } } @@ -269,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()); @@ -281,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); } @@ -305,20 +297,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..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 @@ -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; @@ -95,13 +100,21 @@ public class PrepareRelationsJob { .orElse(new HashSet<>()); log.info("relationFilter: {}", relationFilter); - int maxRelations = Optional - .ofNullable(parser.get("maxRelations")) + int sourceMaxRelations = Optional + .ofNullable(parser.get("sourceMaxRelations")) .map(Integer::valueOf) .orElse(MAX_RELS); - log.info("maxRelations: {}", maxRelations); + log.info("sourceMaxRelations: {}", sourceMaxRelations); + + int targetMaxRelations = Optional + .ofNullable(parser.get("targetMaxRelations")) + .map(Integer::valueOf) + .orElse(MAX_RELS); + log.info("targetMaxRelations: {}", targetMaxRelations); SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ProvisionModelSupport.getModelClasses()); runWithSparkSession( conf, @@ -109,7 +122,8 @@ public class PrepareRelationsJob { spark -> { removeOutputDir(spark, outputPath); prepareRelationsRDD( - spark, inputRelationsPath, outputPath, relationFilter, maxRelations, relPartitions); + spark, inputRelationsPath, outputPath, relationFilter, sourceMaxRelations, targetMaxRelations, + relPartitions); }); } @@ -122,47 +136,124 @@ public class PrepareRelationsJob { * @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 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 maxRelations, - int relPartitions) { + private static void prepareRelationsRDD(SparkSession spark, String inputRelationsPath, String outputPath, + Set relationFilter, int sourceMaxRelations, int targetMaxRelations, int relPartitions) { - // group by SOURCE and apply limit - RDD bySource = readPathRelationRDD(spark, inputRelationsPath) + JavaRDD rels = 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(); - - // 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(); + .filter(rel -> relationFilter.contains(rel.getRelClass()) == false); + JavaRDD pruned = pruneRels( + pruneRels( + rels, + sourceMaxRelations, relPartitions, (Function) r -> r.getSource()), + targetMaxRelations, relPartitions, (Function) r -> r.getTarget()); spark - .createDataset(bySource.union(byTarget), Encoders.bean(Relation.class)) + .createDataset(pruned.rdd(), Encoders.bean(Relation.class)) .repartition(relPartitions) .write() .mode(SaveMode.Overwrite) .parquet(outputPath); } + 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); + } + + // 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 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/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; + +} 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/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/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..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 @@ -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)) { @@ -333,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 @@ -374,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 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 } + ] 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..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 @@ -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,6 +137,9 @@ --inputRelationsPath${inputGraphRootPath}/relation --outputPath${workingDir}/relation + --sourceMaxRelations${sourceMaxRelations} + --targetMaxRelations${targetMaxRelations} + --relationFilter${relationFilter} --relPartitions5000 @@ -164,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 @@ -191,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 @@ -218,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 @@ -245,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 @@ -272,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 @@ -299,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 @@ -326,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 @@ -365,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 @@ -393,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 @@ -421,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 @@ -449,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 @@ -477,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 @@ -505,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 @@ -533,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 @@ -571,10 +578,18 @@ --isLookupUrl${isLookupUrl} --otherDsTypeId${otherDsTypeId} - +
+ + + ${wf:conf('shouldIndex') eq 'true'} + ${wf:conf('shouldIndex') eq 'false'} + + + + yarn @@ -605,5 +620,4 @@ - \ No newline at end of file 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..528532edd --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/test/java/eu/dnetlib/dhp/oa/provision/PrepareRelationsJobTest.java @@ -0,0 +1,100 @@ + +package eu.dnetlib.dhp.oa.provision; + +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; +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 com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport; +import eu.dnetlib.dhp.schema.oaf.Relation; + +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 000000000..13bc01c8c Binary files /dev/null and b/dhp-workflows/dhp-graph-provision/src/test/resources/eu/dnetlib/dhp/oa/provision/relations.gz differ diff --git a/dhp-workflows/dhp-graph-provision/src/test/resources/log4j.properties b/dhp-workflows/dhp-graph-provision/src/test/resources/log4j.properties new file mode 100644 index 000000000..20f56e38d --- /dev/null +++ b/dhp-workflows/dhp-graph-provision/src/test/resources/log4j.properties @@ -0,0 +1,11 @@ +# Set root logger level to DEBUG and its only appender to A1. +log4j.rootLogger=INFO, A1 + +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender + +# A1 uses PatternLayout. +log4j.logger.org = ERROR +log4j.logger.eu.dnetlib = DEBUG +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n \ 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/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..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 @@ -16,15 +16,15 @@ spark2
- hiveMetastoreUris + 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 - + \ 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 new file mode 100644 index 000000000..70112dc7b --- /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,18 @@ +export PYTHON_EGG_CACHE=/home/$(whoami)/.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 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" +rm $2 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..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 new file mode 100644 index 000000000..46ff295f4 --- /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,21 @@ +------------------------------------------------------------------------------------------------ +------------------------------------------------------------------------------------------------ +-- 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 ${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 ${stats_db_name}.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/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..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 new file mode 100644 index 000000000..25439852e --- /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,38 @@ +------------------------------------------------------------------------------------------------------ +-- Creating parquet tables from the updated temporary tables and removing unnecessary temporary tables +------------------------------------------------------------------------------------------------------ + +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; 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..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,26 +1,59 @@ ------------------------------------------------------------------------------------------------------- --- Creating parquet tables from the updated temporary tables and removing unnecessary temporary tables ------------------------------------------------------------------------------------------------------- - -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; +------------------------------------------------------ +------------------------------------------------------ +-- 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; +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 24e9e3107..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,7 +1,49 @@ ----------------------------------------------- --- Re-creating views from final parquet tables ---------------------------------------------- +------------------------------------------------------ +------------------------------------------------------ +-- 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; --- 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; +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..60b37048b --- /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,36 @@ +------------------------------------------------------ +------------------------------------------------------ +-- 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.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.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.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.classname 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..33849b960 --- /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,80 @@ +---------------------------------------------------- +-- Shortcuts for various definitions in stats db --- +---------------------------------------------------- + +-- 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 +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. +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 + 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' + ) 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 +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. +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!=''; \ 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..f737c1ea6 --- /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 ${stats_db_name}.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 new file mode 100644 index 000000000..e002f656e --- /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,163 @@ +------------------------------------------------------ +------------------------------------------------------ +-- 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 the shadow database, in case it doesn't exist +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; +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/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..312a8b82e --- /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,35 @@ +-------------------------------------------------------------- +-------------------------------------------------------------- +-- 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'); + +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; + +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 ${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 ${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 ${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; + +-- 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/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..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") !=""; +------------------------------------------------------ +------------------------------------------------------ +-- 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 new file mode 100644 index 000000000..ca1059cc8 --- /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,36 @@ +-------------------------------------------------------- +-------------------------------------------------------- +-- 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'); + +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; + +-- 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}.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}.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}.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/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..b4fb5aec6 --- /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,37 @@ +-------------------------------------------------------------------------------- +-------------------------------------------------------------------------------- +-- 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'); + +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 ${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}.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; 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..21a944164 --- /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,30 @@ +-- noinspection SqlNoDataSourceInspectionForFile + +------------------------------------------------------ +------------------------------------------------------ +-- 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; + +-- 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'; + +-- 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' and r.datainfo.deletedbyinference=false; + +-- 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 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, 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 +from ${openaire_db_name}.project p lateral view explode(p.fundingtree.value) fundingtree as fund 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..7acabf1dd --- /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,31 @@ +---------------------------------------------------- +---------------------------------------------------- +-- 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; + +-- 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; + +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; + +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/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..4e13b3dd8 --- /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,58 @@ +-- noinspection SqlNoDataSourceInspectionForFile + +------------------------------------------------------------ +------------------------------------------------------------ +-- 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'); + +-- 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 +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'; + +-- 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; + + 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..a918e4de4 --- /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,12 @@ +---------------------------------------------------------------- +---------------------------------------------------------------- +-- 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.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; + +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/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_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/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..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 @@ -1,4 +1,4 @@ - + stats_db_name @@ -14,11 +14,15 @@ the external stats that should be added since they are not included in the graph database - hiveMetastoreUris + stats_db_shadow_name + the name of the shadow schema + + + hive_metastore_uris hive server metastore URIs - hiveJdbcUrl + hive_jdbc_url hive server jdbc url @@ -33,7 +37,7 @@ hive.metastore.uris - ${hiveMetastoreUris} + ${hive_metastore_uris} hive.txn.timeout @@ -50,856 +54,131 @@ - ${hiveJdbcUrl} + ${hive_jdbc_url} 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} - + ${hive_jdbc_url} + stats_db_name=${stats_db_name} openaire_db_name=${openaire_db_name} - + - ${hiveJdbcUrl} + ${hive_jdbc_url} stats_db_name=${stats_db_name} openaire_db_name=${openaire_db_name} - + - - - ${hiveJdbcUrl} - + + + ${hive_jdbc_url} + 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} - + ${hive_jdbc_url} + 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} - + ${hive_jdbc_url} + 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} - + ${hive_jdbc_url} + 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} - + ${hive_jdbc_url} + 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} - + ${hive_jdbc_url} + 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} - + ${hive_jdbc_url} + stats_db_name=${stats_db_name} openaire_db_name=${openaire_db_name} + external_stats_db_name=${external_stats_db_name} - - + + - ${hiveJdbcUrl} + ${hive_jdbc_url} stats_db_name=${stats_db_name} openaire_db_name=${openaire_db_name} external_stats_db_name=${external_stats_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} - + + + ${hive_jdbc_url} + stats_db_name=${stats_db_name} openaire_db_name=${openaire_db_name} @@ -909,7 +188,7 @@ - ${hiveJdbcUrl} + ${hive_jdbc_url} stats_db_name=${stats_db_name} openaire_db_name=${openaire_db_name} @@ -920,14 +199,72 @@ - ${hiveJdbcUrl} + ${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} + 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} + impala-shell.sh + ${stats_db_name} + step18.sql + /user/${wf:user()}/oa/graph/stats/oozie_app/scripts/step18.sql + impala-shell.sh + + + + + 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_prod_construction.xml similarity index 96% 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_prod_construction.xml index 819b3e12d..047433320 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_prod_construction.xml @@ -7,10 +7,33 @@ - Graph Construction [OCEAN] + Graph Construction [PROD] 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 @@ -371,45 +413,10 @@ build-report - + - - 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' } diff --git a/pom.xml b/pom.xml index 89b7e8829..cec3dd75a 100644 --- a/pom.xml +++ b/pom.xml @@ -315,7 +315,7 @@ eu.dnetlib dnet-pace-core - 4.0.1 + 4.0.4 eu.dnetlib @@ -324,6 +324,12 @@ + eu.dnetlib.dhp + dnet-openaire-broker-common + ${dnet.openaire.broker.common} + + + org.apache.cxf cxf-rt-transports-http 3.1.5 @@ -618,5 +624,6 @@ 3.3.3 3.4.2 [2.12,3.0) + 3.1.0