diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/common/Constants.java b/dhp-common/src/main/java/eu/dnetlib/dhp/common/Constants.java index 4f2c6341e..0477d6399 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/common/Constants.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/common/Constants.java @@ -51,6 +51,7 @@ public class Constants { public static final String RETRY_DELAY = "retryDelay"; public static final String CONNECT_TIMEOUT = "connectTimeOut"; public static final String READ_TIMEOUT = "readTimeOut"; + public static final String REQUEST_METHOD = "requestMethod"; public static final String FROM_DATE_OVERRIDE = "fromDateOverride"; public static final String UNTIL_DATE_OVERRIDE = "untilDateOverride"; diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/common/collection/HttpClientParams.java b/dhp-common/src/main/java/eu/dnetlib/dhp/common/collection/HttpClientParams.java index 6fcec00dd..d26d9c0e9 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/common/collection/HttpClientParams.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/common/collection/HttpClientParams.java @@ -1,6 +1,9 @@ package eu.dnetlib.dhp.common.collection; +import java.util.HashMap; +import java.util.Map; + /** * Bundles the http connection parameters driving the client behaviour. */ @@ -13,6 +16,8 @@ public class HttpClientParams { public static int _connectTimeOut = 10; // seconds public static int _readTimeOut = 30; // seconds + public static String _requestMethod = "GET"; + /** * Maximum number of allowed retires before failing */ @@ -38,17 +43,30 @@ public class HttpClientParams { */ private int readTimeOut; + /** + * Custom http headers + */ + private Map headers; + + /** + * Request method (i.e., GET, POST etc) + */ + private String requestMethod; + public HttpClientParams() { - this(_maxNumberOfRetry, _requestDelay, _retryDelay, _connectTimeOut, _readTimeOut); + this(_maxNumberOfRetry, _requestDelay, _retryDelay, _connectTimeOut, _readTimeOut, new HashMap<>(), + _requestMethod); } public HttpClientParams(int maxNumberOfRetry, int requestDelay, int retryDelay, int connectTimeOut, - int readTimeOut) { + int readTimeOut, Map headers, String requestMethod) { this.maxNumberOfRetry = maxNumberOfRetry; this.requestDelay = requestDelay; this.retryDelay = retryDelay; this.connectTimeOut = connectTimeOut; this.readTimeOut = readTimeOut; + this.headers = headers; + this.requestMethod = requestMethod; } public int getMaxNumberOfRetry() { @@ -91,4 +109,19 @@ public class HttpClientParams { this.readTimeOut = readTimeOut; } + public Map getHeaders() { + return headers; + } + + public void setHeaders(Map headers) { + this.headers = headers; + } + + public String getRequestMethod() { + return requestMethod; + } + + public void setRequestMethod(String requestMethod) { + this.requestMethod = requestMethod; + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/common/collection/HttpConnector2.java b/dhp-common/src/main/java/eu/dnetlib/dhp/common/collection/HttpConnector2.java index dd46ab1f4..905457bcd 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/common/collection/HttpConnector2.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/common/collection/HttpConnector2.java @@ -107,7 +107,14 @@ public class HttpConnector2 { urlConn.setReadTimeout(getClientParams().getReadTimeOut() * 1000); urlConn.setConnectTimeout(getClientParams().getConnectTimeOut() * 1000); urlConn.addRequestProperty(HttpHeaders.USER_AGENT, userAgent); + urlConn.setRequestMethod(getClientParams().getRequestMethod()); + // if provided, add custom headers + if (!getClientParams().getHeaders().isEmpty()) { + for (Map.Entry headerEntry : getClientParams().getHeaders().entrySet()) { + urlConn.addRequestProperty(headerEntry.getKey(), headerEntry.getValue()); + } + } if (log.isDebugEnabled()) { logHeaderFields(urlConn); } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/DispatchEntitiesSparkJob.java b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/DispatchEntitiesSparkJob.java deleted file mode 100644 index cf0a183d7..000000000 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/DispatchEntitiesSparkJob.java +++ /dev/null @@ -1,97 +0,0 @@ - -package eu.dnetlib.dhp.oa.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.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; -import org.slf4j.LoggerFactory; - -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.schema.common.ModelSupport; - -public class DispatchEntitiesSparkJob { - - private static final Logger log = LoggerFactory.getLogger(DispatchEntitiesSparkJob.class); - - public static void main(String[] args) throws Exception { - - String jsonConfiguration = IOUtils - .toString( - Objects - .requireNonNull( - DispatchEntitiesSparkJob.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/merge/dispatch_entities_parameters.json"))); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); - - Boolean isSparkSessionManaged = Optional - .ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - - String inputPath = parser.get("inputPath"); - log.info("inputPath: {}", inputPath); - - String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); - - boolean filterInvisible = Boolean.parseBoolean(parser.get("filterInvisible")); - log.info("filterInvisible: {}", filterInvisible); - - SparkConf conf = new SparkConf(); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> dispatchEntities(spark, inputPath, outputPath, filterInvisible)); - } - - private static void dispatchEntities( - SparkSession spark, - String inputPath, - String outputPath, - boolean filterInvisible) { - - Dataset df = spark.read().textFile(inputPath); - - ModelSupport.oafTypes.entrySet().parallelStream().forEach(entry -> { - String entityType = entry.getKey(); - Class clazz = entry.getValue(); - - final String entityPath = outputPath + "/" + entityType; - if (!entityType.equalsIgnoreCase("relation")) { - HdfsSupport.remove(entityPath, spark.sparkContext().hadoopConfiguration()); - Dataset entityDF = spark - .read() - .schema(Encoders.bean(clazz).schema()) - .json( - df - .filter((FilterFunction) s -> s.startsWith(clazz.getName())) - .map( - (MapFunction) s -> StringUtils.substringAfter(s, "|"), - Encoders.STRING())); - - if (filterInvisible) { - entityDF = entityDF.filter("dataInfo.invisible != true"); - } - - entityDF - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .json(entityPath); - } - }); - } -} diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/GroupEntitiesSparkJob.java b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/GroupEntitiesSparkJob.java index e652bd5b6..f5c8eea19 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/GroupEntitiesSparkJob.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/GroupEntitiesSparkJob.java @@ -2,36 +2,28 @@ package eu.dnetlib.dhp.oa.merge; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import static eu.dnetlib.dhp.utils.DHPUtils.toSeq; +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.when; -import java.io.IOException; -import java.util.List; -import java.util.Objects; +import java.util.Map; import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ForkJoinPool; 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.JavaSparkContext; -import org.apache.spark.api.java.function.FilterFunction; import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.ReduceFunction; 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.DeserializationFeature; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.jayway.jsonpath.Configuration; -import com.jayway.jsonpath.DocumentContext; -import com.jayway.jsonpath.JsonPath; -import com.jayway.jsonpath.Option; - import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.schema.common.EntityType; import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.OafEntity; import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; import scala.Tuple2; @@ -39,13 +31,9 @@ import scala.Tuple2; * Groups the graph content by entity identifier to ensure ID uniqueness */ public class GroupEntitiesSparkJob { - private static final Logger log = LoggerFactory.getLogger(GroupEntitiesSparkJob.class); - private static final String ID_JPATH = "$.id"; - - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper() - .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + private static final Encoder OAFENTITY_KRYO_ENC = Encoders.kryo(OafEntity.class); public static void main(String[] args) throws Exception { @@ -66,9 +54,15 @@ public class GroupEntitiesSparkJob { String graphInputPath = parser.get("graphInputPath"); log.info("graphInputPath: {}", graphInputPath); + String checkpointPath = parser.get("checkpointPath"); + log.info("checkpointPath: {}", checkpointPath); + String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); + boolean filterInvisible = Boolean.valueOf(parser.get("filterInvisible")); + log.info("filterInvisible: {}", filterInvisible); + SparkConf conf = new SparkConf(); conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); conf.registerKryoClasses(ModelSupport.getOafModelClasses()); @@ -77,127 +71,96 @@ public class GroupEntitiesSparkJob { conf, isSparkSessionManaged, spark -> { - HdfsSupport.remove(outputPath, spark.sparkContext().hadoopConfiguration()); - groupEntities(spark, graphInputPath, outputPath); + HdfsSupport.remove(checkpointPath, spark.sparkContext().hadoopConfiguration()); + groupEntities(spark, graphInputPath, checkpointPath, outputPath, filterInvisible); }); } private static void groupEntities( SparkSession spark, String inputPath, - String outputPath) { + String checkpointPath, + String outputPath, + boolean filterInvisible) { - final TypedColumn aggregator = new GroupingAggregator().toColumn(); - final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - spark - .read() - .textFile(toSeq(listEntityPaths(inputPath, sc))) - .map((MapFunction) GroupEntitiesSparkJob::parseOaf, Encoders.kryo(OafEntity.class)) - .filter((FilterFunction) e -> StringUtils.isNotBlank(ModelSupport.idFn().apply(e))) - .groupByKey((MapFunction) oaf -> ModelSupport.idFn().apply(oaf), Encoders.STRING()) - .agg(aggregator) + Dataset allEntities = spark.emptyDataset(OAFENTITY_KRYO_ENC); + + for (Map.Entry e : ModelSupport.entityTypes.entrySet()) { + String entity = e.getKey().name(); + Class entityClass = e.getValue(); + String entityInputPath = inputPath + "/" + entity; + + if (!HdfsSupport.exists(entityInputPath, spark.sparkContext().hadoopConfiguration())) { + continue; + } + + allEntities = allEntities + .union( + ((Dataset) spark + .read() + .schema(Encoders.bean(entityClass).schema()) + .json(entityInputPath) + .filter("length(id) > 0") + .as(Encoders.bean(entityClass))) + .map((MapFunction) r -> r, OAFENTITY_KRYO_ENC)); + } + + Dataset groupedEntities = allEntities + .groupByKey((MapFunction) OafEntity::getId, Encoders.STRING()) + .reduceGroups((ReduceFunction) (b, a) -> OafMapperUtils.mergeEntities(b, a)) .map( - (MapFunction, String>) t -> t._2().getClass().getName() + - "|" + OBJECT_MAPPER.writeValueAsString(t._2()), - Encoders.STRING()) + (MapFunction, Tuple2>) t -> new Tuple2( + t._2().getClass().getName(), t._2()), + Encoders.tuple(Encoders.STRING(), OAFENTITY_KRYO_ENC)); + + // pivot on "_1" (classname of the entity) + // created columns containing only entities of the same class + for (Map.Entry e : ModelSupport.entityTypes.entrySet()) { + String entity = e.getKey().name(); + Class entityClass = e.getValue(); + + groupedEntities = groupedEntities + .withColumn( + entity, + when(col("_1").equalTo(entityClass.getName()), col("_2"))); + } + + groupedEntities + .drop("_1", "_2") .write() - .option("compression", "gzip") .mode(SaveMode.Overwrite) - .text(outputPath); - } + .option("compression", "gzip") + .save(checkpointPath); - public static class GroupingAggregator extends Aggregator { + ForkJoinPool parPool = new ForkJoinPool(ModelSupport.entityTypes.size()); - @Override - public OafEntity zero() { - return null; - } - - @Override - public OafEntity reduce(OafEntity b, OafEntity a) { - return mergeAndGet(b, a); - } - - private OafEntity mergeAndGet(OafEntity b, OafEntity a) { - if (Objects.nonNull(a) && Objects.nonNull(b)) { - return OafMapperUtils.mergeEntities(b, a); - } - return Objects.isNull(a) ? b : a; - } - - @Override - public OafEntity merge(OafEntity b, OafEntity a) { - return mergeAndGet(b, a); - } - - @Override - public OafEntity finish(OafEntity j) { - return j; - } - - @Override - public Encoder bufferEncoder() { - return Encoders.kryo(OafEntity.class); - } - - @Override - public Encoder outputEncoder() { - return Encoders.kryo(OafEntity.class); - } - - } - - private static OafEntity parseOaf(String s) { - - DocumentContext dc = JsonPath - .parse(s, Configuration.defaultConfiguration().addOptions(Option.SUPPRESS_EXCEPTIONS)); - final String id = dc.read(ID_JPATH); - if (StringUtils.isNotBlank(id)) { - - String prefix = StringUtils.substringBefore(id, "|"); - switch (prefix) { - case "10": - return parse(s, Datasource.class); - case "20": - return parse(s, Organization.class); - case "40": - return parse(s, Project.class); - case "50": - String resultType = dc.read("$.resulttype.classid"); - switch (resultType) { - case "publication": - return parse(s, Publication.class); - case "dataset": - return parse(s, eu.dnetlib.dhp.schema.oaf.Dataset.class); - case "software": - return parse(s, Software.class); - case "other": - return parse(s, OtherResearchProduct.class); - default: - throw new IllegalArgumentException(String.format("invalid resultType: '%s'", resultType)); - } - default: - throw new IllegalArgumentException(String.format("invalid id prefix: '%s'", prefix)); - } - } else { - throw new IllegalArgumentException(String.format("invalid oaf: '%s'", s)); - } - } - - private static OafEntity parse(String s, Class clazz) { - try { - return OBJECT_MAPPER.readValue(s, clazz); - } catch (IOException e) { - throw new IllegalArgumentException(e); - } - } - - private static List listEntityPaths(String inputPath, JavaSparkContext sc) { - return HdfsSupport - .listFiles(inputPath, sc.hadoopConfiguration()) + ModelSupport.entityTypes + .entrySet() .stream() - .filter(f -> !f.toLowerCase().contains("relation")) - .collect(Collectors.toList()); - } + .map(e -> parPool.submit(() -> { + String entity = e.getKey().name(); + Class entityClass = e.getValue(); + spark + .read() + .load(checkpointPath) + .select(col(entity).as("value")) + .filter("value IS NOT NULL") + .as(OAFENTITY_KRYO_ENC) + .map((MapFunction) r -> r, (Encoder) Encoders.bean(entityClass)) + .filter(filterInvisible ? "dataInfo.invisible != TRUE" : "TRUE") + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath + "/" + entity); + })) + .collect(Collectors.toList()) + .forEach(t -> { + try { + t.get(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + }); + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java index 8afa41f95..b4402a2fb 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java @@ -36,6 +36,19 @@ public class GraphCleaningFunctions extends CleaningFunctions { public static final int TITLE_FILTER_RESIDUAL_LENGTH = 5; private static final String NAME_CLEANING_REGEX = "[\\r\\n\\t\\s]+"; + private static final HashSet PEER_REVIEWED_TYPES = new HashSet<>(); + + static { + PEER_REVIEWED_TYPES.add("Article"); + PEER_REVIEWED_TYPES.add("Part of book or chapter of book"); + PEER_REVIEWED_TYPES.add("Book"); + PEER_REVIEWED_TYPES.add("Doctoral thesis"); + PEER_REVIEWED_TYPES.add("Master thesis"); + PEER_REVIEWED_TYPES.add("Data Paper"); + PEER_REVIEWED_TYPES.add("Thesis"); + PEER_REVIEWED_TYPES.add("Bachelor thesis"); + PEER_REVIEWED_TYPES.add("Conference object"); + } public static T cleanContext(T value, String contextId, String verifyParam) { if (ModelSupport.isSubClass(value, Result.class)) { @@ -273,6 +286,12 @@ public class GraphCleaningFunctions extends CleaningFunctions { public static T cleanup(T value, VocabularyGroup vocs) { + if (Objects.isNull(value.getDataInfo())) { + final DataInfo d = new DataInfo(); + d.setDeletedbyinference(false); + value.setDataInfo(d); + } + if (value instanceof OafEntity) { OafEntity e = (OafEntity) value; @@ -292,6 +311,10 @@ public class GraphCleaningFunctions extends CleaningFunctions { } else if (value instanceof Result) { Result r = (Result) value; + if (Objects.isNull(r.getContext())) { + r.setContext(new ArrayList<>()); + } + if (Objects.nonNull(r.getFulltext()) && (ModelConstants.SOFTWARE_RESULTTYPE_CLASSID.equals(r.getResulttype().getClassid()) || ModelConstants.DATASET_RESULTTYPE_CLASSID.equals(r.getResulttype().getClassid()))) { @@ -493,6 +516,35 @@ public class GraphCleaningFunctions extends CleaningFunctions { if (Objects.isNull(i.getRefereed()) || StringUtils.isBlank(i.getRefereed().getClassid())) { i.setRefereed(qualifier("0000", "Unknown", ModelConstants.DNET_REVIEW_LEVELS)); } + + // from the script from Dimitris + if ("0000".equals(i.getRefereed().getClassid())) { + final boolean isFromCrossref = Optional + .ofNullable(i.getCollectedfrom()) + .map(KeyValue::getKey) + .map(id -> id.equals(ModelConstants.CROSSREF_ID)) + .orElse(false); + final boolean hasDoi = Optional + .ofNullable(i.getPid()) + .map( + pid -> pid + .stream() + .anyMatch( + p -> PidType.doi.toString().equals(p.getQualifier().getClassid()))) + .orElse(false); + final boolean isPeerReviewedType = PEER_REVIEWED_TYPES + .contains(i.getInstancetype().getClassname()); + final boolean noOtherLitType = r + .getInstance() + .stream() + .noneMatch(ii -> "Other literature type".equals(ii.getInstancetype().getClassname())); + if (isFromCrossref && hasDoi && isPeerReviewedType && noOtherLitType) { + i.setRefereed(qualifier("0001", "peerReviewed", ModelConstants.DNET_REVIEW_LEVELS)); + } else { + i.setRefereed(qualifier("0002", "nonPeerReviewed", ModelConstants.DNET_REVIEW_LEVELS)); + } + } + if (Objects.nonNull(i.getDateofacceptance())) { Optional date = cleanDateField(i.getDateofacceptance()); if (date.isPresent()) { diff --git a/dhp-common/src/main/resources/eu/dnetlib/dhp/oa/merge/dispatch_entities_parameters.json b/dhp-common/src/main/resources/eu/dnetlib/dhp/oa/merge/dispatch_entities_parameters.json deleted file mode 100644 index 60f11ac84..000000000 --- a/dhp-common/src/main/resources/eu/dnetlib/dhp/oa/merge/dispatch_entities_parameters.json +++ /dev/null @@ -1,26 +0,0 @@ -[ - { - "paramName": "issm", - "paramLongName": "isSparkSessionManaged", - "paramDescription": "when true will stop SparkSession after job execution", - "paramRequired": false - }, - { - "paramName": "i", - "paramLongName": "inputPath", - "paramDescription": "the source path", - "paramRequired": true - }, - { - "paramName": "o", - "paramLongName": "outputPath", - "paramDescription": "path of the output graph", - "paramRequired": true - }, - { - "paramName": "fi", - "paramLongName": "filterInvisible", - "paramDescription": "if true filters out invisible entities", - "paramRequired": true - } -] \ No newline at end of file diff --git a/dhp-common/src/main/resources/eu/dnetlib/dhp/oa/merge/group_graph_entities_parameters.json b/dhp-common/src/main/resources/eu/dnetlib/dhp/oa/merge/group_graph_entities_parameters.json index e65acb3c4..58e3ca711 100644 --- a/dhp-common/src/main/resources/eu/dnetlib/dhp/oa/merge/group_graph_entities_parameters.json +++ b/dhp-common/src/main/resources/eu/dnetlib/dhp/oa/merge/group_graph_entities_parameters.json @@ -8,13 +8,25 @@ { "paramName": "gin", "paramLongName": "graphInputPath", - "paramDescription": "the graph root path", + "paramDescription": "the input graph root path", + "paramRequired": true + }, + { + "paramName": "cp", + "paramLongName": "checkpointPath", + "paramDescription": "checkpoint directory", "paramRequired": true }, { "paramName": "out", "paramLongName": "outputPath", - "paramDescription": "the output merged graph root path", + "paramDescription": "the output graph root path", + "paramRequired": true + }, + { + "paramName": "fi", + "paramLongName": "filterInvisible", + "paramDescription": "if true filters out invisible entities", "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/util/DiffPatchMatch.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/util/DiffPatchMatch.java index 12c96500e..84d49bd5c 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/util/DiffPatchMatch.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/util/DiffPatchMatch.java @@ -1,24 +1,6 @@ package eu.dnetlib.pace.util; -/* - * Diff Match and Patch - * Copyright 2018 The diff-match-patch Authors. - * https://github.com/google/diff-match-patch - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - /* * Diff Match and Patch * Copyright 2018 The diff-match-patch Authors. diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java index 62556b16b..006d3af76 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java @@ -40,6 +40,7 @@ public class Constants { public static final String SDG_CLASS_NAME = "Sustainable Development Goals"; public static final String NULL = "NULL"; + public static final String NA = "N/A"; public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); @@ -61,10 +62,16 @@ public class Constants { .map((MapFunction) value -> OBJECT_MAPPER.readValue(value, clazz), Encoders.bean(clazz)); } - public static Subject getSubject(String sbj, String classid, String classname, - String diqualifierclassid) { - if (sbj == null || sbj.equals(NULL)) + public static Subject getSubject(String sbj, String classid, String classname, String diqualifierclassid, + Boolean split) { + if (sbj == null || sbj.equals(NULL) || sbj.startsWith(NA)) return null; + String trust = ""; + String subject = sbj; + if (split) { + sbj = subject.split("@@")[0]; + trust = subject.split("@@")[1]; + } Subject s = new Subject(); s.setValue(sbj); s @@ -89,9 +96,14 @@ public class Constants { UPDATE_CLASS_NAME, ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), - "")); + trust)); return s; + } + + public static Subject getSubject(String sbj, String classid, String classname, + String diqualifierclassid) { + return getSubject(sbj, classid, classname, diqualifierclassid, false); } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/GetFOSSparkJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/GetFOSSparkJob.java index 0cc2f93df..abea6acd7 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/GetFOSSparkJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/GetFOSSparkJob.java @@ -75,9 +75,12 @@ public class GetFOSSparkJob implements Serializable { fosData.map((MapFunction) r -> { FOSDataModel fosDataModel = new FOSDataModel(); fosDataModel.setDoi(r.getString(0).toLowerCase()); - fosDataModel.setLevel1(r.getString(1)); - fosDataModel.setLevel2(r.getString(2)); - fosDataModel.setLevel3(r.getString(3)); + fosDataModel.setLevel1(r.getString(2)); + fosDataModel.setLevel2(r.getString(3)); + fosDataModel.setLevel3(r.getString(4)); + fosDataModel.setLevel4(r.getString(5)); + fosDataModel.setScoreL3(String.valueOf(r.getDouble(6))); + fosDataModel.setScoreL4(String.valueOf(r.getDouble(7))); return fosDataModel; }, Encoders.bean(FOSDataModel.class)) .write() diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareBipFinder.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareBipFinder.java deleted file mode 100644 index 0507f90e5..000000000 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareBipFinder.java +++ /dev/null @@ -1,178 +0,0 @@ - -package eu.dnetlib.dhp.actionmanager.createunresolvedentities; - -import static eu.dnetlib.dhp.actionmanager.Constants.*; -import static eu.dnetlib.dhp.actionmanager.Constants.UPDATE_CLASS_NAME; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; -import java.util.Optional; -import java.util.stream.Collectors; - -import org.apache.commons.io.IOUtils; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.MapFunction; -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.actionmanager.bipmodel.BipScore; -import eu.dnetlib.dhp.actionmanager.bipmodel.score.deserializers.BipResultModel; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.schema.common.ModelConstants; -import eu.dnetlib.dhp.schema.oaf.Instance; -import eu.dnetlib.dhp.schema.oaf.KeyValue; -import eu.dnetlib.dhp.schema.oaf.Measure; -import eu.dnetlib.dhp.schema.oaf.Result; -import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions; -import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; -import eu.dnetlib.dhp.utils.DHPUtils; - -public class PrepareBipFinder implements Serializable { - - private static final Logger log = LoggerFactory.getLogger(PrepareBipFinder.class); - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - - public static void main(String[] args) throws Exception { - - String jsonConfiguration = IOUtils - .toString( - PrepareBipFinder.class - .getResourceAsStream( - "/eu/dnetlib/dhp/actionmanager/createunresolvedentities/prepare_parameters.json")); - - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - - parser.parseArgument(args); - - Boolean isSparkSessionManaged = Optional - .ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - - final String sourcePath = parser.get("sourcePath"); - log.info("sourcePath {}: ", sourcePath); - - final String outputPath = parser.get("outputPath"); - log.info("outputPath {}: ", outputPath); - - SparkConf conf = new SparkConf(); - - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - HdfsSupport.remove(outputPath, spark.sparkContext().hadoopConfiguration()); - prepareResults(spark, sourcePath, outputPath); - }); - } - - private static void prepareResults(SparkSession spark, String inputPath, String outputPath) { - - final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - - JavaRDD bipDeserializeJavaRDD = sc - .textFile(inputPath) - .map(item -> OBJECT_MAPPER.readValue(item, BipResultModel.class)); - - spark - .createDataset(bipDeserializeJavaRDD.flatMap(entry -> entry.keySet().stream().map(key -> { - BipScore bs = new BipScore(); - bs.setId(key); - bs.setScoreList(entry.get(key)); - - return bs; - }).collect(Collectors.toList()).iterator()).rdd(), Encoders.bean(BipScore.class)) - .map((MapFunction) v -> { - Result r = new Result(); - final String cleanedPid = CleaningFunctions.normalizePidValue(DOI, v.getId()); - - r.setId(DHPUtils.generateUnresolvedIdentifier(v.getId(), DOI)); - Instance inst = new Instance(); - inst.setMeasures(getMeasure(v)); - - inst - .setPid( - Arrays - .asList( - OafMapperUtils - .structuredProperty( - cleanedPid, - OafMapperUtils - .qualifier( - DOI, DOI_CLASSNAME, - ModelConstants.DNET_PID_TYPES, - ModelConstants.DNET_PID_TYPES), - null))); - r.setInstance(Arrays.asList(inst)); - r - .setDataInfo( - OafMapperUtils - .dataInfo( - false, null, true, - false, - OafMapperUtils - .qualifier( - ModelConstants.PROVENANCE_ENRICH, - null, - ModelConstants.DNET_PROVENANCE_ACTIONS, - ModelConstants.DNET_PROVENANCE_ACTIONS), - null)); - return r; - }, Encoders.bean(Result.class)) - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .json(outputPath + "/bip"); - } - - private static List getMeasure(BipScore value) { - return value - .getScoreList() - .stream() - .map(score -> { - Measure m = new Measure(); - m.setId(score.getId()); - m - .setUnit( - score - .getUnit() - .stream() - .map(unit -> { - KeyValue kv = new KeyValue(); - kv.setValue(unit.getValue()); - kv.setKey(unit.getKey()); - kv - .setDataInfo( - OafMapperUtils - .dataInfo( - false, - UPDATE_DATA_INFO_TYPE, - true, - false, - OafMapperUtils - .qualifier( - UPDATE_MEASURE_BIP_CLASS_ID, - UPDATE_CLASS_NAME, - ModelConstants.DNET_PROVENANCE_ACTIONS, - ModelConstants.DNET_PROVENANCE_ACTIONS), - "")); - return kv; - }) - .collect(Collectors.toList())); - return m; - }) - .collect(Collectors.toList()); - } -} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareFOSSparkJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareFOSSparkJob.java index 4d2d25215..57ad8b96a 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareFOSSparkJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareFOSSparkJob.java @@ -78,12 +78,20 @@ public class PrepareFOSSparkJob implements Serializable { HashSet level1 = new HashSet<>(); HashSet level2 = new HashSet<>(); HashSet level3 = new HashSet<>(); - addLevels(level1, level2, level3, first); - it.forEachRemaining(v -> addLevels(level1, level2, level3, v)); + HashSet level4 = new HashSet<>(); + addLevels(level1, level2, level3, level4, first); + it.forEachRemaining(v -> addLevels(level1, level2, level3, level4, v)); List sbjs = new ArrayList<>(); - level1.forEach(l -> sbjs.add(getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID))); - level2.forEach(l -> sbjs.add(getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID))); - level3.forEach(l -> sbjs.add(getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID))); + level1 + .forEach(l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID))); + level2 + .forEach(l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID))); + level3 + .forEach( + l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID, true))); + level4 + .forEach( + l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID, true))); r.setSubject(sbjs); r .setDataInfo( @@ -106,11 +114,18 @@ public class PrepareFOSSparkJob implements Serializable { .json(outputPath + "/fos"); } + private static void add(List sbsjs, Subject sbj) { + if (sbj != null) + sbsjs.add(sbj); + } + private static void addLevels(HashSet level1, HashSet level2, HashSet level3, + HashSet level4, FOSDataModel first) { level1.add(first.getLevel1()); level2.add(first.getLevel2()); - level3.add(first.getLevel3()); + level3.add(first.getLevel3() + "@@" + first.getScoreL3()); + level4.add(first.getLevel4() + "@@" + first.getScoreL4()); } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/SparkSaveUnresolved.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/SparkSaveUnresolved.java index 3b9775094..93bbfcc88 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/SparkSaveUnresolved.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/SparkSaveUnresolved.java @@ -69,9 +69,9 @@ public class SparkSaveUnresolved implements Serializable { .mapGroups((MapGroupsFunction) (k, it) -> { Result ret = it.next(); it.forEachRemaining(r -> { - if (r.getInstance() != null) { - ret.setInstance(r.getInstance()); - } +// if (r.getInstance() != null) { +// ret.setInstance(r.getInstance()); +// } if (r.getSubject() != null) { if (ret.getSubject() != null) ret.getSubject().addAll(r.getSubject()); diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/model/FOSDataModel.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/model/FOSDataModel.java index e98ba74a1..a82d7bfd6 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/model/FOSDataModel.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/model/FOSDataModel.java @@ -11,21 +11,43 @@ public class FOSDataModel implements Serializable { private String doi; @CsvBindByPosition(position = 1) +// @CsvBindByName(column = "doi") + private String oaid; + @CsvBindByPosition(position = 2) // @CsvBindByName(column = "level1") private String level1; - @CsvBindByPosition(position = 2) + @CsvBindByPosition(position = 3) // @CsvBindByName(column = "level2") private String level2; - @CsvBindByPosition(position = 3) + @CsvBindByPosition(position = 4) // @CsvBindByName(column = "level3") private String level3; + @CsvBindByPosition(position = 5) +// @CsvBindByName(column = "level3") + private String level4; + @CsvBindByPosition(position = 6) + private String scoreL3; + @CsvBindByPosition(position = 7) + private String scoreL4; + public FOSDataModel() { } + public FOSDataModel(String doi, String level1, String level2, String level3, String level4, String l3score, + String l4score) { + this.doi = doi; + this.level1 = level1; + this.level2 = level2; + this.level3 = level3; + this.level4 = level4; + this.scoreL3 = l3score; + this.scoreL4 = l4score; + } + public FOSDataModel(String doi, String level1, String level2, String level3) { this.doi = doi; this.level1 = level1; @@ -33,8 +55,41 @@ public class FOSDataModel implements Serializable { this.level3 = level3; } - public static FOSDataModel newInstance(String d, String level1, String level2, String level3) { - return new FOSDataModel(d, level1, level2, level3); + public static FOSDataModel newInstance(String d, String level1, String level2, String level3, String level4, + String scorel3, String scorel4) { + return new FOSDataModel(d, level1, level2, level3, level4, scorel3, scorel4); + } + + public String getOaid() { + return oaid; + } + + public void setOaid(String oaid) { + this.oaid = oaid; + } + + public String getLevel4() { + return level4; + } + + public void setLevel4(String level4) { + this.level4 = level4; + } + + public String getScoreL3() { + return scoreL3; + } + + public void setScoreL3(String scoreL3) { + this.scoreL3 = scoreL3; + } + + public String getScoreL4() { + return scoreL4; + } + + public void setScoreL4(String scoreL4) { + this.scoreL4 = scoreL4; } public String getDoi() { diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/CreateActionSetSparkJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/CreateActionSetSparkJob.java index 4c658e52f..b707fdcd3 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/CreateActionSetSparkJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/CreateActionSetSparkJob.java @@ -10,8 +10,10 @@ import java.util.*; import org.apache.commons.cli.ParseException; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.compress.GzipCodec; import org.apache.hadoop.mapred.SequenceFileOutputFormat; import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.function.FilterFunction; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.MapFunction; @@ -26,19 +28,29 @@ import eu.dnetlib.dhp.actionmanager.opencitations.model.COCI; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.action.AtomicAction; import eu.dnetlib.dhp.schema.common.ModelConstants; -import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.*; -import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions; -import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; +import eu.dnetlib.dhp.schema.oaf.utils.*; +import eu.dnetlib.dhp.utils.DHPUtils; import scala.Tuple2; public class CreateActionSetSparkJob implements Serializable { public static final String OPENCITATIONS_CLASSID = "sysimport:crosswalk:opencitations"; public static final String OPENCITATIONS_CLASSNAME = "Imported from OpenCitations"; - private static final String ID_PREFIX = "50|doi_________::"; + + // DOI-to-DOI citations + public static final String COCI = "COCI"; + + // PMID-to-PMID citations + public static final String POCI = "POCI"; + + private static final String DOI_PREFIX = "50|doi_________::"; + + private static final String PMID_PREFIX = "50|pmid________::"; + private static final String TRUST = "0.91"; private static final Logger log = LoggerFactory.getLogger(CreateActionSetSparkJob.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); public static void main(final String[] args) throws IOException, ParseException { @@ -62,7 +74,7 @@ public class CreateActionSetSparkJob implements Serializable { log.info("isSparkSessionManaged: {}", isSparkSessionManaged); final String inputPath = parser.get("inputPath"); - log.info("inputPath {}", inputPath.toString()); + log.info("inputPath {}", inputPath); final String outputPath = parser.get("outputPath"); log.info("outputPath {}", outputPath); @@ -76,41 +88,68 @@ public class CreateActionSetSparkJob implements Serializable { runWithSparkSession( conf, isSparkSessionManaged, - spark -> { - extractContent(spark, inputPath, outputPath, shouldDuplicateRels); - }); + spark -> extractContent(spark, inputPath, outputPath, shouldDuplicateRels)); } private static void extractContent(SparkSession spark, String inputPath, String outputPath, boolean shouldDuplicateRels) { - spark + + getTextTextJavaPairRDD(spark, inputPath, shouldDuplicateRels, COCI) + .union(getTextTextJavaPairRDD(spark, inputPath, shouldDuplicateRels, POCI)) + .saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class); + } + + private static JavaPairRDD getTextTextJavaPairRDD(SparkSession spark, String inputPath, + boolean shouldDuplicateRels, String prefix) { + return spark .read() - .textFile(inputPath + "/*") + .textFile(inputPath + "/" + prefix + "/" + prefix + "_JSON/*") .map( (MapFunction) value -> OBJECT_MAPPER.readValue(value, COCI.class), Encoders.bean(COCI.class)) .flatMap( - (FlatMapFunction) value -> createRelation(value, shouldDuplicateRels).iterator(), + (FlatMapFunction) value -> createRelation( + value, shouldDuplicateRels, prefix) + .iterator(), Encoders.bean(Relation.class)) - .filter((FilterFunction) value -> value != null) + .filter((FilterFunction) Objects::nonNull) .toJavaRDD() .map(p -> new AtomicAction(p.getClass(), p)) .mapToPair( aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), - new Text(OBJECT_MAPPER.writeValueAsString(aa)))) - .saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class); - + new Text(OBJECT_MAPPER.writeValueAsString(aa)))); } - private static List createRelation(COCI value, boolean duplicate) { + private static List createRelation(COCI value, boolean duplicate, String p) { List relationList = new ArrayList<>(); + String prefix; + String citing; + String cited; - String citing = ID_PREFIX - + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", value.getCiting())); - final String cited = ID_PREFIX - + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", value.getCited())); + switch (p) { + case COCI: + prefix = DOI_PREFIX; + citing = prefix + + IdentifierFactory + .md5(PidCleaner.normalizePidValue(PidType.doi.toString(), value.getCiting())); + cited = prefix + + IdentifierFactory + .md5(PidCleaner.normalizePidValue(PidType.doi.toString(), value.getCited())); + break; + case POCI: + prefix = PMID_PREFIX; + citing = prefix + + IdentifierFactory + .md5(PidCleaner.normalizePidValue(PidType.pmid.toString(), value.getCiting())); + cited = prefix + + IdentifierFactory + .md5(PidCleaner.normalizePidValue(PidType.pmid.toString(), value.getCited())); + break; + default: + throw new IllegalStateException("Invalid prefix: " + p); + } if (!citing.equals(cited)) { relationList @@ -120,7 +159,7 @@ public class CreateActionSetSparkJob implements Serializable { cited, ModelConstants.CITES)); if (duplicate && value.getCiting().endsWith(".refs")) { - citing = ID_PREFIX + IdentifierFactory + citing = prefix + IdentifierFactory .md5( CleaningFunctions .normalizePidValue( @@ -132,59 +171,30 @@ public class CreateActionSetSparkJob implements Serializable { return relationList; } - private static Collection getRelations(String citing, String cited) { - - return Arrays - .asList( - getRelation(citing, cited, ModelConstants.CITES), - getRelation(cited, citing, ModelConstants.IS_CITED_BY)); - } - public static Relation getRelation( String source, String target, - String relclass) { - Relation r = new Relation(); - r.setCollectedfrom(getCollectedFrom()); - r.setSource(source); - r.setTarget(target); - r.setRelClass(relclass); - r.setRelType(ModelConstants.RESULT_RESULT); - r.setSubRelType(ModelConstants.CITATION); - r - .setDataInfo( - getDataInfo()); - return r; - } + String relClass) { - public static List getCollectedFrom() { - KeyValue kv = new KeyValue(); - kv.setKey(ModelConstants.OPENOCITATIONS_ID); - kv.setValue(ModelConstants.OPENOCITATIONS_NAME); - - return Arrays.asList(kv); - } - - public static DataInfo getDataInfo() { - DataInfo di = new DataInfo(); - di.setInferred(false); - di.setDeletedbyinference(false); - di.setTrust(TRUST); - - di - .setProvenanceaction( - getQualifier(OPENCITATIONS_CLASSID, OPENCITATIONS_CLASSNAME, ModelConstants.DNET_PROVENANCE_ACTIONS)); - return di; - } - - public static Qualifier getQualifier(String class_id, String class_name, - String qualifierSchema) { - Qualifier pa = new Qualifier(); - pa.setClassid(class_id); - pa.setClassname(class_name); - pa.setSchemeid(qualifierSchema); - pa.setSchemename(qualifierSchema); - return pa; + return OafMapperUtils + .getRelation( + source, + target, + ModelConstants.RESULT_RESULT, + ModelConstants.CITATION, + relClass, + Arrays + .asList( + OafMapperUtils.keyValue(ModelConstants.OPENOCITATIONS_ID, ModelConstants.OPENOCITATIONS_NAME)), + OafMapperUtils + .dataInfo( + false, null, false, false, + OafMapperUtils + .qualifier( + OPENCITATIONS_CLASSID, OPENCITATIONS_CLASSNAME, + ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS), + TRUST), + null); } } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/GetOpenCitationsRefs.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/GetOpenCitationsRefs.java index 3530c9980..600cf7df1 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/GetOpenCitationsRefs.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/GetOpenCitationsRefs.java @@ -3,6 +3,7 @@ package eu.dnetlib.dhp.actionmanager.opencitations; import java.io.*; import java.io.Serializable; +import java.util.Arrays; import java.util.Objects; import java.util.zip.GZIPOutputStream; import java.util.zip.ZipEntry; @@ -37,7 +38,7 @@ public class GetOpenCitationsRefs implements Serializable { parser.parseArgument(args); final String[] inputFile = parser.get("inputFile").split(";"); - log.info("inputFile {}", inputFile.toString()); + log.info("inputFile {}", Arrays.asList(inputFile)); final String workingPath = parser.get("workingPath"); log.info("workingPath {}", workingPath); @@ -45,6 +46,9 @@ public class GetOpenCitationsRefs implements Serializable { final String hdfsNameNode = parser.get("hdfsNameNode"); log.info("hdfsNameNode {}", hdfsNameNode); + final String prefix = parser.get("prefix"); + log.info("prefix {}", prefix); + Configuration conf = new Configuration(); conf.set("fs.defaultFS", hdfsNameNode); @@ -53,30 +57,31 @@ public class GetOpenCitationsRefs implements Serializable { GetOpenCitationsRefs ocr = new GetOpenCitationsRefs(); for (String file : inputFile) { - ocr.doExtract(workingPath + "/Original/" + file, workingPath, fileSystem); + ocr.doExtract(workingPath + "/Original/" + file, workingPath, fileSystem, prefix); } } - private void doExtract(String inputFile, String workingPath, FileSystem fileSystem) + private void doExtract(String inputFile, String workingPath, FileSystem fileSystem, String prefix) throws IOException { final Path path = new Path(inputFile); FSDataInputStream oc_zip = fileSystem.open(path); - int count = 1; + // int count = 1; try (ZipInputStream zis = new ZipInputStream(oc_zip)) { ZipEntry entry = null; while ((entry = zis.getNextEntry()) != null) { if (!entry.isDirectory()) { String fileName = entry.getName(); - fileName = fileName.substring(0, fileName.indexOf("T")) + "_" + count; - count++; + // fileName = fileName.substring(0, fileName.indexOf("T")) + "_" + count; + fileName = fileName.substring(0, fileName.lastIndexOf(".")); + // count++; try ( FSDataOutputStream out = fileSystem - .create(new Path(workingPath + "/COCI/" + fileName + ".gz")); + .create(new Path(workingPath + "/" + prefix + "/" + fileName + ".gz")); GZIPOutputStream gzipOs = new GZIPOutputStream(new BufferedOutputStream(out))) { IOUtils.copy(zis, gzipOs); diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/ReadCOCI.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/ReadCOCI.java index 4293ca187..b9c24df3b 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/ReadCOCI.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/ReadCOCI.java @@ -7,6 +7,7 @@ import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import java.io.IOException; import java.io.Serializable; +import java.util.Arrays; import java.util.Optional; import org.apache.commons.io.IOUtils; @@ -42,13 +43,16 @@ public class ReadCOCI implements Serializable { log.info("outputPath: {}", outputPath); final String[] inputFile = parser.get("inputFile").split(";"); - log.info("inputFile {}", inputFile.toString()); + log.info("inputFile {}", Arrays.asList(inputFile)); Boolean isSparkSessionManaged = isSparkSessionManaged(parser); log.info("isSparkSessionManaged: {}", isSparkSessionManaged); final String workingPath = parser.get("workingPath"); log.info("workingPath {}", workingPath); + final String format = parser.get("format"); + log.info("format {}", format); + SparkConf sconf = new SparkConf(); final String delimiter = Optional @@ -64,16 +68,17 @@ public class ReadCOCI implements Serializable { workingPath, inputFile, outputPath, - delimiter); + delimiter, + format); }); } private static void doRead(SparkSession spark, String workingPath, String[] inputFiles, String outputPath, - String delimiter) throws IOException { + String delimiter, String format) { for (String inputFile : inputFiles) { - String p_string = workingPath + "/" + inputFile + ".gz"; + String pString = workingPath + "/" + inputFile + ".gz"; Dataset cociData = spark .read() @@ -82,14 +87,20 @@ public class ReadCOCI implements Serializable { .option("inferSchema", "true") .option("header", "true") .option("quotes", "\"") - .load(p_string) + .load(pString) .repartition(100); cociData.map((MapFunction) row -> { COCI coci = new COCI(); + if (format.equals("COCI")) { + coci.setCiting(row.getString(1)); + coci.setCited(row.getString(2)); + } else { + coci.setCiting(String.valueOf(row.getInt(1))); + coci.setCited(String.valueOf(row.getInt(2))); + } coci.setOci(row.getString(0)); - coci.setCiting(row.getString(1)); - coci.setCited(row.getString(2)); + return coci; }, Encoders.bean(COCI.class)) .write() diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/createunresolvedentities/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/createunresolvedentities/oozie_app/workflow.xml index c8af64594..c8e9547dc 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/createunresolvedentities/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/createunresolvedentities/oozie_app/workflow.xml @@ -5,11 +5,6 @@ fosPath the input path of the resources to be extended - - - bipScorePath - the path where to find the bipFinder scores - outputPath the path where to store the actionset @@ -77,35 +72,10 @@ - - - - yarn - cluster - Produces the unresolved from BIP! Finder - eu.dnetlib.dhp.actionmanager.createunresolvedentities.PrepareBipFinder - dhp-aggregation-${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.warehouse.dir=${sparkSqlWarehouseDir} - - --sourcePath${bipScorePath} - --outputPath${workingDir}/prepared - - - - - yarn @@ -125,6 +95,7 @@ --sourcePath${fosPath} --outputPath${workingDir}/input/fos + --delimiter${delimiter} @@ -213,7 +184,7 @@ yarn cluster - Saves the result produced for bip and fos by grouping results with the same id + Save the unresolved entities grouping results with the same id eu.dnetlib.dhp.actionmanager.createunresolvedentities.SparkSaveUnresolved dhp-aggregation-${projectVersion}.jar diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/as_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/as_parameters.json index 308e02026..5244a6fe4 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/as_parameters.json +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/as_parameters.json @@ -16,10 +16,11 @@ "paramLongName": "isSparkSessionManaged", "paramDescription": "the hdfs name node", "paramRequired": false - }, { - "paramName": "sdr", - "paramLongName": "shouldDuplicateRels", - "paramDescription": "the hdfs name node", - "paramRequired": false -} + }, + { + "paramName": "sdr", + "paramLongName": "shouldDuplicateRels", + "paramDescription": "activates/deactivates the construction of bidirectional relations Cites/IsCitedBy", + "paramRequired": false + } ] diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_parameters.json index 4910ad11d..96db7eeb7 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_parameters.json +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_parameters.json @@ -16,5 +16,11 @@ "paramLongName": "hdfsNameNode", "paramDescription": "the hdfs name node", "paramRequired": true + }, + { + "paramName": "p", + "paramLongName": "prefix", + "paramDescription": "COCI or POCI", + "paramRequired": true } ] diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_readcoci_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_readcoci_parameters.json index b57cb5d9a..fa840089d 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_readcoci_parameters.json +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/input_readcoci_parameters.json @@ -30,7 +30,12 @@ "paramLongName": "inputFile", "paramDescription": "the hdfs name node", "paramRequired": true - } + }, { + "paramName": "f", + "paramLongName": "format", + "paramDescription": "the hdfs name node", + "paramRequired": true +} ] diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/oozie_app/workflow.xml index 0f01039f7..deb32459b 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/opencitations/oozie_app/workflow.xml @@ -34,6 +34,7 @@ Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + ${jobTracker} @@ -46,7 +47,7 @@ download.sh ${filelist} - ${workingPath}/Original + ${workingPath}/${prefix}/Original HADOOP_USER_NAME=${wf:user()} download.sh @@ -54,12 +55,14 @@ + eu.dnetlib.dhp.actionmanager.opencitations.GetOpenCitationsRefs --hdfsNameNode${nameNode} --inputFile${inputFile} - --workingPath${workingPath} + --workingPath${workingPath}/${prefix} + --prefix${prefix} @@ -82,10 +85,11 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} - --workingPath${workingPath}/COCI - --outputPath${workingPath}/COCI_JSON/ + --workingPath${workingPath}/${prefix}/${prefix} + --outputPath${workingPath}/${prefix}/${prefix}_JSON/ --delimiter${delimiter} --inputFile${inputFileCoci} + --format${prefix} @@ -108,7 +112,7 @@ --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} - --inputPath${workingPath}/COCI_JSON + --inputPath${workingPath} --outputPath${outputPath} diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/GetFosTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/GetFosTest.java index 7e0acc2bb..d4fe129df 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/GetFosTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/GetFosTest.java @@ -13,10 +13,7 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; 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.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,6 +65,7 @@ public class GetFosTest { } @Test + @Disabled void test3() throws Exception { final String sourcePath = getClass() .getResource("/eu/dnetlib/dhp/actionmanager/createunresolvedentities/fos/fos_sbs.tsv") @@ -96,4 +94,37 @@ public class GetFosTest { tmp.foreach(t -> Assertions.assertTrue(t.getLevel3() != null)); } + + @Test + void test4() throws Exception { + final String sourcePath = getClass() + .getResource("/eu/dnetlib/dhp/actionmanager/createunresolvedentities/fos/fos_sbs2.csv") + .getPath(); + + final String outputPath = workingDir.toString() + "/fos.json"; + GetFOSSparkJob + .main( + new String[] { + "--isSparkSessionManaged", Boolean.FALSE.toString(), + "--sourcePath", sourcePath, + "--delimiter", ",", + "-outputPath", outputPath + + }); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(outputPath) + .map(item -> OBJECT_MAPPER.readValue(item, FOSDataModel.class)); + + tmp.foreach(t -> Assertions.assertTrue(t.getDoi() != null)); + tmp.foreach(t -> Assertions.assertTrue(t.getLevel1() != null)); + tmp.foreach(t -> Assertions.assertTrue(t.getLevel2() != null)); + tmp.foreach(t -> Assertions.assertTrue(t.getLevel3() != null)); + tmp.foreach(t -> Assertions.assertTrue(t.getLevel4() != null)); + tmp.foreach(t -> Assertions.assertTrue(t.getScoreL3() != null)); + tmp.foreach(t -> Assertions.assertTrue(t.getScoreL4() != null)); + + } } diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareTest.java index cc8108bde..da7bcd3de 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareTest.java @@ -67,92 +67,6 @@ public class PrepareTest { spark.stop(); } - @Test - void bipPrepareTest() throws Exception { - final String sourcePath = getClass() - .getResource("/eu/dnetlib/dhp/actionmanager/createunresolvedentities/bip/bip.json") - .getPath(); - - PrepareBipFinder - .main( - new String[] { - "--isSparkSessionManaged", Boolean.FALSE.toString(), - "--sourcePath", sourcePath, - "--outputPath", workingDir.toString() + "/work" - - }); - - final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - - JavaRDD tmp = sc - .textFile(workingDir.toString() + "/work/bip") - .map(item -> OBJECT_MAPPER.readValue(item, Result.class)); - - Assertions.assertEquals(86, tmp.count()); - - String doi1 = "unresolved::10.0000/096020199389707::doi"; - - Assertions.assertEquals(1, tmp.filter(r -> r.getId().equals(doi1)).count()); - Assertions.assertEquals(1, tmp.filter(r -> r.getId().equals(doi1)).collect().get(0).getInstance().size()); - Assertions - .assertEquals( - 3, tmp.filter(r -> r.getId().equals(doi1)).collect().get(0).getInstance().get(0).getMeasures().size()); - Assertions - .assertEquals( - "6.34596412687e-09", tmp - .filter(r -> r.getId().equals(doi1)) - .collect() - .get(0) - .getInstance() - .get(0) - .getMeasures() - .stream() - .filter(sl -> sl.getId().equals("influence")) - .collect(Collectors.toList()) - .get(0) - .getUnit() - .get(0) - .getValue()); - Assertions - .assertEquals( - "0.641151896994", tmp - .filter(r -> r.getId().equals(doi1)) - .collect() - .get(0) - .getInstance() - .get(0) - .getMeasures() - .stream() - .filter(sl -> sl.getId().equals("popularity_alt")) - .collect(Collectors.toList()) - .get(0) - .getUnit() - .get(0) - .getValue()); - Assertions - .assertEquals( - "2.33375102921e-09", tmp - .filter(r -> r.getId().equals(doi1)) - .collect() - .get(0) - .getInstance() - .get(0) - .getMeasures() - .stream() - .filter(sl -> sl.getId().equals("popularity")) - .collect(Collectors.toList()) - .get(0) - .getUnit() - .get(0) - .getValue()); - - final String doi2 = "unresolved::10.3390/s18072310::doi"; - - Assertions.assertEquals(1, tmp.filter(r -> r.getId().equals(doi2)).count()); - Assertions.assertEquals(1, tmp.filter(r -> r.getId().equals(doi2)).collect().get(0).getInstance().size()); - - } - @Test void fosPrepareTest() throws Exception { final String sourcePath = getClass() @@ -222,6 +136,76 @@ public class PrepareTest { } + @Test + void fosPrepareTest2() throws Exception { + final String sourcePath = getClass() + .getResource("/eu/dnetlib/dhp/actionmanager/createunresolvedentities/fos/fos_sbs_2.json") + .getPath(); + + PrepareFOSSparkJob + .main( + new String[] { + "--isSparkSessionManaged", Boolean.FALSE.toString(), + "--sourcePath", sourcePath, + + "-outputPath", workingDir.toString() + "/work" + + }); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/work/fos") + .map(item -> OBJECT_MAPPER.readValue(item, Result.class)); + + String doi1 = "unresolved::10.1016/j.revmed.2006.07.012::doi"; + + assertEquals(13, tmp.count()); + assertEquals(1, tmp.filter(row -> row.getId().equals(doi1)).count()); + + Result result = tmp + .filter(r -> r.getId().equals(doi1)) + .first(); + + result.getSubject().forEach(s -> System.out.println(s.getValue() + " trust = " + s.getDataInfo().getTrust())); + Assertions.assertEquals(6, result.getSubject().size()); + + assertTrue( + result + .getSubject() + .stream() + .anyMatch( + s -> s.getValue().contains("03 medical and health sciences") + && s.getDataInfo().getTrust().equals(""))); + + assertTrue( + result + .getSubject() + .stream() + .anyMatch( + s -> s.getValue().contains("0302 clinical medicine") && s.getDataInfo().getTrust().equals(""))); + + assertTrue( + result + .getSubject() + .stream() + .anyMatch( + s -> s + .getValue() + .contains("030204 cardiovascular system & hematology") + && s.getDataInfo().getTrust().equals("0.5101401805877686"))); + assertTrue( + result + .getSubject() + .stream() + .anyMatch( + s -> s + .getValue() + .contains("03020409 Hematology/Coagulopathies") + && s.getDataInfo().getTrust().equals("0.0546871414174914"))); + + } + @Test void sdgPrepareTest() throws Exception { final String sourcePath = getClass() @@ -268,57 +252,4 @@ public class PrepareTest { } -// @Test -// void test3() throws Exception { -// final String sourcePath = "/Users/miriam.baglioni/Downloads/doi_fos_results_20_12_2021.csv.gz"; -// -// final String outputPath = workingDir.toString() + "/fos.json"; -// GetFOSSparkJob -// .main( -// new String[] { -// "--isSparkSessionManaged", Boolean.FALSE.toString(), -// "--sourcePath", sourcePath, -// -// "-outputPath", outputPath -// -// }); -// -// final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); -// -// JavaRDD tmp = sc -// .textFile(outputPath) -// .map(item -> OBJECT_MAPPER.readValue(item, FOSDataModel.class)); -// -// tmp.foreach(t -> Assertions.assertTrue(t.getDoi() != null)); -// tmp.foreach(t -> Assertions.assertTrue(t.getLevel1() != null)); -// tmp.foreach(t -> Assertions.assertTrue(t.getLevel2() != null)); -// tmp.foreach(t -> Assertions.assertTrue(t.getLevel3() != null)); -// -// } -// -// @Test -// void test4() throws Exception { -// final String sourcePath = "/Users/miriam.baglioni/Downloads/doi_sdg_results_20_12_21.csv.gz"; -// -// final String outputPath = workingDir.toString() + "/sdg.json"; -// GetSDGSparkJob -// .main( -// new String[] { -// "--isSparkSessionManaged", Boolean.FALSE.toString(), -// "--sourcePath", sourcePath, -// -// "-outputPath", outputPath -// -// }); -// -// final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); -// -// JavaRDD tmp = sc -// .textFile(outputPath) -// .map(item -> OBJECT_MAPPER.readValue(item, SDGDataModel.class)); -// -// tmp.foreach(t -> Assertions.assertTrue(t.getDoi() != null)); -// tmp.foreach(t -> Assertions.assertTrue(t.getSbj() != null)); -// -// } } diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/ProduceTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/ProduceTest.java index c3c110f09..ce116688a 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/ProduceTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/ProduceTest.java @@ -340,18 +340,7 @@ public class ProduceTest { } private JavaRDD getResultJavaRDD() throws Exception { - final String bipPath = getClass() - .getResource("/eu/dnetlib/dhp/actionmanager/createunresolvedentities/bip/bip.json") - .getPath(); - PrepareBipFinder - .main( - new String[] { - "--isSparkSessionManaged", Boolean.FALSE.toString(), - "--sourcePath", bipPath, - "--outputPath", workingDir.toString() + "/work" - - }); final String fosPath = getClass() .getResource("/eu/dnetlib/dhp/actionmanager/createunresolvedentities/fos/fos.json") .getPath(); @@ -379,6 +368,40 @@ public class ProduceTest { .map(item -> OBJECT_MAPPER.readValue(item, Result.class)); } + @Test + public JavaRDD getResultFosJavaRDD() throws Exception { + + final String fosPath = getClass() + .getResource("/eu/dnetlib/dhp/actionmanager/createunresolvedentities/fos/fos_sbs_2.json") + .getPath(); + + PrepareFOSSparkJob + .main( + new String[] { + "--isSparkSessionManaged", Boolean.FALSE.toString(), + "--sourcePath", fosPath, + "-outputPath", workingDir.toString() + "/work" + }); + + SparkSaveUnresolved.main(new String[] { + "--isSparkSessionManaged", Boolean.FALSE.toString(), + "--sourcePath", workingDir.toString() + "/work", + + "-outputPath", workingDir.toString() + "/unresolved" + + }); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/unresolved") + .map(item -> OBJECT_MAPPER.readValue(item, Result.class)); + tmp.foreach(r -> System.out.println(new ObjectMapper().writeValueAsString(r))); + + return tmp; + + } + @Test void prepareTest5Subjects() throws Exception { final String doi = "unresolved::10.1063/5.0032658::doi"; @@ -415,18 +438,7 @@ public class ProduceTest { } private JavaRDD getResultJavaRDDPlusSDG() throws Exception { - final String bipPath = getClass() - .getResource("/eu/dnetlib/dhp/actionmanager/createunresolvedentities/bip/bip.json") - .getPath(); - PrepareBipFinder - .main( - new String[] { - "--isSparkSessionManaged", Boolean.FALSE.toString(), - "--sourcePath", bipPath, - "--outputPath", workingDir.toString() + "/work" - - }); final String fosPath = getClass() .getResource("/eu/dnetlib/dhp/actionmanager/createunresolvedentities/fos/fos.json") .getPath(); @@ -483,14 +495,6 @@ public class ProduceTest { .filter(row -> row.getSubject() != null) .count()); - Assertions - .assertEquals( - 85, - tmp - .filter(row -> !row.getId().equals(doi)) - .filter(r -> r.getInstance() != null && r.getInstance().size() > 0) - .count()); - } @Test diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/createunresolvedentities/fos/fos_sbs2.csv b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/createunresolvedentities/fos/fos_sbs2.csv new file mode 100644 index 000000000..3b1f2304f --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/createunresolvedentities/fos/fos_sbs2.csv @@ -0,0 +1,26 @@ +DOI,OAID,level1,level2,level3,level4,score_for_L3,score_for_L4 +10.1016/j.anucene.2006.02.004,doi_________::00059d9963edf633bec756fb21b5bd72,02 engineering and technology,"0202 electrical engineering, electronic engineering, information engineering",020209 energy,02020908 Climate change policy/Ethanol fuel,0.5,0.5 +10.1016/j.anucene.2006.02.004,doi_________::00059d9963edf633bec756fb21b5bd72,02 engineering and technology,0211 other engineering and technologies,021108 energy,02110808 Climate change policy/Ethanol fuel,0.5,0.5 +10.1016/j.revmed.2006.07.010,doi_________::0026476c1651a92c933d752ff12496c7,03 medical and health sciences,0302 clinical medicine,030220 oncology & carcinogenesis,N/A,0.5036656856536865,0.0 +10.1016/j.revmed.2006.07.010,doi_________::0026476c1651a92c933d752ff12496c7,03 medical and health sciences,0302 clinical medicine,030212 general & internal medicine,N/A,0.4963343143463135,0.0 +10.20965/jrm.2006.p0312,doi_________::0028336a2f3826cc83c47dbefac71543,02 engineering and technology,0209 industrial biotechnology,020901 industrial engineering & automation,02090104 Robotics/Robots,0.6111094951629639,0.5053805979936855 +10.20965/jrm.2006.p0312,doi_________::0028336a2f3826cc83c47dbefac71543,01 natural sciences,0104 chemical sciences,010401 analytical chemistry,N/A,0.3888905048370361,0.0 +10.1111/j.1747-7379.2006.040_1.x,doi_________::002c7077e7c114a8304eb90f59e45fa4,05 social sciences,0506 political science,050602 political science & public administration,05060202 Ethnic groups/Ethnicity,0.6159052848815918,0.7369035568037298 +10.1111/j.1747-7379.2006.040_1.x,doi_________::002c7077e7c114a8304eb90f59e45fa4,05 social sciences,0502 economics and business,050207 economics,N/A,0.3840946555137634,0.0 +10.1007/s10512-006-0049-9,doi_________::003f29f9254819cf4c78558b1bc25f10,02 engineering and technology,"0202 electrical engineering, electronic engineering, information engineering",020209 energy,02020908 Climate change policy/Ethanol fuel,0.5,0.5 +10.1007/s10512-006-0049-9,doi_________::003f29f9254819cf4c78558b1bc25f10,02 engineering and technology,0211 other engineering and technologies,021108 energy,02110808 Climate change policy/Ethanol fuel,0.5,0.5 +10.1111/j.1365-2621.2005.01045.x,doi_________::00419355b4c3e0646bd0e1b301164c8e,04 agricultural and veterinary sciences,0404 agricultural biotechnology,040401 food science,04040102 Food science/Food industry,0.5,0.5 +10.1111/j.1365-2621.2005.01045.x,doi_________::00419355b4c3e0646bd0e1b301164c8e,04 agricultural and veterinary sciences,0405 other agricultural sciences,040502 food science,04050202 Food science/Food industry,0.5,0.5 +10.1002/chin.200617262,doi_________::004c8cef80668904961b9e62841793c8,01 natural sciences,0104 chemical sciences,010405 organic chemistry,01040508 Functional groups/Ethers,0.5566747188568115,0.5582916736602783 +10.1002/chin.200617262,doi_________::004c8cef80668904961b9e62841793c8,01 natural sciences,0104 chemical sciences,010402 general chemistry,01040207 Chemical synthesis/Total synthesis,0.4433253407478332,0.4417082965373993 +10.1016/j.revmed.2006.07.012,doi_________::005b1d0fb650b680abaf6cfe26a21604,03 medical and health sciences,0302 clinical medicine,030204 cardiovascular system & hematology,03020409 Hematology/Coagulopathies,0.5101401805877686,0.0546871414174914 +10.1016/j.revmed.2006.07.012,doi_________::005b1d0fb650b680abaf6cfe26a21604,03 medical and health sciences,0301 basic medicine,030105 genetics & heredity,N/A,0.4898599088191986,0.0 +10.4109/jslab.17.132,doi_________::00889baa06de363e37930daaf8e800c0,03 medical and health sciences,0301 basic medicine,030104 developmental biology,N/A,0.5,0.0 +10.4109/jslab.17.132,doi_________::00889baa06de363e37930daaf8e800c0,03 medical and health sciences,0303 health sciences,030304 developmental biology,N/A,0.5,0.0 +10.1108/00251740610715687,doi_________::0092cb1b1920d556719385a26363ecaa,05 social sciences,0502 economics and business,050203 business & management,05020311 International business/International trade,0.605047881603241,0.2156608108845153 +10.1108/00251740610715687,doi_________::0092cb1b1920d556719385a26363ecaa,05 social sciences,0502 economics and business,050211 marketing,N/A,0.394952118396759,0.0 +10.1080/03067310500248098,doi_________::00a76678d230e3f20b6356804448028f,04 agricultural and veterinary sciences,0404 agricultural biotechnology,040401 food science,04040102 Food science/Food industry,0.5,0.5 +10.1080/03067310500248098,doi_________::00a76678d230e3f20b6356804448028f,04 agricultural and veterinary sciences,0405 other agricultural sciences,040502 food science,04050202 Food science/Food industry,0.5,0.5 +10.3152/147154306781778533,doi_________::00acc520f3939e5a6675343881fed4f2,05 social sciences,0502 economics and business,050203 business & management,05020307 Innovation/Product management,0.5293408632278442,0.5326762795448303 +10.3152/147154306781778533,doi_________::00acc520f3939e5a6675343881fed4f2,05 social sciences,0509 other social sciences,050905 science studies,05090502 Social philosophy/Capitalism,0.4706590473651886,0.4673237204551697 +10.1785/0120050806,doi_________::00d5831d329e7ae4523d78bfc3042e98,02 engineering and technology,0211 other engineering and technologies,021101 geological & geomatics engineering,02110103 Concrete/Building materials,0.5343400835990906,0.3285667930180677 \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/createunresolvedentities/fos/fos_sbs_2.json b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/createunresolvedentities/fos/fos_sbs_2.json new file mode 100644 index 000000000..00ffad70c --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/createunresolvedentities/fos/fos_sbs_2.json @@ -0,0 +1,25 @@ +{"doi":"10.1016/j.anucene.2006.02.004","level1":"02 engineering and technology","level2":"0202 electrical engineering, electronic engineering, information engineering","level3":"020209 energy","level4":"02020908 Climate change policy/Ethanol fuel","scoreL3":"0.5","scoreL4":"0.5"} +{"doi":"10.1016/j.anucene.2006.02.004","level1":"02 engineering and technology","level2":"0211 other engineering and technologies","level3":"021108 energy","level4":"02110808 Climate change policy/Ethanol fuel","scoreL3":"0.5","scoreL4":"0.5"} +{"doi":"10.1016/j.revmed.2006.07.010","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030220 oncology & carcinogenesis","level4":"N/A","scoreL3":"0.5036656856536865","scoreL4":"0.0"} +{"doi":"10.1016/j.revmed.2006.07.010","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030212 general & internal medicine","level4":"N/A","scoreL3":"0.4963343143463135","scoreL4":"0.0"} +{"doi":"10.20965/jrm.2006.p0312","level1":"02 engineering and technology","level2":"0209 industrial biotechnology","level3":"020901 industrial engineering & automation","level4":"02090104 Robotics/Robots","scoreL3":"0.6111094951629639","scoreL4":"0.5053805979936855"} +{"doi":"10.20965/jrm.2006.p0312","level1":"01 natural sciences","level2":"0104 chemical sciences","level3":"010401 analytical chemistry","level4":"N/A","scoreL3":"0.3888905048370361","scoreL4":"0.0"} +{"doi":"10.1111/j.1747-7379.2006.040_1.x","level1":"05 social sciences","level2":"0506 political science","level3":"050602 political science & public administration","level4":"05060202 Ethnic groups/Ethnicity","scoreL3":"0.6159052848815918","scoreL4":"0.7369035568037298"} +{"doi":"10.1111/j.1747-7379.2006.040_1.x","level1":"05 social sciences","level2":"0502 economics and business","level3":"050207 economics","level4":"N/A","scoreL3":"0.3840946555137634","scoreL4":"0.0"} +{"doi":"10.1007/s10512-006-0049-9","level1":"02 engineering and technology","level2":"0202 electrical engineering, electronic engineering, information engineering","level3":"020209 energy","level4":"02020908 Climate change policy/Ethanol fuel","scoreL3":"0.5","scoreL4":"0.5"} +{"doi":"10.1007/s10512-006-0049-9","level1":"02 engineering and technology","level2":"0211 other engineering and technologies","level3":"021108 energy","level4":"02110808 Climate change policy/Ethanol fuel","scoreL3":"0.5","scoreL4":"0.5"} +{"doi":"10.1111/j.1365-2621.2005.01045.x","level1":"04 agricultural and veterinary sciences","level2":"0404 agricultural biotechnology","level3":"040401 food science","level4":"04040102 Food science/Food industry","scoreL3":"0.5","scoreL4":"0.5"} +{"doi":"10.1111/j.1365-2621.2005.01045.x","level1":"04 agricultural and veterinary sciences","level2":"0405 other agricultural sciences","level3":"040502 food science","level4":"04050202 Food science/Food industry","scoreL3":"0.5","scoreL4":"0.5"} +{"doi":"10.1002/chin.200617262","level1":"01 natural sciences","level2":"0104 chemical sciences","level3":"010405 organic chemistry","level4":"01040508 Functional groups/Ethers","scoreL3":"0.5566747188568115","scoreL4":"0.5582916736602783"} +{"doi":"10.1002/chin.200617262","level1":"01 natural sciences","level2":"0104 chemical sciences","level3":"010402 general chemistry","level4":"01040207 Chemical synthesis/Total synthesis","scoreL3":"0.4433253407478332","scoreL4":"0.4417082965373993"} +{"doi":"10.1016/j.revmed.2006.07.012","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030204 cardiovascular system & hematology","level4":"03020409 Hematology/Coagulopathies","scoreL3":"0.5101401805877686","scoreL4":"0.0546871414174914"} +{"doi":"10.1016/j.revmed.2006.07.012","level1":"03 medical and health sciences","level2":"0301 basic medicine","level3":"030105 genetics & heredity","level4":"N/A","scoreL3":"0.4898599088191986","scoreL4":"0.0"} +{"doi":"10.4109/jslab.17.132","level1":"03 medical and health sciences","level2":"0301 basic medicine","level3":"030104 developmental biology","level4":"N/A","scoreL3":"0.5","scoreL4":"0.0"} +{"doi":"10.4109/jslab.17.132","level1":"03 medical and health sciences","level2":"0303 health sciences","level3":"030304 developmental biology","level4":"N/A","scoreL3":"0.5","scoreL4":"0.0"} +{"doi":"10.1108/00251740610715687","level1":"05 social sciences","level2":"0502 economics and business","level3":"050203 business & management","level4":"05020311 International business/International trade","scoreL3":"0.605047881603241","scoreL4":"0.2156608108845153"} +{"doi":"10.1108/00251740610715687","level1":"05 social sciences","level2":"0502 economics and business","level3":"050211 marketing","level4":"N/A","scoreL3":"0.394952118396759","scoreL4":"0.0"} +{"doi":"10.1080/03067310500248098","level1":"04 agricultural and veterinary sciences","level2":"0404 agricultural biotechnology","level3":"040401 food science","level4":"04040102 Food science/Food industry","scoreL3":"0.5","scoreL4":"0.5"} +{"doi":"10.1080/03067310500248098","level1":"04 agricultural and veterinary sciences","level2":"0405 other agricultural sciences","level3":"040502 food science","level4":"04050202 Food science/Food industry","scoreL3":"0.5","scoreL4":"0.5"} +{"doi":"10.3152/147154306781778533","level1":"05 social sciences","level2":"0502 economics and business","level3":"050203 business & management","level4":"05020307 Innovation/Product management","scoreL3":"0.5293408632278442","scoreL4":"0.5326762795448303"} +{"doi":"10.3152/147154306781778533","level1":"05 social sciences","level2":"0509 other social sciences","level3":"050905 science studies","level4":"05090502 Social philosophy/Capitalism","scoreL3":"0.4706590473651886","scoreL4":"0.4673237204551697"} +{"doi":"10.1785/0120050806","level1":"02 engineering and technology","level2":"0211 other engineering and technologies","level3":"021101 geological & geomatics engineering","level4":"02110103 Concrete/Building materials","scoreL3":"0.5343400835990906","scoreL4":"0.3285667930180677"} 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 deleted file mode 100644 index 96d783dbf..000000000 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/RelationAggregator.java +++ /dev/null @@ -1,57 +0,0 @@ - -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 final 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/SparkCleanRelation.scala b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCleanRelation.scala deleted file mode 100644 index 5d8da42c2..000000000 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCleanRelation.scala +++ /dev/null @@ -1,78 +0,0 @@ -package eu.dnetlib.dhp.oa.dedup - -import eu.dnetlib.dhp.application.ArgumentApplicationParser -import eu.dnetlib.dhp.common.HdfsSupport -import eu.dnetlib.dhp.schema.oaf.Relation -import eu.dnetlib.dhp.utils.ISLookupClientFactory -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService -import org.apache.commons.io.IOUtils -import org.apache.spark.SparkConf -import org.apache.spark.sql._ -import org.apache.spark.sql.functions.col -import org.apache.spark.sql.types.{DataTypes, StructField, StructType} -import org.slf4j.LoggerFactory - -object SparkCleanRelation { - private val log = LoggerFactory.getLogger(classOf[SparkCleanRelation]) - - @throws[Exception] - def main(args: Array[String]): Unit = { - val parser = new ArgumentApplicationParser( - IOUtils.toString( - classOf[SparkCleanRelation].getResourceAsStream("/eu/dnetlib/dhp/oa/dedup/cleanRelation_parameters.json") - ) - ) - parser.parseArgument(args) - val conf = new SparkConf - - new SparkCleanRelation(parser, AbstractSparkAction.getSparkSession(conf)) - .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))) - } -} - -class SparkCleanRelation(parser: ArgumentApplicationParser, spark: SparkSession) - extends AbstractSparkAction(parser, spark) { - override def run(isLookUpService: ISLookUpService): Unit = { - val graphBasePath = parser.get("graphBasePath") - val inputPath = parser.get("inputPath") - val outputPath = parser.get("outputPath") - - SparkCleanRelation.log.info("graphBasePath: '{}'", graphBasePath) - SparkCleanRelation.log.info("inputPath: '{}'", inputPath) - SparkCleanRelation.log.info("outputPath: '{}'", outputPath) - - AbstractSparkAction.removeOutputDir(spark, outputPath) - - val entities = - Seq("datasource", "project", "organization", "publication", "dataset", "software", "otherresearchproduct") - - val idsSchema = StructType.fromDDL("`id` STRING, `dataInfo` STRUCT<`deletedbyinference`:BOOLEAN,`invisible`:BOOLEAN>") - - val emptyIds = spark.createDataFrame(spark.sparkContext.emptyRDD[Row].setName("empty"), - idsSchema) - - val ids = entities - .foldLeft(emptyIds)((ds, entity) => { - val entityPath = graphBasePath + '/' + entity - if (HdfsSupport.exists(entityPath, spark.sparkContext.hadoopConfiguration)) { - ds.union(spark.read.schema(idsSchema).json(entityPath)) - } else { - ds - } - }) - .filter("dataInfo.deletedbyinference != true AND dataInfo.invisible != true") - .select("id") - .distinct() - - val relations = spark.read.schema(Encoders.bean(classOf[Relation]).schema).json(inputPath) - .filter("dataInfo.deletedbyinference != true AND dataInfo.invisible != true") - - AbstractSparkAction.save( - relations - .join(ids, col("source") === ids("id"), "leftsemi") - .join(ids, col("target") === ids("id"), "leftsemi"), - outputPath, - SaveMode.Overwrite - ) - } -} diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java index 9d0f61007..eca2193af 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java @@ -7,6 +7,7 @@ import java.util.Optional; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; @@ -77,13 +78,12 @@ public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction { log.info("Number of Openorgs Merge Relations collected: {}", mergeRelsRDD.count()); - spark + final Dataset relations = spark .createDataset( mergeRelsRDD.rdd(), - Encoders.bean(Relation.class)) - .write() - .mode(SaveMode.Append) - .parquet(outputPath); + Encoders.bean(Relation.class)); + + saveParquet(relations, outputPath, SaveMode.Append); } private boolean isMergeRel(Relation rel) { diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRelationsNoOpenorgs.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRelationsNoOpenorgs.java index 62cbb5bff..e10f41c82 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRelationsNoOpenorgs.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRelationsNoOpenorgs.java @@ -67,12 +67,7 @@ public class SparkCopyRelationsNoOpenorgs extends AbstractSparkAction { log.debug("Number of non-Openorgs relations collected: {}", simRels.count()); } - spark - .createDataset(simRels.rdd(), Encoders.bean(Relation.class)) - .write() - .mode(SaveMode.Overwrite) - .json(outputPath); - + save(spark.createDataset(simRels.rdd(), Encoders.bean(Relation.class)), outputPath, SaveMode.Overwrite); } } 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 2f551b244..babbaaabd 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 @@ -155,7 +155,7 @@ public class SparkCreateMergeRels extends AbstractSparkAction { (FlatMapFunction) cc -> ccToMergeRel(cc, dedupConf), Encoders.bean(Relation.class)); - mergeRels.write().mode(SaveMode.Overwrite).parquet(mergeRelPath); + saveParquet(mergeRels, mergeRelPath, SaveMode.Overwrite); } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateOrgsDedupRecord.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateOrgsDedupRecord.java index 8e5e9fd69..25e394f25 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateOrgsDedupRecord.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateOrgsDedupRecord.java @@ -72,11 +72,7 @@ public class SparkCreateOrgsDedupRecord extends AbstractSparkAction { final String mergeRelsPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, "organization"); - rootOrganization(spark, entityPath, mergeRelsPath) - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .json(outputPath); + save(rootOrganization(spark, entityPath, mergeRelsPath), outputPath, SaveMode.Overwrite); } 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 5b3cc3111..5f54c34df 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 @@ -82,8 +82,6 @@ public class SparkCreateSimRels extends AbstractSparkAction { final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity); removeOutputDir(spark, outputPath); - JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - SparkDeduper deduper = new SparkDeduper(dedupConf); Dataset simRels = spark 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 175ebf8a6..cb1c70059 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPropagateRelation.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPropagateRelation.java @@ -3,23 +3,19 @@ package eu.dnetlib.dhp.oa.dedup; import static org.apache.spark.sql.functions.col; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.Objects; - -import org.apache.commons.beanutils.BeanUtils; 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.api.java.function.ReduceFunction; import org.apache.spark.sql.*; +import org.apache.spark.sql.catalyst.encoders.RowEncoder; +import org.apache.spark.sql.types.StructType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.schema.common.EntityType; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.DataInfo; @@ -70,73 +66,63 @@ public class SparkPropagateRelation extends AbstractSparkAction { log.info("workingPath: '{}'", workingPath); log.info("graphOutputPath: '{}'", graphOutputPath); - final String outputRelationPath = DedupUtility.createEntityPath(graphOutputPath, "relation"); - removeOutputDir(spark, outputRelationPath); - Dataset mergeRels = spark .read() .load(DedupUtility.createMergeRelPath(workingPath, "*", "*")) .as(REL_BEAN_ENC); // - Dataset mergedIds = mergeRels + Dataset idsToMerge = mergeRels .where(col("relClass").equalTo(ModelConstants.MERGES)) .select(col("source").as("dedupID"), col("target").as("mergedObjectID")) - .distinct() - .cache(); + .distinct(); Dataset allRels = spark .read() .schema(REL_BEAN_ENC.schema()) - .json(DedupUtility.createEntityPath(graphBasePath, "relation")); + .json(graphBasePath + "/relation"); Dataset dedupedRels = allRels - .joinWith(mergedIds, allRels.col("source").equalTo(mergedIds.col("mergedObjectID")), "left_outer") - .joinWith(mergedIds, col("_1.target").equalTo(mergedIds.col("mergedObjectID")), "left_outer") + .joinWith(idsToMerge, allRels.col("source").equalTo(idsToMerge.col("mergedObjectID")), "left_outer") + .joinWith(idsToMerge, col("_1.target").equalTo(idsToMerge.col("mergedObjectID")), "left_outer") .select("_1._1", "_1._2.dedupID", "_2.dedupID") .as(Encoders.tuple(REL_BEAN_ENC, Encoders.STRING(), Encoders.STRING())) - .flatMap(SparkPropagateRelation::addInferredRelations, REL_KRYO_ENC); + .map((MapFunction, Relation>) t -> { + Relation rel = t._1(); + String newSource = t._2(); + String newTarget = t._3(); - Dataset processedRelations = distinctRelations( - dedupedRels.union(mergeRels.map((MapFunction) r -> r, REL_KRYO_ENC))) - .filter((FilterFunction) r -> !Objects.equals(r.getSource(), r.getTarget())); + if (rel.getDataInfo() == null) { + rel.setDataInfo(new DataInfo()); + } - save(processedRelations, outputRelationPath, SaveMode.Overwrite); - } + if (newSource != null || newTarget != null) { + rel.getDataInfo().setDeletedbyinference(false); - private static Iterator addInferredRelations(Tuple3 t) throws Exception { - Relation existingRel = t._1(); - String newSource = t._2(); - String newTarget = t._3(); + if (newSource != null) + rel.setSource(newSource); - if (newSource == null && newTarget == null) { - return Collections.singleton(t._1()).iterator(); - } + if (newTarget != null) + rel.setTarget(newTarget); + } - // update existing relation - if (existingRel.getDataInfo() == null) { - existingRel.setDataInfo(new DataInfo()); - } - existingRel.getDataInfo().setDeletedbyinference(true); + return rel; + }, REL_BEAN_ENC); - // Create new relation inferred by dedupIDs - Relation inferredRel = (Relation) BeanUtils.cloneBean(existingRel); + // ids of records that are both not deletedbyinference and not invisible + Dataset ids = validIds(spark, graphBasePath); - inferredRel.setDataInfo((DataInfo) BeanUtils.cloneBean(existingRel.getDataInfo())); - inferredRel.getDataInfo().setDeletedbyinference(false); + // filter relations that point to valid records, can force them to be visible + Dataset cleanedRels = dedupedRels + .join(ids, col("source").equalTo(ids.col("id")), "leftsemi") + .join(ids, col("target").equalTo(ids.col("id")), "leftsemi") + .as(REL_BEAN_ENC) + .map((MapFunction) r -> { + r.getDataInfo().setInvisible(false); + return r; + }, REL_KRYO_ENC); - if (newSource != null) - inferredRel.setSource(newSource); - - if (newTarget != null) - inferredRel.setTarget(newTarget); - - return Arrays.asList(existingRel, inferredRel).iterator(); - } - - private Dataset distinctRelations(Dataset rels) { - return rels - .filter(getRelationFilterFunction()) + Dataset distinctRels = cleanedRels .groupByKey( (MapFunction) r -> String .join(" ", r.getSource(), r.getTarget(), r.getRelType(), r.getSubRelType(), r.getRelClass()), @@ -146,13 +132,33 @@ public class SparkPropagateRelation extends AbstractSparkAction { return b; }) .map((MapFunction, Relation>) Tuple2::_2, REL_BEAN_ENC); + + final String outputRelationPath = graphOutputPath + "/relation"; + removeOutputDir(spark, outputRelationPath); + save( + distinctRels + .union(mergeRels) + .filter("source != target AND dataInfo.deletedbyinference != true AND dataInfo.invisible != true"), + outputRelationPath, + SaveMode.Overwrite); } - private FilterFunction getRelationFilterFunction() { - return r -> StringUtils.isNotBlank(r.getSource()) || - StringUtils.isNotBlank(r.getTarget()) || - StringUtils.isNotBlank(r.getRelType()) || - StringUtils.isNotBlank(r.getSubRelType()) || - StringUtils.isNotBlank(r.getRelClass()); + static Dataset validIds(SparkSession spark, String graphBasePath) { + StructType idsSchema = StructType + .fromDDL("`id` STRING, `dataInfo` STRUCT<`deletedbyinference`:BOOLEAN,`invisible`:BOOLEAN>"); + + Dataset allIds = spark.emptyDataset(RowEncoder.apply(idsSchema)); + + for (EntityType entityType : ModelSupport.entityTypes.keySet()) { + String entityPath = graphBasePath + '/' + entityType.name(); + if (HdfsSupport.exists(entityPath, spark.sparkContext().hadoopConfiguration())) { + allIds = allIds.union(spark.read().schema(idsSchema).json(entityPath)); + } + } + + return allIds + .filter("dataInfo.deletedbyinference != true AND dataInfo.invisible != true") + .select("id") + .distinct(); } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkWhitelistSimRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkWhitelistSimRels.java index 94a09ed05..65ad0c327 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkWhitelistSimRels.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkWhitelistSimRels.java @@ -67,8 +67,6 @@ public class SparkWhitelistSimRels extends AbstractSparkAction { log.info("workingPath: '{}'", workingPath); log.info("whiteListPath: '{}'", whiteListPath); - JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - // file format: source####target Dataset whiteListRels = spark .read() diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/cleanRelation_parameters.json b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/cleanRelation_parameters.json deleted file mode 100644 index 860539ad9..000000000 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/cleanRelation_parameters.json +++ /dev/null @@ -1,20 +0,0 @@ -[ - { - "paramName": "i", - "paramLongName": "graphBasePath", - "paramDescription": "the base path of raw graph", - "paramRequired": true - }, - { - "paramName": "w", - "paramLongName": "inputPath", - "paramDescription": "the path to the input relation to cleanup", - "paramRequired": true - }, - { - "paramName": "o", - "paramLongName": "outputPath", - "paramDescription": "the path of the output relation cleaned", - "paramRequired": true - } -] \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/consistency/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/consistency/oozie_app/workflow.xml index b724e5d0b..5e2fc0a01 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/consistency/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/consistency/oozie_app/workflow.xml @@ -100,35 +100,9 @@ --conf spark.sql.shuffle.partitions=15000 --graphBasePath${graphBasePath} - --graphOutputPath${workingPath}/propagaterelation/ + --graphOutputPath${graphOutputPath} --workingPath${workingPath} - - - - - - - yarn - cluster - Clean Relations - eu.dnetlib.dhp.oa.dedup.SparkCleanRelation - dhp-dedup-openaire-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --conf spark.executor.memoryOverhead=${sparkExecutorMemoryOverhead} - --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=15000 - - --graphBasePath${graphBasePath} - --inputPath${workingPath}/propagaterelation/relation - --outputPath${graphOutputPath}/relation - @@ -152,31 +126,7 @@ --conf spark.sql.shuffle.partitions=15000 --graphInputPath${graphBasePath} - --outputPath${workingPath}/grouped_entities - - - - - - - - yarn - cluster - Dispatch grouped entitities - eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob - dhp-dedup-openaire-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --conf spark.executor.memoryOverhead=${sparkExecutorMemoryOverhead} - --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=7680 - - --inputPath${workingPath}/grouped_entities + --checkpointPath${workingPath}/grouped_entities --outputPath${graphOutputPath} --filterInvisible${filterInvisible} 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 38bd72a5e..6c4935637 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,7 +3,6 @@ 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.*; import static org.mockito.Mockito.lenient; @@ -23,14 +22,13 @@ import java.util.stream.Collectors; import org.apache.commons.io.FileUtils; 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.MapFunction; -import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.sql.*; import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; import org.junit.jupiter.api.*; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; @@ -46,8 +44,6 @@ import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -import eu.dnetlib.pace.util.MapDocumentUtil; -import scala.Tuple2; @ExtendWith(MockitoExtension.class) @TestMethodOrder(MethodOrderer.OrderAnnotation.class) @@ -62,6 +58,8 @@ public class SparkDedupTest implements Serializable { private static String testGraphBasePath; private static String testOutputBasePath; private static String testDedupGraphBasePath; + private static String testConsistencyGraphBasePath; + private static final String testActionSetId = "test-orchestrator"; private static String whitelistPath; private static List whiteList; @@ -75,6 +73,7 @@ public class SparkDedupTest implements Serializable { .get(SparkDedupTest.class.getResource("/eu/dnetlib/dhp/dedup/entities").toURI()) .toFile() .getAbsolutePath(); + testOutputBasePath = createTempDirectory(SparkDedupTest.class.getSimpleName() + "-") .toAbsolutePath() .toString(); @@ -83,6 +82,10 @@ public class SparkDedupTest implements Serializable { .toAbsolutePath() .toString(); + testConsistencyGraphBasePath = createTempDirectory(SparkDedupTest.class.getSimpleName() + "-") + .toAbsolutePath() + .toString(); + whitelistPath = Paths .get(SparkDedupTest.class.getResource("/eu/dnetlib/dhp/dedup/whitelist.simrels.txt").toURI()) .toFile() @@ -674,22 +677,45 @@ public class SparkDedupTest implements Serializable { assertEquals(mergedOrp, deletedOrp); } + @Test + @Order(6) + void copyRelationsNoOpenorgsTest() throws Exception { + + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCopyRelationsNoOpenorgs.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json"))); + parser + .parseArgument( + new String[] { + "-i", testGraphBasePath, "-w", testOutputBasePath, "-o", testDedupGraphBasePath + }); + + new SparkCopyRelationsNoOpenorgs(parser, spark).run(isLookUpService); + + final Dataset outputRels = spark.read().text(testDedupGraphBasePath + "/relation"); + + System.out.println(outputRels.count()); + // assertEquals(2382, outputRels.count()); + } + @Test @Order(7) void propagateRelationTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/propagateRelation_parameters.json")); - String outputRelPath = testDedupGraphBasePath + "/propagaterelation"; parser .parseArgument( new String[] { - "-i", testGraphBasePath, "-w", testOutputBasePath, "-o", outputRelPath + "-i", testDedupGraphBasePath, "-w", testOutputBasePath, "-o", testConsistencyGraphBasePath }); new SparkPropagateRelation(parser, spark).run(isLookUpService); - long relations = jsc.textFile(outputRelPath + "/relation").count(); + long relations = jsc.textFile(testDedupGraphBasePath + "/relation").count(); // assertEquals(4860, relations); System.out.println("relations = " + relations); @@ -699,95 +725,52 @@ public class SparkDedupTest implements Serializable { .read() .load(DedupUtility.createMergeRelPath(testOutputBasePath, "*", "*")) .as(Encoders.bean(Relation.class)); - final JavaPairRDD mergedIds = mergeRels - .where("relClass == 'merges'") - .select(mergeRels.col("target")) - .distinct() - .toJavaRDD() - .mapToPair( - (PairFunction) r -> new Tuple2(r.getString(0), "d")); - JavaRDD toCheck = jsc - .textFile(outputRelPath + "/relation") - .mapToPair(json -> new Tuple2<>(MapDocumentUtil.getJPathString("$.source", json), json)) - .join(mergedIds) - .map(t -> t._2()._1()) - .mapToPair(json -> new Tuple2<>(MapDocumentUtil.getJPathString("$.target", json), json)) - .join(mergedIds) - .map(t -> t._2()._1()); + Dataset inputRels = spark + .read() + .json(testDedupGraphBasePath + "/relation"); - long deletedbyinference = toCheck.filter(this::isDeletedByInference).count(); - long updated = toCheck.count(); + Dataset outputRels = spark + .read() + .json(testConsistencyGraphBasePath + "/relation"); - assertEquals(updated, deletedbyinference); + assertEquals( + 0, outputRels + .filter("dataInfo.deletedbyinference == true OR dataInfo.invisible == true") + .count()); + + assertEquals( + 5, outputRels + .filter("relClass NOT IN ('merges', 'isMergedIn')") + .count()); + + assertEquals(5 + mergeRels.count(), outputRels.count()); } @Test @Order(8) - void testCleanBaseRelations() throws Exception { - ArgumentApplicationParser parser = new ArgumentApplicationParser( - classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/cleanRelation_parameters.json")); - - // append dangling relations to be cleaned up + void testCleanedPropagatedRelations() throws Exception { Dataset df_before = spark .read() .schema(Encoders.bean(Relation.class).schema()) - .json(testGraphBasePath + "/relation"); - Dataset df_input = df_before - .unionByName(df_before.drop("source").withColumn("source", functions.lit("n/a"))) - .unionByName(df_before.drop("target").withColumn("target", functions.lit("n/a"))); - df_input.write().mode(SaveMode.Overwrite).json(testOutputBasePath + "_tmp"); - - parser - .parseArgument( - new String[] { - "--graphBasePath", testGraphBasePath, - "--inputPath", testGraphBasePath + "/relation", - "--outputPath", testDedupGraphBasePath + "/relation" - }); - - new SparkCleanRelation(parser, spark).run(isLookUpService); + .json(testDedupGraphBasePath + "/relation"); Dataset df_after = spark .read() .schema(Encoders.bean(Relation.class).schema()) - .json(testDedupGraphBasePath + "/relation"); - - assertNotEquals(df_before.count(), df_input.count()); - assertNotEquals(df_input.count(), df_after.count()); - assertEquals(5, df_after.count()); - } - - @Test - @Order(9) - void testCleanDedupedRelations() throws Exception { - ArgumentApplicationParser parser = new ArgumentApplicationParser( - classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/cleanRelation_parameters.json")); - - String inputRelPath = testDedupGraphBasePath + "/propagaterelation/relation"; - - // append dangling relations to be cleaned up - Dataset df_before = spark.read().schema(Encoders.bean(Relation.class).schema()).json(inputRelPath); - - df_before.filter(col("dataInfo.deletedbyinference").notEqual(true)).show(50, false); - - parser - .parseArgument( - new String[] { - "--graphBasePath", testGraphBasePath, - "--inputPath", inputRelPath, - "--outputPath", testDedupGraphBasePath + "/relation" - }); - - new SparkCleanRelation(parser, spark).run(isLookUpService); - - Dataset df_after = spark - .read() - .schema(Encoders.bean(Relation.class).schema()) - .json(testDedupGraphBasePath + "/relation"); + .json(testConsistencyGraphBasePath + "/relation"); assertNotEquals(df_before.count(), df_after.count()); - assertEquals(0, df_after.count()); + + assertEquals( + 0, df_after + .filter("dataInfo.deletedbyinference == true OR dataInfo.invisible == true") + .count()); + + assertEquals( + 5, df_after + .filter("relClass NOT IN ('merges', 'isMergedIn')") + .count()); } @Test @@ -813,6 +796,7 @@ public class SparkDedupTest implements Serializable { public static void finalCleanUp() throws IOException { FileUtils.deleteDirectory(new File(testOutputBasePath)); FileUtils.deleteDirectory(new File(testDedupGraphBasePath)); + FileUtils.deleteDirectory(new File(testConsistencyGraphBasePath)); } public boolean isDeletedByInference(String s) { diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsProvisionTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsProvisionTest.java index a0bf6b37e..73e768cf1 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsProvisionTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsProvisionTest.java @@ -3,6 +3,7 @@ package eu.dnetlib.dhp.oa.dedup; import static java.nio.file.Files.createTempDirectory; +import static org.apache.spark.sql.functions.col; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.lenient; @@ -15,10 +16,6 @@ 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.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.PairFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; @@ -33,8 +30,6 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -import eu.dnetlib.pace.util.MapDocumentUtil; -import scala.Tuple2; @ExtendWith(MockitoExtension.class) @TestMethodOrder(MethodOrderer.OrderAnnotation.class) @@ -44,11 +39,11 @@ public class SparkOpenorgsProvisionTest implements Serializable { ISLookUpService isLookUpService; private static SparkSession spark; - private static JavaSparkContext jsc; private static String testGraphBasePath; private static String testOutputBasePath; private static String testDedupGraphBasePath; + private static String testConsistencyGraphBasePath; private static final String testActionSetId = "test-orchestrator"; @BeforeAll @@ -64,6 +59,9 @@ public class SparkOpenorgsProvisionTest implements Serializable { testDedupGraphBasePath = createTempDirectory(SparkOpenorgsProvisionTest.class.getSimpleName() + "-") .toAbsolutePath() .toString(); + testConsistencyGraphBasePath = createTempDirectory(SparkOpenorgsProvisionTest.class.getSimpleName() + "-") + .toAbsolutePath() + .toString(); FileUtils.deleteDirectory(new File(testOutputBasePath)); FileUtils.deleteDirectory(new File(testDedupGraphBasePath)); @@ -76,8 +74,13 @@ public class SparkOpenorgsProvisionTest implements Serializable { .master("local[*]") .config(conf) .getOrCreate(); + } - jsc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + @AfterAll + public static void finalCleanUp() throws IOException { + FileUtils.deleteDirectory(new File(testOutputBasePath)); + FileUtils.deleteDirectory(new File(testDedupGraphBasePath)); + FileUtils.deleteDirectory(new File(testConsistencyGraphBasePath)); } @BeforeEach @@ -186,26 +189,21 @@ public class SparkOpenorgsProvisionTest implements Serializable { new SparkUpdateEntity(parser, spark).run(isLookUpService); - long organizations = jsc.textFile(testDedupGraphBasePath + "/organization").count(); + Dataset organizations = spark.read().json(testDedupGraphBasePath + "/organization"); - long mergedOrgs = spark + Dataset mergedOrgs = spark .read() .load(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel") - .as(Encoders.bean(Relation.class)) .where("relClass=='merges'") - .javaRDD() - .map(Relation::getTarget) - .distinct() - .count(); + .select("target") + .distinct(); - assertEquals(80, organizations); + assertEquals(80, organizations.count()); - long deletedOrgs = jsc - .textFile(testDedupGraphBasePath + "/organization") - .filter(this::isDeletedByInference) - .count(); + Dataset deletedOrgs = organizations + .filter("dataInfo.deletedbyinference = TRUE"); - assertEquals(mergedOrgs, deletedOrgs); + assertEquals(mergedOrgs.count(), deletedOrgs.count()); } @Test @@ -226,10 +224,9 @@ public class SparkOpenorgsProvisionTest implements Serializable { new SparkCopyRelationsNoOpenorgs(parser, spark).run(isLookUpService); - final JavaRDD rels = jsc.textFile(testDedupGraphBasePath + "/relation"); - - assertEquals(2382, rels.count()); + final Dataset outputRels = spark.read().text(testDedupGraphBasePath + "/relation"); + assertEquals(2382, outputRels.count()); } @Test @@ -244,51 +241,41 @@ public class SparkOpenorgsProvisionTest implements Serializable { parser .parseArgument( new String[] { - "-i", testGraphBasePath, "-w", testOutputBasePath, "-o", testDedupGraphBasePath + "-i", testDedupGraphBasePath, "-w", testOutputBasePath, "-o", testConsistencyGraphBasePath }); new SparkPropagateRelation(parser, spark).run(isLookUpService); - long relations = jsc.textFile(testDedupGraphBasePath + "/relation").count(); - - assertEquals(4896, relations); - - // check deletedbyinference final Dataset mergeRels = spark .read() .load(DedupUtility.createMergeRelPath(testOutputBasePath, "*", "*")) .as(Encoders.bean(Relation.class)); - final JavaPairRDD mergedIds = mergeRels + + Dataset inputRels = spark + .read() + .json(testDedupGraphBasePath + "/relation"); + + Dataset outputRels = spark + .read() + .json(testConsistencyGraphBasePath + "/relation"); + + final Dataset mergedIds = mergeRels .where("relClass == 'merges'") - .select(mergeRels.col("target")) - .distinct() - .toJavaRDD() - .mapToPair( - (PairFunction) r -> new Tuple2(r.getString(0), "d")); + .select(col("target").as("id")) + .distinct(); - JavaRDD toCheck = jsc - .textFile(testDedupGraphBasePath + "/relation") - .mapToPair(json -> new Tuple2<>(MapDocumentUtil.getJPathString("$.source", json), json)) - .join(mergedIds) - .map(t -> t._2()._1()) - .mapToPair(json -> new Tuple2<>(MapDocumentUtil.getJPathString("$.target", json), json)) - .join(mergedIds) - .map(t -> t._2()._1()); + Dataset toUpdateRels = inputRels + .as("rel") + .join(mergedIds.as("s"), col("rel.source").equalTo(col("s.id")), "left_outer") + .join(mergedIds.as("t"), col("rel.target").equalTo(col("t.id")), "left_outer") + .filter("s.id IS NOT NULL OR t.id IS NOT NULL") + .distinct(); - long deletedbyinference = toCheck.filter(this::isDeletedByInference).count(); - long updated = toCheck.count(); + Dataset updatedRels = inputRels + .select("source", "target", "relClass") + .except(outputRels.select("source", "target", "relClass")); - assertEquals(updated, deletedbyinference); + assertEquals(toUpdateRels.count(), updatedRels.count()); + assertEquals(140, outputRels.count()); } - - @AfterAll - public static void finalCleanUp() throws IOException { - FileUtils.deleteDirectory(new File(testOutputBasePath)); - FileUtils.deleteDirectory(new File(testDedupGraphBasePath)); - } - - public boolean isDeletedByInference(String s) { - return s.contains("\"deletedbyinference\":true"); - } - } diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/crossref/irish_funder.json b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/crossref/irish_funder.json new file mode 100644 index 000000000..15eb1b711 --- /dev/null +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/crossref/irish_funder.json @@ -0,0 +1,940 @@ +[ + { + "id": "100007630", + "uri": "http://dx.doi.org/10.13039/100007630", + "name": "College of Engineering and Informatics, National University of Ireland, Galway", + "synonym": [] + }, + { + "id": "100007731", + "uri": "http://dx.doi.org/10.13039/100007731", + "name": "Endo International", + "synonym": [] + }, + { + "id": "100008099", + "uri": "http://dx.doi.org/10.13039/100008099", + "name": "Food Safety Authority of Ireland", + "synonym": [] + }, + { + "id": "100008124", + "uri": "http://dx.doi.org/10.13039/100008124", + "name": "Department of Jobs, Enterprise and Innovation", + "synonym": [] + }, + { + "id": "100009098", + "uri": "http://dx.doi.org/10.13039/100009098", + "name": "Department of Foreign Affairs and Trade, Ireland", + "synonym": [] + }, + { + "id": "100009099", + "uri": "http://dx.doi.org/10.13039/100009099", + "name": "Irish Aid", + "synonym": [] + }, + { + "id": "100009770", + "uri": "http://dx.doi.org/10.13039/100009770", + "name": "National University of Ireland", + "synonym": [] + }, + { + "id": "100009985", + "uri": "http://dx.doi.org/10.13039/100009985", + "name": "Parkinson's Association of Ireland", + "synonym": [] + }, + { + "id": "100010399", + "uri": "http://dx.doi.org/10.13039/100010399", + "name": "European Society of Cataract and Refractive Surgeons", + "synonym": [] + }, + { + "id": "100010414", + "uri": "http://dx.doi.org/10.13039/100010414", + "name": "Health Research Board", + "synonym": [ + "501100001590" + ] + }, + { + "id": "100010546", + "uri": "http://dx.doi.org/10.13039/100010546", + "name": "Deparment of Children and Youth Affairs, Ireland", + "synonym": [] + }, + { + "id": "100010993", + "uri": "http://dx.doi.org/10.13039/100010993", + "name": "Irish Nephrology Society", + "synonym": [] + }, + { + "id": "100011062", + "uri": "http://dx.doi.org/10.13039/100011062", + "name": "Asian Spinal Cord Network", + "synonym": [] + }, + { + "id": "100011096", + "uri": "http://dx.doi.org/10.13039/100011096", + "name": "Jazz Pharmaceuticals", + "synonym": [] + }, + { + "id": "100011396", + "uri": "http://dx.doi.org/10.13039/100011396", + "name": "Irish College of General Practitioners", + "synonym": [] + }, + { + "id": "100012734", + "uri": "http://dx.doi.org/10.13039/100012734", + "name": "Department for Culture, Heritage and the Gaeltacht, Ireland", + "synonym": [] + }, + { + "id": "100012754", + "uri": "http://dx.doi.org/10.13039/100012754", + "name": "Horizon Pharma", + "synonym": [] + }, + { + "id": "100012891", + "uri": "http://dx.doi.org/10.13039/100012891", + "name": "Medical Research Charities Group", + "synonym": [] + }, + { + "id": "100012919", + "uri": "http://dx.doi.org/10.13039/100012919", + "name": "Epilepsy Ireland", + "synonym": [] + }, + { + "id": "100012920", + "uri": "http://dx.doi.org/10.13039/100012920", + "name": "GLEN", + "synonym": [] + }, + { + "id": "100012921", + "uri": "http://dx.doi.org/10.13039/100012921", + "name": "Royal College of Surgeons in Ireland", + "synonym": [] + }, + { + "id": "100013029", + "uri": "http://dx.doi.org/10.13039/100013029", + "name": "Iris O'Brien Foundation", + "synonym": [] + }, + { + "id": "100013206", + "uri": "http://dx.doi.org/10.13039/100013206", + "name": "Food Institutional Research Measure", + "synonym": [] + }, + { + "id": "100013381", + "uri": "http://dx.doi.org/10.13039/100013381", + "name": "Irish Phytochemical Food Network", + "synonym": [] + }, + { + "id": "100013433", + "uri": "http://dx.doi.org/10.13039/100013433", + "name": "Transport Infrastructure Ireland", + "synonym": [] + }, + { + "id": "100013461", + "uri": "http://dx.doi.org/10.13039/100013461", + "name": "Arts and Disability Ireland", + "synonym": [] + }, + { + "id": "100013548", + "uri": "http://dx.doi.org/10.13039/100013548", + "name": "Filmbase", + "synonym": [] + }, + { + "id": "100013917", + "uri": "http://dx.doi.org/10.13039/100013917", + "name": "Society for Musicology in Ireland", + "synonym": [] + }, + { + "id": "100014251", + "uri": "http://dx.doi.org/10.13039/100014251", + "name": "Humanities in the European Research Area", + "synonym": [] + }, + { + "id": "100014364", + "uri": "http://dx.doi.org/10.13039/100014364", + "name": "National Children's Research Centre", + "synonym": [] + }, + { + "id": "100014384", + "uri": "http://dx.doi.org/10.13039/100014384", + "name": "Amarin Corporation", + "synonym": [] + }, + { + "id": "100014902", + "uri": "http://dx.doi.org/10.13039/100014902", + "name": "Irish Association for Cancer Research", + "synonym": [] + }, + { + "id": "100015023", + "uri": "http://dx.doi.org/10.13039/100015023", + "name": "Ireland Funds", + "synonym": [] + }, + { + "id": "100015037", + "uri": "http://dx.doi.org/10.13039/100015037", + "name": "Simon Cumbers Media Fund", + "synonym": [] + }, + { + "id": "100015319", + "uri": "http://dx.doi.org/10.13039/100015319", + "name": "Sport Ireland Institute", + "synonym": [] + }, + { + "id": "100015320", + "uri": "http://dx.doi.org/10.13039/100015320", + "name": "Paralympics Ireland", + "synonym": [] + }, + { + "id": "100015442", + "uri": "http://dx.doi.org/10.13039/100015442", + "name": "Global Brain Health Institute", + "synonym": [] + }, + { + "id": "100015776", + "uri": "http://dx.doi.org/10.13039/100015776", + "name": "Health and Social Care Board", + "synonym": [] + }, + { + "id": "100015992", + "uri": "http://dx.doi.org/10.13039/100015992", + "name": "St. Luke's Institute of Cancer Research", + "synonym": [] + }, + { + "id": "100017897", + "uri": "http://dx.doi.org/10.13039/100017897", + "name": "Friedreich\u2019s Ataxia Research Alliance Ireland", + "synonym": [] + }, + { + "id": "100018064", + "uri": "http://dx.doi.org/10.13039/100018064", + "name": "Department of Tourism, Culture, Arts, Gaeltacht, Sport and Media", + "synonym": [] + }, + { + "id": "100018172", + "uri": "http://dx.doi.org/10.13039/100018172", + "name": "Department of the Environment, Climate and Communications", + "synonym": [] + }, + { + "id": "100018175", + "uri": "http://dx.doi.org/10.13039/100018175", + "name": "Dairy Processing Technology Centre", + "synonym": [] + }, + { + "id": "100018270", + "uri": "http://dx.doi.org/10.13039/100018270", + "name": "Health Service Executive", + "synonym": [] + }, + { + "id": "100018529", + "uri": "http://dx.doi.org/10.13039/100018529", + "name": "Alkermes", + "synonym": [] + }, + { + "id": "100018542", + "uri": "http://dx.doi.org/10.13039/100018542", + "name": "Irish Endocrine Society", + "synonym": [] + }, + { + "id": "100018754", + "uri": "http://dx.doi.org/10.13039/100018754", + "name": "An Roinn Sl\u00e1inte", + "synonym": [] + }, + { + "id": "100018998", + "uri": "http://dx.doi.org/10.13039/100018998", + "name": "Irish Research eLibrary", + "synonym": [] + }, + { + "id": "100019428", + "uri": "http://dx.doi.org/10.13039/100019428", + "name": "Nabriva Therapeutics", + "synonym": [] + }, + { + "id": "100019637", + "uri": "http://dx.doi.org/10.13039/100019637", + "name": "Horizon Therapeutics", + "synonym": [] + }, + { + "id": "100020174", + "uri": "http://dx.doi.org/10.13039/100020174", + "name": "Health Research Charities Ireland", + "synonym": [] + }, + { + "id": "100020202", + "uri": "http://dx.doi.org/10.13039/100020202", + "name": "UCD Foundation", + "synonym": [] + }, + { + "id": "100020233", + "uri": "http://dx.doi.org/10.13039/100020233", + "name": "Ireland Canada University Foundation", + "synonym": [] + }, + { + "id": "100022943", + "uri": "http://dx.doi.org/10.13039/100022943", + "name": "National Cancer Registry Ireland", + "synonym": [] + }, + { + "id": "501100001581", + "uri": "http://dx.doi.org/10.13039/501100001581", + "name": "Arts Council of Ireland", + "synonym": [] + }, + { + "id": "501100001582", + "uri": "http://dx.doi.org/10.13039/501100001582", + "name": "Centre for Ageing Research and Development in Ireland", + "synonym": [] + }, + { + "id": "501100001583", + "uri": "http://dx.doi.org/10.13039/501100001583", + "name": "Cystinosis Foundation Ireland", + "synonym": [] + }, + { + "id": "501100001584", + "uri": "http://dx.doi.org/10.13039/501100001584", + "name": "Department of Agriculture, Food and the Marine, Ireland", + "synonym": [] + }, + { + "id": "501100001586", + "uri": "http://dx.doi.org/10.13039/501100001586", + "name": "Department of Education and Skills, Ireland", + "synonym": [] + }, + { + "id": "501100001587", + "uri": "http://dx.doi.org/10.13039/501100001587", + "name": "Economic and Social Research Institute", + "synonym": [] + }, + { + "id": "501100001588", + "uri": "http://dx.doi.org/10.13039/501100001588", + "name": "Enterprise Ireland", + "synonym": [] + }, + { + "id": "501100001589", + "uri": "http://dx.doi.org/10.13039/501100001589", + "name": "Environmental Protection Agency", + "synonym": [] + }, + { + "id": "501100001591", + "uri": "http://dx.doi.org/10.13039/501100001591", + "name": "Heritage Council", + "synonym": [] + }, + { + "id": "501100001592", + "uri": "http://dx.doi.org/10.13039/501100001592", + "name": "Higher Education Authority", + "synonym": [] + }, + { + "id": "501100001593", + "uri": "http://dx.doi.org/10.13039/501100001593", + "name": "Irish Cancer Society", + "synonym": [] + }, + { + "id": "501100001594", + "uri": "http://dx.doi.org/10.13039/501100001594", + "name": "Irish Heart Foundation", + "synonym": [] + }, + { + "id": "501100001595", + "uri": "http://dx.doi.org/10.13039/501100001595", + "name": "Irish Hospice Foundation", + "synonym": [] + }, + { + "id": "501100001596", + "uri": "http://dx.doi.org/10.13039/501100001596", + "name": "Irish Research Council for Science, Engineering and Technology", + "synonym": [] + }, + { + "id": "501100001597", + "uri": "http://dx.doi.org/10.13039/501100001597", + "name": "Irish Research Council for the Humanities and Social Sciences", + "synonym": [] + }, + { + "id": "501100001598", + "uri": "http://dx.doi.org/10.13039/501100001598", + "name": "Mental Health Commission", + "synonym": [] + }, + { + "id": "501100001600", + "uri": "http://dx.doi.org/10.13039/501100001600", + "name": "Research and Education Foundation, Sligo General Hospital", + "synonym": [] + }, + { + "id": "501100001601", + "uri": "http://dx.doi.org/10.13039/501100001601", + "name": "Royal Irish Academy", + "synonym": [] + }, + { + "id": "501100001603", + "uri": "http://dx.doi.org/10.13039/501100001603", + "name": "Sustainable Energy Authority of Ireland", + "synonym": [] + }, + { + "id": "501100001604", + "uri": "http://dx.doi.org/10.13039/501100001604", + "name": "Teagasc", + "synonym": [] + }, + { + "id": "501100001627", + "uri": "http://dx.doi.org/10.13039/501100001627", + "name": "Marine Institute", + "synonym": [] + }, + { + "id": "501100001628", + "uri": "http://dx.doi.org/10.13039/501100001628", + "name": "Central Remedial Clinic", + "synonym": [] + }, + { + "id": "501100001629", + "uri": "http://dx.doi.org/10.13039/501100001629", + "name": "Royal Dublin Society", + "synonym": [] + }, + { + "id": "501100001630", + "uri": "http://dx.doi.org/10.13039/501100001630", + "name": "Dublin Institute for Advanced Studies", + "synonym": [] + }, + { + "id": "501100001631", + "uri": "http://dx.doi.org/10.13039/501100001631", + "name": "University College Dublin", + "synonym": [] + }, + { + "id": "501100001633", + "uri": "http://dx.doi.org/10.13039/501100001633", + "name": "National University of Ireland, Maynooth", + "synonym": [] + }, + { + "id": "501100001634", + "uri": "http://dx.doi.org/10.13039/501100001634", + "name": "University of Galway", + "synonym": [] + }, + { + "id": "501100001635", + "uri": "http://dx.doi.org/10.13039/501100001635", + "name": "University of Limerick", + "synonym": [] + }, + { + "id": "501100001636", + "uri": "http://dx.doi.org/10.13039/501100001636", + "name": "University College Cork", + "synonym": [] + }, + { + "id": "501100001637", + "uri": "http://dx.doi.org/10.13039/501100001637", + "name": "Trinity College Dublin", + "synonym": [] + }, + { + "id": "501100001638", + "uri": "http://dx.doi.org/10.13039/501100001638", + "name": "Dublin City University", + "synonym": [] + }, + { + "id": "501100002081", + "uri": "http://dx.doi.org/10.13039/501100002081", + "name": "Irish Research Council", + "synonym": [] + }, + { + "id": "501100002736", + "uri": "http://dx.doi.org/10.13039/501100002736", + "name": "Covidien", + "synonym": [] + }, + { + "id": "501100002755", + "uri": "http://dx.doi.org/10.13039/501100002755", + "name": "Brennan and Company", + "synonym": [] + }, + { + "id": "501100002919", + "uri": "http://dx.doi.org/10.13039/501100002919", + "name": "Cork Institute of Technology", + "synonym": [] + }, + { + "id": "501100002959", + "uri": "http://dx.doi.org/10.13039/501100002959", + "name": "Dublin City Council", + "synonym": [] + }, + { + "id": "501100003036", + "uri": "http://dx.doi.org/10.13039/501100003036", + "name": "Perrigo Company Charitable Foundation", + "synonym": [] + }, + { + "id": "501100003037", + "uri": "http://dx.doi.org/10.13039/501100003037", + "name": "Elan", + "synonym": [] + }, + { + "id": "501100003496", + "uri": "http://dx.doi.org/10.13039/501100003496", + "name": "HeyStaks Technologies", + "synonym": [] + }, + { + "id": "501100003553", + "uri": "http://dx.doi.org/10.13039/501100003553", + "name": "Gaelic Athletic Association", + "synonym": [] + }, + { + "id": "501100003840", + "uri": "http://dx.doi.org/10.13039/501100003840", + "name": "Irish Institute of Clinical Neuroscience", + "synonym": [] + }, + { + "id": "501100003956", + "uri": "http://dx.doi.org/10.13039/501100003956", + "name": "Aspect Medical Systems", + "synonym": [] + }, + { + "id": "501100004162", + "uri": "http://dx.doi.org/10.13039/501100004162", + "name": "Meath Foundation", + "synonym": [] + }, + { + "id": "501100004210", + "uri": "http://dx.doi.org/10.13039/501100004210", + "name": "Our Lady's Children's Hospital, Crumlin", + "synonym": [] + }, + { + "id": "501100004321", + "uri": "http://dx.doi.org/10.13039/501100004321", + "name": "Shire", + "synonym": [] + }, + { + "id": "501100004981", + "uri": "http://dx.doi.org/10.13039/501100004981", + "name": "Athlone Institute of Technology", + "synonym": [] + }, + { + "id": "501100006518", + "uri": "http://dx.doi.org/10.13039/501100006518", + "name": "Department of Communications, Energy and Natural Resources, Ireland", + "synonym": [] + }, + { + "id": "501100006553", + "uri": "http://dx.doi.org/10.13039/501100006553", + "name": "Collaborative Centre for Applied Nanotechnology", + "synonym": [] + }, + { + "id": "501100006759", + "uri": "http://dx.doi.org/10.13039/501100006759", + "name": "CLARITY Centre for Sensor Web Technologies", + "synonym": [] + }, + { + "id": "501100009246", + "uri": "http://dx.doi.org/10.13039/501100009246", + "name": "Technological University Dublin", + "synonym": [] + }, + { + "id": "501100009269", + "uri": "http://dx.doi.org/10.13039/501100009269", + "name": "Programme of Competitive Forestry Research for Development", + "synonym": [] + }, + { + "id": "501100009315", + "uri": "http://dx.doi.org/10.13039/501100009315", + "name": "Cystinosis Ireland", + "synonym": [] + }, + { + "id": "501100010808", + "uri": "http://dx.doi.org/10.13039/501100010808", + "name": "Geological Survey of Ireland", + "synonym": [] + }, + { + "id": "501100011030", + "uri": "http://dx.doi.org/10.13039/501100011030", + "name": "Alimentary Glycoscience Research Cluster", + "synonym": [] + }, + { + "id": "501100011031", + "uri": "http://dx.doi.org/10.13039/501100011031", + "name": "Alimentary Health", + "synonym": [] + }, + { + "id": "501100011103", + "uri": "http://dx.doi.org/10.13039/501100011103", + "name": "Rann\u00eds", + "synonym": [] + }, + { + "id": "501100012354", + "uri": "http://dx.doi.org/10.13039/501100012354", + "name": "Inland Fisheries Ireland", + "synonym": [] + }, + { + "id": "501100014384", + "uri": "http://dx.doi.org/10.13039/501100014384", + "name": "X-Bolt Orthopaedics", + "synonym": [] + }, + { + "id": "501100014710", + "uri": "http://dx.doi.org/10.13039/501100014710", + "name": "PrecisionBiotics Group", + "synonym": [] + }, + { + "id": "501100014827", + "uri": "http://dx.doi.org/10.13039/501100014827", + "name": "Dormant Accounts Fund", + "synonym": [] + }, + { + "id": "501100016041", + "uri": "http://dx.doi.org/10.13039/501100016041", + "name": "St Vincents Anaesthesia Foundation", + "synonym": [] + }, + { + "id": "501100017501", + "uri": "http://dx.doi.org/10.13039/501100017501", + "name": "FotoNation", + "synonym": [] + }, + { + "id": "501100018641", + "uri": "http://dx.doi.org/10.13039/501100018641", + "name": "Dairy Research Ireland", + "synonym": [] + }, + { + "id": "501100018839", + "uri": "http://dx.doi.org/10.13039/501100018839", + "name": "Irish Centre for High-End Computing", + "synonym": [] + }, + { + "id": "501100019905", + "uri": "http://dx.doi.org/10.13039/501100019905", + "name": "Galway University Foundation", + "synonym": [] + }, + { + "id": "501100020036", + "uri": "http://dx.doi.org/10.13039/501100020036", + "name": "Dystonia Ireland", + "synonym": [] + }, + { + "id": "501100020221", + "uri": "http://dx.doi.org/10.13039/501100020221", + "name": "Irish Motor Neurone Disease Association", + "synonym": [] + }, + { + "id": "501100020270", + "uri": "http://dx.doi.org/10.13039/501100020270", + "name": "Advanced Materials and Bioengineering Research", + "synonym": [] + }, + { + "id": "501100020403", + "uri": "http://dx.doi.org/10.13039/501100020403", + "name": "Irish Composites Centre", + "synonym": [] + }, + { + "id": "501100020425", + "uri": "http://dx.doi.org/10.13039/501100020425", + "name": "Irish Thoracic Society", + "synonym": [] + }, + { + "id": "501100021102", + "uri": "http://dx.doi.org/10.13039/501100021102", + "name": "Waterford Institute of Technology", + "synonym": [] + }, + { + "id": "501100021110", + "uri": "http://dx.doi.org/10.13039/501100021110", + "name": "Irish MPS Society", + "synonym": [] + }, + { + "id": "501100021525", + "uri": "http://dx.doi.org/10.13039/501100021525", + "name": "Insight SFI Research Centre for Data Analytics", + "synonym": [] + }, + { + "id": "501100021694", + "uri": "http://dx.doi.org/10.13039/501100021694", + "name": "Elan Pharma International", + "synonym": [] + }, + { + "id": "501100021838", + "uri": "http://dx.doi.org/10.13039/501100021838", + "name": "Royal College of Physicians of Ireland", + "synonym": [] + }, + { + "id": "501100022542", + "uri": "http://dx.doi.org/10.13039/501100022542", + "name": "Breakthrough Cancer Research", + "synonym": [] + }, + { + "id": "501100022610", + "uri": "http://dx.doi.org/10.13039/501100022610", + "name": "Breast Cancer Ireland", + "synonym": [] + }, + { + "id": "501100022728", + "uri": "http://dx.doi.org/10.13039/501100022728", + "name": "Munster Technological University", + "synonym": [] + }, + { + "id": "501100022729", + "uri": "http://dx.doi.org/10.13039/501100022729", + "name": "Institute of Technology, Tralee", + "synonym": [] + }, + { + "id": "501100023273", + "uri": "http://dx.doi.org/10.13039/501100023273", + "name": "HRB Clinical Research Facility Galway", + "synonym": [] + }, + { + "id": "501100023378", + "uri": "http://dx.doi.org/10.13039/501100023378", + "name": "Lauritzson Foundation", + "synonym": [] + }, + { + "id": "501100023551", + "uri": "http://dx.doi.org/10.13039/501100023551", + "name": "Cystic Fibrosis Ireland", + "synonym": [] + }, + { + "id": "501100023970", + "uri": "http://dx.doi.org/10.13039/501100023970", + "name": "Tyndall National Institute", + "synonym": [] + }, + { + "id": "501100024094", + "uri": "http://dx.doi.org/10.13039/501100024094", + "name": "Raidi\u00f3 Teilif\u00eds \u00c9ireann", + "synonym": [] + }, + { + "id": "501100024242", + "uri": "http://dx.doi.org/10.13039/501100024242", + "name": "Synthesis and Solid State Pharmaceutical Centre", + "synonym": [] + }, + { + "id": "501100024313", + "uri": "http://dx.doi.org/10.13039/501100024313", + "name": "Irish Rugby Football Union", + "synonym": [] + }, + { + "id": "100007490", + "uri": "http://dx.doi.org/10.13039/100007490", + "name": "Bausch and Lomb Ireland", + "synonym": [] + }, + { + "id": "100007819", + "uri": "http://dx.doi.org/10.13039/100007819", + "name": "Allergan", + "synonym": [] + }, + { + "id": "100010547", + "uri": "http://dx.doi.org/10.13039/100010547", + "name": "Irish Youth Justice Service", + "synonym": [] + }, + { + "id": "100012733", + "uri": "http://dx.doi.org/10.13039/100012733", + "name": "National Parks and Wildlife Service", + "synonym": [] + }, + { + "id": "100015278", + "uri": "http://dx.doi.org/10.13039/100015278", + "name": "Pfizer Healthcare Ireland", + "synonym": [] + }, + { + "id": "100017144", + "uri": "http://dx.doi.org/10.13039/100017144", + "name": "Shell E and P Ireland", + "synonym": [] + }, + { + "id": "100022895", + "uri": "http://dx.doi.org/10.13039/100022895", + "name": "Health Research Institute, University of Limerick", + "synonym": [] + }, + { + "id": "501100001599", + "uri": "http://dx.doi.org/10.13039/501100001599", + "name": "National Council for Forest Research and Development", + "synonym": [] + }, + { + "id": "501100006554", + "uri": "http://dx.doi.org/10.13039/501100006554", + "name": "IDA Ireland", + "synonym": [] + }, + { + "id": "501100011626", + "uri": "http://dx.doi.org/10.13039/501100011626", + "name": "Energy Policy Research Centre, Economic and Social Research Institute", + "synonym": [] + }, + { + "id": "501100014531", + "uri": "http://dx.doi.org/10.13039/501100014531", + "name": "Physical Education and Sport Sciences Department, University of Limerick", + "synonym": [] + }, + { + "id": "501100014745", + "uri": "http://dx.doi.org/10.13039/501100014745", + "name": "APC Microbiome Institute", + "synonym": [] + }, + { + "id": "501100014826", + "uri": "http://dx.doi.org/10.13039/501100014826", + "name": "ADAPT - Centre for Digital Content Technology", + "synonym": [] + }, + { + "id": "501100020570", + "uri": "http://dx.doi.org/10.13039/501100020570", + "name": "College of Medicine, Nursing and Health Sciences, National University of Ireland, Galway", + "synonym": [] + }, + { + "id": "501100020871", + "uri": "http://dx.doi.org/10.13039/501100020871", + "name": "Bernal Institute, University of Limerick", + "synonym": [] + }, + { + "id": "501100023852", + "uri": "http://dx.doi.org/10.13039/501100023852", + "name": "Moore Institute for Research in the Humanities and Social Studies, University of Galway", + "synonym": [] + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala index 9c63b709b..565d34e62 100644 --- a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala @@ -16,6 +16,7 @@ import org.slf4j.{Logger, LoggerFactory} import java.util import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.io.Source import scala.util.matching.Regex case class CrossrefDT(doi: String, json: String, timestamp: Long) {} @@ -30,11 +31,22 @@ case class mappingAuthor( affiliation: Option[mappingAffiliation] ) {} +case class funderInfo(id: String, uri: String, name: String, synonym: List[String]) {} + case class mappingFunder(name: String, DOI: Option[String], award: Option[List[String]]) {} case object Crossref2Oaf { val logger: Logger = LoggerFactory.getLogger(Crossref2Oaf.getClass) + val irishFunder: List[funderInfo] = { + val s = Source + .fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/crossref/irish_funder.json")) + .mkString + implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats + lazy val json: org.json4s.JValue = parse(s) + json.extract[List[funderInfo]] + } + val mappingCrossrefType = Map( "book-section" -> "publication", "book" -> "publication", @@ -88,6 +100,13 @@ case object Crossref2Oaf { "report" -> "0017 Report" ) + def getIrishId(doi: String): Option[String] = { + val id = doi.split("/").last + irishFunder + .find(f => id.equalsIgnoreCase(f.id) || (f.synonym.nonEmpty && f.synonym.exists(s => s.equalsIgnoreCase(id)))) + .map(f => f.id) + } + def mappingResult(result: Result, json: JValue, cobjCategory: String): Result = { implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats @@ -467,6 +486,14 @@ case object Crossref2Oaf { if (funders != null) funders.foreach(funder => { if (funder.DOI.isDefined && funder.DOI.get.nonEmpty) { + + if (getIrishId(funder.DOI.get).isDefined) { + val nsPrefix = getIrishId(funder.DOI.get).get.padTo(12, '_') + val targetId = getProjectId(nsPrefix, "1e5e62235d094afd01cd56e65112fc63") + queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY) + queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES) + } + funder.DOI.get match { case "10.13039/100010663" | "10.13039/100010661" | "10.13039/501100007601" | "10.13039/501100000780" | "10.13039/100010665" => diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java index 3186ed5c0..fc3882b73 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java @@ -25,6 +25,7 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.bulktag.community.*; import eu.dnetlib.dhp.schema.oaf.Datasource; import eu.dnetlib.dhp.schema.oaf.Result; +import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; public class SparkBulkTagJob { @@ -170,10 +171,12 @@ public class SparkBulkTagJob { // TODO remove this hack as soon as the values fixed by this method will be provided as NON null private static MapFunction patchResult() { return r -> { - if (r.getDataInfo().getDeletedbyinference() == null) { + if (Objects.isNull(r.getDataInfo())) { + r.setDataInfo(OafMapperUtils.dataInfo(false, "", false, false, OafMapperUtils.unknown("", ""), "")); + } else if (r.getDataInfo().getDeletedbyinference() == null) { r.getDataInfo().setDeletedbyinference(false); } - if (r.getContext() == null) { + if (Objects.isNull(r.getContext())) { r.setContext(new ArrayList<>()); } return r; diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java index 28b6f616d..184d24751 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/countrypropagation/PrepareResultCountrySet.java @@ -5,10 +5,7 @@ import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.Set; +import java.util.*; import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; @@ -88,14 +85,33 @@ public class PrepareResultCountrySet { // selects all the results non deleted by inference and non invisible Dataset result = readPath(spark, inputPath, resultClazz) .filter( - (FilterFunction) r -> !r.getDataInfo().getDeletedbyinference() && - !r.getDataInfo().getInvisible()); + (FilterFunction) r -> Optional + .ofNullable(r.getDataInfo()) + .map(dataInfo -> !dataInfo.getDeletedbyinference() && !dataInfo.getInvisible()) + .orElse(true)); // of the results collects the distinct keys for collected from (at the level of the result) and hosted by // and produces pairs resultId, key for each distinct key associated to the result result.flatMap((FlatMapFunction) r -> { - Set cfhb = r.getCollectedfrom().stream().map(cf -> cf.getKey()).collect(Collectors.toSet()); - cfhb.addAll(r.getInstance().stream().map(i -> i.getHostedby().getKey()).collect(Collectors.toSet())); + Set cfhb = Optional + .ofNullable(r.getCollectedfrom()) + .map(cf -> cf.stream().map(KeyValue::getKey).collect(Collectors.toSet())) + .orElse(new HashSet<>()); + cfhb + .addAll( + Optional + .ofNullable(r.getInstance()) + .map( + i -> i + .stream() + .map( + ii -> Optional + .ofNullable(ii.getHostedby()) + .map(KeyValue::getKey) + .orElse(null)) + .filter(Objects::nonNull) + .collect(Collectors.toSet())) + .orElse(new HashSet<>())); return cfhb .stream() .map(value -> EntityEntityRel.newInstance(r.getId(), value)) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/group/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/group/oozie_app/workflow.xml index 219dc7331..190788c9d 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/group/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/group/oozie_app/workflow.xml @@ -96,30 +96,7 @@ --conf spark.sql.shuffle.partitions=15000 --graphInputPath${graphBasePath} - --outputPath${workingPath}/grouped_entities - - - - - - - - yarn - cluster - Dispatch grouped entities - eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob - 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 - - --inputPath${workingPath}/grouped_entities + --checkpointPath${workingPath}/grouped_entities --outputPath${graphOutputPath} --filterInvisible${filterInvisible} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/resolution/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/resolution/oozie_app/config-default.xml index 6fb2a1253..86847ed46 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/resolution/oozie_app/config-default.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/resolution/oozie_app/config-default.xml @@ -1,4 +1,12 @@ + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + oozie.use.system.libpath true @@ -7,4 +15,28 @@ oozie.action.sharelib.for.spark spark2 + + hive_metastore_uris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + + spark2YarnHistoryServerAddress + http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089 + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + + + sparkExecutorNumber + 4 + + + spark2EventLogDir + /user/spark/spark2ApplicationHistory + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/group/GroupEntitiesSparkJobTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/group/GroupEntitiesSparkJobTest.java index 61baf80dc..0887adf45 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/group/GroupEntitiesSparkJobTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/group/GroupEntitiesSparkJobTest.java @@ -10,7 +10,6 @@ import java.nio.file.Path; import java.nio.file.Paths; import org.apache.commons.io.FileUtils; -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; @@ -23,9 +22,9 @@ import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob; import eu.dnetlib.dhp.oa.merge.GroupEntitiesSparkJob; import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.OafEntity; import eu.dnetlib.dhp.schema.oaf.Result; import eu.dnetlib.dhp.utils.DHPUtils; @@ -40,8 +39,9 @@ public class GroupEntitiesSparkJobTest { private static Path workingDir; private Path dataInputPath; - private Path groupEntityPath; - private Path dispatchEntityPath; + private Path checkpointPath; + + private Path outputPath; @BeforeAll public static void beforeAll() throws IOException { @@ -58,8 +58,8 @@ public class GroupEntitiesSparkJobTest { @BeforeEach public void beforeEach() throws IOException, URISyntaxException { dataInputPath = Paths.get(ClassLoader.getSystemResource("eu/dnetlib/dhp/oa/graph/group").toURI()); - groupEntityPath = workingDir.resolve("grouped_entity"); - dispatchEntityPath = workingDir.resolve("dispatched_entity"); + checkpointPath = workingDir.resolve("grouped_entity"); + outputPath = workingDir.resolve("dispatched_entity"); } @AfterAll @@ -76,39 +76,28 @@ public class GroupEntitiesSparkJobTest { Boolean.FALSE.toString(), "-graphInputPath", dataInputPath.toString(), + "-checkpointPath", + checkpointPath.toString(), "-outputPath", - groupEntityPath.toString() + outputPath.toString(), + "-filterInvisible", + Boolean.FALSE.toString() }); - Dataset output = spark + Dataset checkpointTable = spark .read() - .textFile(groupEntityPath.toString()) - .map((MapFunction) s -> StringUtils.substringAfter(s, "|"), Encoders.STRING()) - .map((MapFunction) s -> mapper.readValue(s, Result.class), Encoders.bean(Result.class)); + .load(checkpointPath.toString()) + .selectExpr("COALESCE(*)") + .as(Encoders.kryo(OafEntity.class)); assertEquals( 1, - output + checkpointTable .filter( - (FilterFunction) r -> "50|doi_________::09821844208a5cd6300b2bfb13bca1b9" + (FilterFunction) r -> "50|doi_________::09821844208a5cd6300b2bfb13bca1b9" .equals(r.getId()) && r.getCollectedfrom().stream().anyMatch(kv -> kv.getValue().equalsIgnoreCase("zenodo"))) .count()); - } - - @Test - @Order(2) - void testDispatchEntities() throws Exception { - DispatchEntitiesSparkJob.main(new String[] { - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-inputPath", - groupEntityPath.toString(), - "-outputPath", - dispatchEntityPath.resolve(".").toString(), - "-filterInvisible", - Boolean.TRUE.toString() - }); Dataset output = spark .read() @@ -116,7 +105,7 @@ public class GroupEntitiesSparkJobTest { DHPUtils .toSeq( HdfsSupport - .listFiles(dispatchEntityPath.toString(), spark.sparkContext().hadoopConfiguration()))) + .listFiles(outputPath.toString(), spark.sparkContext().hadoopConfiguration()))) .map((MapFunction) s -> mapper.readValue(s, Result.class), Encoders.bean(Result.class)); assertEquals(3, output.count()); diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh index 87294f6e9..431978997 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh @@ -24,13 +24,13 @@ function copydb() { # drop tables from db for i in `impala-shell --user $HADOOP_USER_NAME -i impala-cluster-dn1.openaire.eu -d ${db} --delimited -q "show tables"`; do - `impala-shell -i impala-cluster-dn1.openaire.eu -d -d ${db} -q "drop table $i;"`; + `impala-shell -i impala-cluster-dn1.openaire.eu -d ${db} -q "drop table $i;"`; done # drop views from db for i in `impala-shell --user $HADOOP_USER_NAME -i impala-cluster-dn1.openaire.eu -d ${db} --delimited -q "show tables"`; do - `impala-shell -i impala-cluster-dn1.openaire.eu -d -d ${db} -q "drop view $i;"`; + `impala-shell -i impala-cluster-dn1.openaire.eu -d ${db} -q "drop view $i;"`; done # delete the database @@ -82,12 +82,12 @@ copydb $USAGE_STATS_DB copydb $PROD_USAGE_STATS_DB copydb $EXT_DB copydb $STATS_DB -#copydb $MONITOR_DB +copydb $MONITOR_DB copydb $OBSERVATORY_DB copydb $MONITOR_DB'_funded' copydb $MONITOR_DB'_institutions' -copydb $MONITOR_DB'_RIs_tail' +copydb $MONITOR_DB'_ris_tail' contexts="knowmad::other dh-ch::other enermaps::other gotriple::other neanias-atmospheric::other rural-digital-europe::other covid-19::other aurora::other neanias-space::other north-america-studies::other north-american-studies::other eutopia::other" for i in ${contexts} diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/finalizeImpalaCluster.sh b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/finalizeImpalaCluster.sh index 857635b6c..86a93216c 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/finalizeImpalaCluster.sh +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/finalizeImpalaCluster.sh @@ -13,7 +13,7 @@ function createShadowDB() { # drop views from db for i in `impala-shell -i impala-cluster-dn1.openaire.eu -d ${SHADOW} --delimited -q "show tables"`; do - `impala-shell -i impala-cluster-dn1.openaire.eu -d -d ${SHADOW} -q "drop view $i;"`; + `impala-shell -i impala-cluster-dn1.openaire.eu -d ${SHADOW} -q "drop view $i;"`; done impala-shell -i impala-cluster-dn1.openaire.eu -q "drop database ${SHADOW} CASCADE"; @@ -36,13 +36,13 @@ createShadowDB $MONITOR_DB $MONITOR_DB_SHADOW createShadowDB $OBSERVATORY_DB $OBSERVATORY_DB_SHADOW createShadowDB USAGE_STATS_DB USAGE_STATS_DB_SHADOW -createShadowDB $MONITOR_DB'_funded' $MONITOR_DB'_funded_shadow' -createShadowDB $MONITOR_DB'_institutions' $MONITOR_DB'_institutions_shadow' -createShadowDB $MONITOR_DB'_RIs_tail' $MONITOR_DB'_RIs_tail_shadow' +createShadowDB $MONITOR_DB'_funded' $MONITOR_DB_SHADOW'_shadow_funded' +createShadowDB $MONITOR_DB'_institutions' $MONITOR_DB_SHADOW'_shadow_institutions' +createShadowDB $MONITOR_DB'_ris_tail' $MONITOR_DB_SHADOW'_shadow_ris_tail' contexts="knowmad::other dh-ch::other enermaps::other gotriple::other neanias-atmospheric::other rural-digital-europe::other covid-19::other aurora::other neanias-space::other north-america-studies::other north-american-studies::other eutopia::other" for i in ${contexts} do tmp=`echo "$i" | sed 's/'-'/'_'/g' | sed 's/'::'/'_'/g'` - createShadowDB ${MONITOR_DB}'_'${tmp} ${MONITOR_DB}'_'${tmp}'_shadow' + createShadowDB ${MONITOR_DB}'_'${tmp} ${MONITOR_DB_SHADOW}'_shadow_'${tmp} done \ 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/monitor.sh b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/monitor.sh old mode 100644 new mode 100755 index 08f4c9232..872456973 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/monitor.sh +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/monitor.sh @@ -14,6 +14,7 @@ export SCRIPT_PATH2=$5 export SCRIPT_PATH3=$6 export SCRIPT_PATH4=$7 export SCRIPT_PATH5=$8 +export SCRIPT_PATH6=$9 export HIVE_OPTS="-hiveconf mapred.job.queue.name=analytics -hiveconf hive.spark.client.connect.timeout=120000ms -hiveconf hive.spark.client.server.connect.timeout=300000ms -hiveconf spark.executor.memory=19166291558 -hiveconf spark.yarn.executor.memoryOverhead=3225 -hiveconf spark.driver.memory=11596411699 -hiveconf spark.yarn.driver.memoryOverhead=1228" export HADOOP_USER_NAME="oozie" @@ -33,12 +34,19 @@ hdfs dfs -copyToLocal $7 echo "Getting file from " $8 hdfs dfs -copyToLocal $8 +echo "Getting file from " $9 +hdfs dfs -copyToLocal $9 + + echo "Creating monitor database" +cat step20-createMonitorDBAll.sql | sed "s/SOURCE/$1/g" | sed "s/TARGET/$2/g1" > foo +hive $HIVE_OPTS -f foo + cat step20-createMonitorDB_funded.sql | sed "s/SOURCE/$1/g" | sed "s/TARGET/$2_funded/g1" > foo hive $HIVE_OPTS -f foo cat step20-createMonitorDB.sql | sed "s/SOURCE/$1/g" | sed "s/TARGET/$2_funded/g1" > foo hive $HIVE_OPTS -f foo -# + cat step20-createMonitorDB_institutions.sql | sed "s/SOURCE/$1/g" | sed "s/TARGET/$2_institutions/g1" > foo hive $HIVE_OPTS -f foo cat step20-createMonitorDB.sql | sed "s/SOURCE/$1/g" | sed "s/TARGET/$2_institutions/g1" > foo @@ -56,14 +64,20 @@ do hive $HIVE_OPTS -f foo done - -cat step20-createMonitorDB_RIs_tail.sql | sed "s/SOURCE/$1/g" | sed "s/TARGET/$2_RIs_tail/g1" | sed "s/CONTEXTS/\"'knowmad::other','dh-ch::other', 'enermaps::other', 'gotriple::other', 'neanias-atmospheric::other', 'rural-digital-europe::other', 'covid-19::other', 'aurora::other', 'neanias-space::other', 'north-america-studies::other', 'north-american-studies::other', 'eutopia::other'\"/g" > foo +cat step20-createMonitorDB_RIs_tail.sql | sed "s/SOURCE/$1/g" | sed "s/TARGET/$2_ris_tail/g1" | sed "s/CONTEXTS/\"'knowmad::other','dh-ch::other', 'enermaps::other', 'gotriple::other', 'neanias-atmospheric::other', 'rural-digital-europe::other', 'covid-19::other', 'aurora::other', 'neanias-space::other', 'north-america-studies::other', 'north-american-studies::other', 'eutopia::other'\"/g" > foo hive $HIVE_OPTS -f foo -cat step20-createMonitorDB.sql | sed "s/SOURCE/$1/g" | sed "s/TARGET/$2_RIs_tail/g1" > foo +cat step20-createMonitorDB.sql | sed "s/SOURCE/$1/g" | sed "s/TARGET/$2_ris_tail/g1" > foo hive $HIVE_OPTS -f foo echo "Hive shell finished" +echo "Updating shadow monitor all database" +hive -e "drop database if exists ${SHADOW} cascade" +hive -e "create database if not exists ${SHADOW}" +hive $HIVE_OPTS --database ${2} -e "show tables" | grep -v WARN | sed "s/\(.*\)/create view ${SHADOW}.\1 as select * from ${2}.\1;/" > foo +hive -f foo +echo "Updated shadow monitor all database" + echo "Updating shadow monitor funded database" hive -e "drop database if exists ${SHADOW}_funded cascade" hive -e "create database if not exists ${SHADOW}_funded" 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 old mode 100644 new mode 100755 diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql index 132cb482e..4a8f81943 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql @@ -37,8 +37,15 @@ select * from ${stats_db_name}.otherresearchproduct_refereed; create table if not exists ${stats_db_name}.indi_impact_measures STORED AS PARQUET as select substr(id, 4) as id, measures_ids.id impactmetric, cast(measures_ids.unit.value[0] as double) score, -cast(measures_ids.unit.value[0] as decimal(6,3)) score_dec, measures_ids.unit.value[1] class +cast(measures_ids.unit.value[0] as decimal(6,3)) score_dec, measures_ids.unit.value[1] impact_class from ${openaire_db_name}.result lateral view explode(measures) measures as measures_ids where measures_ids.id!='views' and measures_ids.id!='downloads'; -ANALYZE TABLE indi_impact_measures COMPUTE STATISTICS; +create table if not exists ${stats_db_name}.result_apc_affiliations STORED AS PARQUET as +select distinct substr(rel.target,4) id, substr(rel.source,4) organization, o.legalname.value name, +cast(rel.properties[0].value as double) apc_amount, +rel.properties[1].value apc_currency +from ${openaire_db_name}.relation rel +join ${openaire_db_name}.organization o on o.id=rel.source +join ${openaire_db_name}.result r on r.id=rel.target +where rel.subreltype = 'affiliation' and rel.datainfo.deletedbyinference = false and size(rel.properties)>0; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql index f39ff2afd..615f523ce 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql @@ -35,6 +35,7 @@ create or replace view ${stats_db_name}.doctoratestudents as select * from stats create or replace view ${stats_db_name}.totalresearchers as select * from stats_ext.totalresearchers; create or replace view ${stats_db_name}.totalresearchersft as select * from stats_ext.totalresearchersft; create or replace view ${stats_db_name}.hrrst as select * from stats_ext.hrrst; +create or replace view ${stats_db_name}.graduatedoctorates as select * from stats_ext.graduatedoctorates; create table if not exists ${stats_db_name}.result_instance stored as parquet as select distinct r.* diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql index 36b34cc3c..6af486340 100755 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql @@ -1,89 +1,90 @@ -- Sprint 1 ---- -create table if not exists indi_pub_green_oa stored as parquet as +drop table if exists ${stats_db_name}.indi_pub_green_oa purge; + +create table if not exists ${stats_db_name}.indi_pub_green_oa stored as parquet as select distinct p.id, coalesce(green_oa, 0) as green_oa -from publication p +from ${stats_db_name}.publication p left outer join ( select p.id, 1 as green_oa - from publication p - join result_instance ri on ri.id = p.id - join datasource on datasource.id = ri.hostedby + from ${stats_db_name}.publication p + join ${stats_db_name}.result_instance ri on ri.id = p.id + join ${stats_db_name}.datasource on datasource.id = ri.hostedby where datasource.type like '%Repository%' and (ri.accessright = 'Open Access' or ri.accessright = 'Embargo' or ri.accessright = 'Open Source')) tmp on p.id= tmp.id; -ANALYZE TABLE indi_pub_green_oa COMPUTE STATISTICS; +drop table if exists ${stats_db_name}.indi_pub_grey_lit purge; -create table if not exists indi_pub_grey_lit stored as parquet as +create table if not exists ${stats_db_name}.indi_pub_grey_lit stored as parquet as select distinct p.id, coalesce(grey_lit, 0) as grey_lit -from publication p +from ${stats_db_name}.publication p left outer join ( select p.id, 1 as grey_lit - from publication p - join result_classifications rt on rt.id = p.id + from ${stats_db_name}.publication p + join ${stats_db_name}.result_classifications rt on rt.id = p.id where rt.type not in ('Article','Part of book or chapter of book','Book','Doctoral thesis','Master thesis','Data Paper', 'Thesis', 'Bachelor thesis', 'Conference object') and - not exists (select 1 from result_classifications rc where type ='Other literature type' + not exists (select 1 from ${stats_db_name}.result_classifications rc where type ='Other literature type' and rc.id=p.id)) tmp on p.id=tmp.id; -ANALYZE TABLE indi_pub_grey_lit COMPUTE STATISTICS; +drop table if exists ${stats_db_name}.indi_pub_doi_from_crossref purge; -create table if not exists indi_pub_doi_from_crossref stored as parquet as +create table if not exists ${stats_db_name}.indi_pub_doi_from_crossref stored as parquet as select distinct p.id, coalesce(doi_from_crossref, 0) as doi_from_crossref -from publication p +from ${stats_db_name}.publication p left outer join - (select ri.id, 1 as doi_from_crossref from result_instance ri - join datasource d on d.id = ri.collectedfrom + (select ri.id, 1 as doi_from_crossref from ${stats_db_name}.result_instance ri + join ${stats_db_name}.datasource d on d.id = ri.collectedfrom where pidtype='Digital Object Identifier' and d.name ='Crossref') tmp on tmp.id=p.id; -ANALYZE TABLE indi_pub_doi_from_crossref COMPUTE STATISTICS; - -- Sprint 2 ---- -create table if not exists indi_result_has_cc_licence stored as parquet as +drop table if exists ${stats_db_name}.indi_result_has_cc_licence purge; + +create table if not exists ${stats_db_name}.indi_result_has_cc_licence stored as parquet as select distinct r.id, (case when lic='' or lic is null then 0 else 1 end) as has_cc_license -from result r - left outer join (select r.id, license.type as lic from result r - join result_licenses as license on license.id = r.id +from ${stats_db_name}.result r +left outer join (select r.id, license.type as lic from ${stats_db_name}.result r + join ${stats_db_name}.result_licenses as license on license.id = r.id where lower(license.type) LIKE '%creativecommons.org%' OR lower(license.type) LIKE '%cc-%') tmp on r.id= tmp.id; -ANALYZE TABLE indi_result_has_cc_licence COMPUTE STATISTICS; +drop table if exists ${stats_db_name}.indi_result_has_cc_licence_url purge; -create table if not exists indi_result_has_cc_licence_url stored as parquet as +create table if not exists ${stats_db_name}.indi_result_has_cc_licence_url stored as parquet as select distinct r.id, case when lic_host='' or lic_host is null then 0 else 1 end as has_cc_license_url -from result r +from ${stats_db_name}.result r left outer join (select r.id, lower(parse_url(license.type, "HOST")) as lic_host - from result r - join result_licenses as license on license.id = r.id + from ${stats_db_name}.result r + join ${stats_db_name}.result_licenses as license on license.id = r.id WHERE lower(parse_url(license.type, "HOST")) = "creativecommons.org") tmp on r.id= tmp.id; -ANALYZE TABLE indi_result_has_cc_licence_url COMPUTE STATISTICS; +drop table if exists ${stats_db_name}.indi_pub_has_abstract purge; -create table if not exists indi_pub_has_abstract stored as parquet as +create table if not exists ${stats_db_name}.indi_pub_has_abstract stored as parquet as select distinct publication.id, cast(coalesce(abstract, true) as int) has_abstract -from publication; +from ${stats_db_name}.publication; -ANALYZE TABLE indi_pub_has_abstract COMPUTE STATISTICS; +drop table if exists ${stats_db_name}.indi_result_with_orcid purge; -create table if not exists indi_result_with_orcid stored as parquet as +create table if not exists ${stats_db_name}.indi_result_with_orcid stored as parquet as select distinct r.id, coalesce(has_orcid, 0) as has_orcid -from result r - left outer join (select id, 1 as has_orcid from result_orcid) tmp +from ${stats_db_name}.result r + left outer join (select id, 1 as has_orcid from ${stats_db_name}.result_orcid) tmp on r.id= tmp.id; -ANALYZE TABLE indi_result_with_orcid COMPUTE STATISTICS; - ---- Sprint 3 ---- -create table if not exists indi_funded_result_with_fundref stored as parquet as + +drop table if exists ${stats_db_name}.indi_funded_result_with_fundref purge; + +create table if not exists ${stats_db_name}.indi_funded_result_with_fundref stored as parquet as select distinct r.result as id, coalesce(fundref, 0) as fundref -from project_results r - left outer join (select distinct result, 1 as fundref from project_results +from ${stats_db_name}.project_results r + left outer join (select distinct result, 1 as fundref from ${stats_db_name}.project_results where provenance='Harvested') tmp on r.result= tmp.result; -ANALYZE TABLE indi_funded_result_with_fundref COMPUTE STATISTICS; - -- create table indi_result_org_collab stored as parquet as -- select o1.organization org1, o2.organization org2, count(distinct o1.id) as collaborations -- from result_organization as o1 @@ -92,68 +93,70 @@ ANALYZE TABLE indi_funded_result_with_fundref COMPUTE STATISTICS; -- -- compute stats indi_result_org_collab; -- -create TEMPORARY TABLE tmp AS SELECT ro.organization organization, ro.id, o.name from result_organization ro -join organization o on o.id=ro.organization where o.name is not null; +create TEMPORARY TABLE ${stats_db_name}.tmp AS SELECT ro.organization organization, ro.id, o.name from ${stats_db_name}.result_organization ro +join ${stats_db_name}.organization o on o.id=ro.organization where o.name is not null; -create table if not exists indi_result_org_collab stored as parquet as +drop table if exists ${stats_db_name}.indi_result_org_collab purge; + +create table if not exists ${stats_db_name}.indi_result_org_collab stored as parquet as select o1.organization org1, o1.name org1name1, o2.organization org2, o2.name org2name2, count(o1.id) as collaborations -from tmp as o1 -join tmp as o2 where o1.id=o2.id and o1.organization!=o2.organization and o1.name!=o2.name +from ${stats_db_name}.tmp as o1 +join ${stats_db_name}.tmp as o2 where o1.id=o2.id and o1.organization!=o2.organization and o1.name!=o2.name group by o1.organization, o2.organization, o1.name, o2.name; -drop table tmp purge; +drop table ${stats_db_name}.tmp purge; -ANALYZE TABLE indi_result_org_collab COMPUTE STATISTICS; +create TEMPORARY TABLE ${stats_db_name}.tmp AS +select distinct ro.organization organization, ro.id, o.name, o.country from ${stats_db_name}.result_organization ro +join ${stats_db_name}.organization o on o.id=ro.organization where country <> 'UNKNOWN' and o.name is not null; -create TEMPORARY TABLE tmp AS -select distinct ro.organization organization, ro.id, o.name, o.country from result_organization ro -join organization o on o.id=ro.organization where country <> 'UNKNOWN' and o.name is not null; +drop table if exists ${stats_db_name}.indi_result_org_country_collab purge; -create table if not exists indi_result_org_country_collab stored as parquet as +create table if not exists ${stats_db_name}.indi_result_org_country_collab stored as parquet as select o1.organization org1,o1.name org1name1, o2.country country2, count(o1.id) as collaborations -from tmp as o1 join tmp as o2 on o1.id=o2.id +from ${stats_db_name}.tmp as o1 join ${stats_db_name}.tmp as o2 on o1.id=o2.id where o1.id=o2.id and o1.country!=o2.country group by o1.organization, o1.id, o1.name, o2.country; -drop table tmp purge; +drop table ${stats_db_name}.tmp purge; -ANALYZE TABLE indi_result_org_country_collab COMPUTE STATISTICS; +create TEMPORARY TABLE ${stats_db_name}.tmp AS +select o.id organization, o.name, ro.project as project from ${stats_db_name}.organization o + join ${stats_db_name}.organization_projects ro on o.id=ro.id where o.name is not null; -create TEMPORARY TABLE tmp AS -select o.id organization, o.name, ro.project as project from organization o - join organization_projects ro on o.id=ro.id where o.name is not null; +drop table if exists ${stats_db_name}.indi_project_collab_org purge; -create table if not exists indi_project_collab_org stored as parquet as +create table if not exists ${stats_db_name}.indi_project_collab_org stored as parquet as select o1.organization org1,o1.name orgname1, o2.organization org2, o2.name orgname2, count(distinct o1.project) as collaborations -from tmp as o1 - join tmp as o2 on o1.project=o2.project +from ${stats_db_name}.tmp as o1 + join ${stats_db_name}.tmp as o2 on o1.project=o2.project where o1.organization<>o2.organization and o1.name<>o2.name group by o1.name,o2.name, o1.organization, o2.organization; -drop table tmp purge; +drop table ${stats_db_name}.tmp purge; -ANALYZE TABLE indi_project_collab_org COMPUTE STATISTICS; - -create TEMPORARY TABLE tmp AS -select o.id organization, o.name, o.country , ro.project as project from organization o - join organization_projects ro on o.id=ro.id +create TEMPORARY TABLE ${stats_db_name}.tmp AS +select o.id organization, o.name, o.country , ro.project as project from ${stats_db_name}.organization o + join ${stats_db_name}.organization_projects ro on o.id=ro.id and o.country <> 'UNKNOWN' and o.name is not null; -create table if not exists indi_project_collab_org_country stored as parquet as +drop table if exists ${stats_db_name}.indi_project_collab_org_country purge; + +create table if not exists ${stats_db_name}.indi_project_collab_org_country stored as parquet as select o1.organization org1,o1.name org1name, o2.country country2, count(distinct o1.project) as collaborations -from tmp as o1 - join tmp as o2 on o1.project=o2.project +from ${stats_db_name}.tmp as o1 + join ${stats_db_name}.tmp as o2 on o1.project=o2.project where o1.organization<>o2.organization and o1.country<>o2.country group by o1.organization, o2.country, o1.name; -drop table tmp purge; +drop table ${stats_db_name}.tmp purge; -ANALYZE TABLE indi_project_collab_org_country COMPUTE STATISTICS; +drop table if exists ${stats_db_name}.indi_funder_country_collab purge; -create table if not exists indi_funder_country_collab stored as parquet as - with tmp as (select funder, project, country from organization_projects op - join organization o on o.id=op.id - join project p on p.id=op.project +create table if not exists ${stats_db_name}.indi_funder_country_collab stored as parquet as + with tmp as (select funder, project, country from ${stats_db_name}.organization_projects op + join ${stats_db_name}.organization o on o.id=op.id + join ${stats_db_name}.project p on p.id=op.project where country <> 'UNKNOWN') select f1.funder, f1.country as country1, f2.country as country2, count(distinct f1.project) as collaborations from tmp as f1 @@ -161,155 +164,180 @@ from tmp as f1 where f1.country<>f2.country group by f1.funder, f2.country, f1.country; -ANALYZE TABLE indi_funder_country_collab COMPUTE STATISTICS; - -create TEMPORARY TABLE tmp AS -select distinct country, ro.id as result from organization o - join result_organization ro on o.id=ro.organization +create TEMPORARY TABLE ${stats_db_name}.tmp AS +select distinct country, ro.id as result from ${stats_db_name}.organization o + join ${stats_db_name}.result_organization ro on o.id=ro.organization where country <> 'UNKNOWN' and o.name is not null; -create table if not exists indi_result_country_collab stored as parquet as +drop table if exists ${stats_db_name}.indi_result_country_collab purge; + +create table if not exists ${stats_db_name}.indi_result_country_collab stored as parquet as select o1.country country1, o2.country country2, count(o1.result) as collaborations -from tmp as o1 - join tmp as o2 on o1.result=o2.result +from ${stats_db_name}.tmp as o1 + join ${stats_db_name}.tmp as o2 on o1.result=o2.result where o1.country<>o2.country group by o1.country, o2.country; -drop table tmp purge; - -ANALYZE TABLE indi_result_country_collab COMPUTE STATISTICS; +drop table ${stats_db_name}.tmp purge; ---- Sprint 4 ---- -create table if not exists indi_pub_diamond stored as parquet as +drop table if exists ${stats_db_name}.indi_pub_diamond purge; + +create table if not exists ${stats_db_name}.indi_pub_diamond stored as parquet as select distinct pd.id, coalesce(in_diamond_journal, 0) as in_diamond_journal -from publication_datasources pd +from ${stats_db_name}.publication_datasources pd left outer join ( - select pd.id, 1 as in_diamond_journal from publication_datasources pd - join datasource d on d.id=pd.datasource + select pd.id, 1 as in_diamond_journal from ${stats_db_name}.publication_datasources pd + join ${stats_db_name}.datasource d on d.id=pd.datasource join STATS_EXT.plan_s_jn ps where (ps.issn_print=d.issn_printed and ps.issn_online=d.issn_online) and (ps.journal_is_in_doaj=true or ps.journal_is_oa=true) and ps.has_apc=false) tmp on pd.id=tmp.id; -ANALYZE TABLE indi_pub_diamond COMPUTE STATISTICS; +drop table if exists ${stats_db_name}.indi_pub_in_transformative purge; -create table if not exists indi_pub_in_transformative stored as parquet as +create table if not exists ${stats_db_name}.indi_pub_in_transformative stored as parquet as select distinct pd.id, coalesce(is_transformative, 0) as is_transformative -from publication pd +from ${stats_db_name}.publication pd left outer join ( - select pd.id, 1 as is_transformative from publication_datasources pd - join datasource d on d.id=pd.datasource + select pd.id, 1 as is_transformative from ${stats_db_name}.publication_datasources pd + join ${stats_db_name}.datasource d on d.id=pd.datasource join STATS_EXT.plan_s_jn ps where (ps.issn_print=d.issn_printed and ps.issn_online=d.issn_online) and ps.is_transformative_journal=true) tmp on pd.id=tmp.id; -ANALYZE TABLE indi_pub_in_transformative COMPUTE STATISTICS; +drop table if exists ${stats_db_name}.indi_pub_closed_other_open purge; -create table if not exists indi_pub_closed_other_open stored as parquet as -select distinct ri.id, coalesce(pub_closed_other_open, 0) as pub_closed_other_open from result_instance ri +create table if not exists ${stats_db_name}.indi_pub_closed_other_open stored as parquet as +select distinct ri.id, coalesce(pub_closed_other_open, 0) as pub_closed_other_open from ${stats_db_name}.result_instance ri left outer join - (select ri.id, 1 as pub_closed_other_open from result_instance ri - join publication p on p.id=ri.id - join datasource d on ri.hostedby=d.id + (select ri.id, 1 as pub_closed_other_open from ${stats_db_name}.result_instance ri + join ${stats_db_name}.publication p on p.id=ri.id + join ${stats_db_name}.datasource d on ri.hostedby=d.id where d.type like '%Journal%' and ri.accessright='Closed Access' and (p.bestlicence='Open Access' or p.bestlicence='Open Source')) tmp on tmp.id=ri.id; -ANALYZE TABLE indi_pub_closed_other_open COMPUTE STATISTICS; - ---- Sprint 5 ---- -create table if not exists indi_result_no_of_copies stored as parquet as -select id, count(id) as number_of_copies from result_instance group by id; +drop table if exists ${stats_db_name}.indi_result_no_of_copies purge; -ANALYZE TABLE indi_result_no_of_copies COMPUTE STATISTICS; +create table if not exists ${stats_db_name}.indi_result_no_of_copies stored as parquet as +select id, count(id) as number_of_copies from ${stats_db_name}.result_instance group by id; ---- Sprint 6 ---- -create table if not exists indi_pub_downloads stored as parquet as +drop table if exists ${stats_db_name}.indi_pub_downloads purge; + +create table if not exists ${stats_db_name}.indi_pub_downloads stored as parquet as SELECT result_id, sum(downloads) no_downloads from openaire_prod_usage_stats.usage_stats - join publication on result_id=id + join ${stats_db_name}.publication on result_id=id where downloads>0 GROUP BY result_id order by no_downloads desc; -ANALYZE TABLE indi_pub_downloads COMPUTE STATISTICS; +--ANALYZE TABLE ${stats_db_name}.indi_pub_downloads COMPUTE STATISTICS; -create table if not exists indi_pub_downloads_datasource stored as parquet as +drop table if exists ${stats_db_name}.indi_pub_downloads_datasource purge; + +create table if not exists ${stats_db_name}.indi_pub_downloads_datasource stored as parquet as SELECT result_id, repository_id, sum(downloads) no_downloads from openaire_prod_usage_stats.usage_stats - join publication on result_id=id + join ${stats_db_name}.publication on result_id=id where downloads>0 GROUP BY result_id, repository_id order by result_id; -ANALYZE TABLE indi_pub_downloads_datasource COMPUTE STATISTICS; +drop table if exists ${stats_db_name}.indi_pub_downloads_year purge; -create table if not exists indi_pub_downloads_year stored as parquet as -SELECT result_id, substring(us.`date`, 1,4) as `year`, sum(downloads) no_downloads +create table if not exists ${stats_db_name}.indi_pub_downloads_year stored as parquet as +SELECT result_id, cast(substring(us.`date`, 1,4) as int) as `year`, sum(downloads) no_downloads from openaire_prod_usage_stats.usage_stats us -join publication on result_id=id where downloads>0 +join ${stats_db_name}.publication on result_id=id where downloads>0 GROUP BY result_id, substring(us.`date`, 1,4); -ANALYZE TABLE indi_pub_downloads_year COMPUTE STATISTICS; +drop table if exists ${stats_db_name}.indi_pub_downloads_datasource_year purge; -create table if not exists indi_pub_downloads_datasource_year stored as parquet as -SELECT result_id, substring(us.`date`, 1,4) as `year`, repository_id, sum(downloads) no_downloads from openaire_prod_usage_stats.usage_stats us -join publication on result_id=id +create table if not exists ${stats_db_name}.indi_pub_downloads_datasource_year stored as parquet as +SELECT result_id, cast(substring(us.`date`, 1,4) as int) as `year`, repository_id, sum(downloads) no_downloads from openaire_prod_usage_stats.usage_stats us +join ${stats_db_name}.publication on result_id=id where downloads>0 GROUP BY result_id, repository_id, substring(us.`date`, 1,4); -ANALYZE TABLE indi_pub_downloads_datasource_year COMPUTE STATISTICS; - ---- Sprint 7 ---- -create table if not exists indi_pub_gold_oa stored as parquet as - WITH gold_oa AS ( SELECT - issn_l, - journal_is_in_doaj, - journal_is_oa, - issn_1 as issn - FROM - STATS_EXT.oa_journals - WHERE - issn_1 != "" - UNION - ALL SELECT - issn_l, - journal_is_in_doaj, - journal_is_oa, - issn_2 as issn - FROM - STATS_EXT.oa_journals - WHERE - issn_2 != "" ), issn AS ( SELECT - * - FROM -( SELECT - id, - issn_printed as issn - FROM - datasource - WHERE - issn_printed IS NOT NULL - UNION ALL - SELECT - id, - issn_online as issn - FROM - datasource - WHERE - issn_online IS NOT NULL or id like '%doajarticles%') as issn - WHERE - LENGTH(issn) > 7) -SELECT - DISTINCT pd.id, coalesce(is_gold, 0) as is_gold -FROM - publication_datasources pd - left outer join( - select pd.id, 1 as is_gold FROM publication_datasources pd - JOIN issn on issn.id=pd.datasource - JOIN gold_oa on issn.issn = gold_oa.issn) tmp - on pd.id=tmp.id; +drop table if exists ${stats_db_name}.indi_pub_gold_oa purge; -ANALYZE TABLE indi_pub_gold_oa COMPUTE STATISTICS; +--create table if not exists ${stats_db_name}.indi_pub_gold_oa stored as parquet as +-- WITH gold_oa AS ( SELECT +-- issn_l, +-- journal_is_in_doaj, +-- journal_is_oa, +-- issn_1 as issn +-- FROM +-- STATS_EXT.oa_journals +-- WHERE +-- issn_1 != "" +-- UNION +-- ALL SELECT +-- issn_l, +-- journal_is_in_doaj, +-- journal_is_oa, +-- issn_2 as issn +-- FROM +-- STATS_EXT.oa_journals +-- WHERE +-- issn_2 != "" ), issn AS ( SELECT +-- * +-- FROM +--( SELECT +-- id, +-- issn_printed as issn +-- FROM +-- ${stats_db_name}.datasource +-- WHERE +-- issn_printed IS NOT NULL +-- UNION ALL +-- SELECT +-- id, +-- issn_online as issn +-- FROM +-- ${stats_db_name}.datasource +-- WHERE +-- issn_online IS NOT NULL or id like '%doajarticles%') as issn +-- WHERE +-- LENGTH(issn) > 7) +--SELECT +-- DISTINCT pd.id, coalesce(is_gold, 0) as is_gold +--FROM +-- ${stats_db_name}.publication_datasources pd +-- left outer join( +-- select pd.id, 1 as is_gold FROM ${stats_db_name}.publication_datasources pd +-- JOIN issn on issn.id=pd.datasource +-- JOIN gold_oa on issn.issn = gold_oa.issn) tmp +-- on pd.id=tmp.id; -create table if not exists indi_pub_hybrid_oa_with_cc stored as parquet as +create table if not exists ${stats_db_name}.indi_pub_gold_oa stored as parquet as +with gold_oa as ( +SELECT issn,issn_l from stats_ext.issn_gold_oa_dataset_v5), +issn AS (SELECT * FROM +(SELECT id,issn_printed as issn FROM ${stats_db_name}.datasource +WHERE issn_printed IS NOT NULL +UNION ALL +SELECT id, issn_online as issn FROM ${stats_db_name}.datasource +WHERE issn_online IS NOT NULL or id like '%doajarticles%') as issn +WHERE LENGTH(issn) > 7), +alljournals AS(select issn, issn_l from stats_ext.alljournals +where journal_is_in_doaj=true or journal_is_oa=true) +SELECT DISTINCT pd.id, coalesce(is_gold, 0) as is_gold +FROM ${stats_db_name}.publication_datasources pd +left outer join ( +select pd.id, 1 as is_gold FROM ${stats_db_name}.publication_datasources pd +JOIN issn on issn.id=pd.datasource +JOIN gold_oa on issn.issn = gold_oa.issn +join alljournals on issn.issn=alljournals.issn +left outer join ${stats_db_name}.result_instance ri on ri.id=pd.id +and ri.accessright!='Closed Access' and ri.accessright_uw='gold') tmp +on pd.id=tmp.id; + +drop table if exists ${stats_db_name}.indi_pub_hybrid_oa_with_cc purge; + +create table if not exists ${stats_db_name}.indi_pub_hybrid_oa_with_cc stored as parquet as WITH hybrid_oa AS ( SELECT issn_l, journal_is_in_doaj, journal_is_oa, issn_print as issn FROM STATS_EXT.plan_s_jn @@ -322,480 +350,769 @@ create table if not exists indi_pub_hybrid_oa_with_cc stored as parquet as SELECT * FROM ( SELECT id, issn_printed as issn - FROM datasource + FROM ${stats_db_name}.datasource WHERE issn_printed IS NOT NULL UNION ALL SELECT id,issn_online as issn - FROM datasource + FROM ${stats_db_name}.datasource WHERE issn_online IS NOT NULL ) as issn WHERE LENGTH(issn) > 7) SELECT DISTINCT pd.id, coalesce(is_hybrid_oa, 0) as is_hybrid_oa -FROM publication_datasources pd +FROM ${stats_db_name}.publication_datasources pd LEFT OUTER JOIN ( - SELECT pd.id, 1 as is_hybrid_oa from publication_datasources pd - JOIN datasource d on d.id=pd.datasource + SELECT pd.id, 1 as is_hybrid_oa from ${stats_db_name}.publication_datasources pd + JOIN ${stats_db_name}.datasource d on d.id=pd.datasource JOIN issn on issn.id=pd.datasource JOIN hybrid_oa ON issn.issn = hybrid_oa.issn - JOIN indi_result_has_cc_licence cc on pd.id=cc.id - JOIN indi_pub_gold_oa ga on pd.id=ga.id + JOIN ${stats_db_name}.indi_result_has_cc_licence cc on pd.id=cc.id + JOIN ${stats_db_name}.indi_pub_gold_oa ga on pd.id=ga.id where cc.has_cc_license=1 and ga.is_gold=0) tmp on pd.id=tmp.id; -ANALYZE TABLE indi_pub_hybrid_oa_with_cc COMPUTE STATISTICS; +drop table if exists ${stats_db_name}.indi_pub_hybrid purge; -create table if not exists indi_pub_hybrid stored as parquet as - WITH gold_oa AS ( SELECT - issn_l, - journal_is_in_doaj, - journal_is_oa, - issn_1 as issn, - has_apc - FROM - STATS_EXT.oa_journals - WHERE - issn_1 != "" - UNION - ALL SELECT - issn_l, - journal_is_in_doaj, - journal_is_oa, - issn_2 as issn, - has_apc - FROM - STATS_EXT.oa_journals - WHERE - issn_2 != "" ), issn AS ( SELECT - * - FROM -( SELECT - id, - issn_printed as issn - FROM - datasource - WHERE - issn_printed IS NOT NULL - UNION ALL - SELECT - id, - issn_online as issn - FROM - datasource - WHERE - issn_online IS NOT NULL or id like '%doajarticles%') as issn - WHERE - LENGTH(issn) > 7) -select distinct pd.id, coalesce(is_hybrid, 0) as is_hybrid -from publication_datasources pd - left outer join ( - select pd.id, 1 as is_hybrid from publication_datasources pd - join datasource d on d.id=pd.datasource - join issn on issn.id=pd.datasource - join gold_oa on issn.issn=gold_oa.issn - where (gold_oa.journal_is_in_doaj=false or gold_oa.journal_is_oa=false))tmp - on pd.id=tmp.id; +--create table if not exists ${stats_db_name}.indi_pub_hybrid stored as parquet as +-- WITH gold_oa AS ( SELECT +-- issn_l, +-- journal_is_in_doaj, +-- journal_is_oa, +-- issn_1 as issn, +-- has_apc +-- FROM +-- STATS_EXT.oa_journals +-- WHERE +-- issn_1 != "" +-- UNION +-- ALL SELECT +-- issn_l, +-- journal_is_in_doaj, +-- journal_is_oa, +-- issn_2 as issn, +-- has_apc +-- FROM +-- STATS_EXT.oa_journals +-- WHERE +-- issn_2 != "" ), issn AS ( SELECT +-- * +-- FROM +--( SELECT +-- id, +-- issn_printed as issn +-- FROM +-- ${stats_db_name}.datasource +-- WHERE +-- issn_printed IS NOT NULL +-- UNION ALL +-- SELECT +-- id, +-- issn_online as issn +-- FROM +-- ${stats_db_name}.datasource +-- WHERE +-- issn_online IS NOT NULL or id like '%doajarticles%') as issn +-- WHERE +-- LENGTH(issn) > 7) +--select distinct pd.id, coalesce(is_hybrid, 0) as is_hybrid +--from ${stats_db_name}.publication_datasources pd +-- left outer join ( +-- select pd.id, 1 as is_hybrid from ${stats_db_name}.publication_datasources pd +-- join ${stats_db_name}.datasource d on d.id=pd.datasource +-- join issn on issn.id=pd.datasource +-- join gold_oa on issn.issn=gold_oa.issn +-- where (gold_oa.journal_is_in_doaj=false or gold_oa.journal_is_oa=false))tmp +-- on pd.id=tmp.id; -ANALYZE TABLE indi_pub_hybrid COMPUTE STATISTICS; +create table if not exists ${stats_db_name}.indi_pub_hybrid stored as parquet as +select pd.id,coalesce(is_hybrid,0) is_hybrid from ${stats_db_name}.publication_datasources pd +left outer join (select pd.id, 1 as is_hybrid from ${stats_db_name}.publication_datasources pd +join ${stats_db_name}.datasource d on pd.datasource=d.id +join ${stats_db_name}.result_instance ri on ri.id=pd.id +join ${stats_db_name}.indi_pub_gold_oa indi_gold on indi_gold.id=pd.id +join ${stats_db_name}.result_accessroute ra on ra.id=pd.id +where d.type like '%Journal%' and ri.accessright!='Closed Access' and (ri.accessright_uw!='gold' +or indi_gold.is_gold=0) and (ra.accessroute='hybrid' or ri.license is not null)) tmp +on pd.id=tmp.id; -create table if not exists indi_org_fairness stored as parquet as +drop table if exists ${stats_db_name}.indi_org_fairness purge; + +create table if not exists ${stats_db_name}.indi_org_fairness stored as parquet as --return results with PIDs, and rich metadata group by organization with result_fair as - (select ro.organization organization, count(distinct ro.id) no_result_fair from result_organization ro - join result r on r.id=ro.id + (select ro.organization organization, count(distinct ro.id) no_result_fair from ${stats_db_name}.result_organization ro + join ${stats_db_name}.result r on r.id=ro.id --join result_pids rp on r.id=rp.id where (title is not null) and (publisher is not null) and (abstract=true) and (year is not null) and (authors>0) and cast(year as int)>2003 group by ro.organization), --return all results group by organization - allresults as (select organization, count(distinct ro.id) no_allresults from result_organization ro - join result r on r.id=ro.id + allresults as (select ro.organization, count(distinct ro.id) no_allresults from ${stats_db_name}.result_organization ro + join ${stats_db_name}.result r on r.id=ro.id where cast(year as int)>2003 - group by organization) + group by ro.organization) --return results_fair/all_results select allresults.organization, result_fair.no_result_fair/allresults.no_allresults org_fairness from allresults join result_fair on result_fair.organization=allresults.organization; -ANALYZE TABLE indi_org_fairness COMPUTE STATISTICS; - -create table if not exists indi_org_fairness_pub_pr stored as parquet as - with result_fair as - (select ro.organization organization, count(distinct ro.id) no_result_fair - from result_organization ro - join publication p on p.id=ro.id - join indi_pub_doi_from_crossref dc on dc.id=p.id - join indi_pub_grey_lit gl on gl.id=p.id +CREATE TEMPORARY table ${stats_db_name}.result_fair as +select ro.organization organization, count(distinct ro.id) no_result_fair + from ${stats_db_name}.result_organization ro + join ${stats_db_name}.publication p on p.id=ro.id + join ${stats_db_name}.indi_pub_doi_from_crossref dc on dc.id=p.id + join ${stats_db_name}.indi_pub_grey_lit gl on gl.id=p.id where (title is not null) and (publisher is not null) and (abstract=true) and (year is not null) and (authors>0) and cast(year as int)>2003 and dc.doi_from_crossref=1 and gl.grey_lit=0 - group by ro.organization), - allresults as (select organization, count(distinct ro.id) no_allresults from result_organization ro - join publication p on p.id=ro.id + group by ro.organization; + +CREATE TEMPORARY TABLE ${stats_db_name}.allresults as +select ro.organization, count(distinct ro.id) no_allresults from ${stats_db_name}.result_organization ro + join ${stats_db_name}.publication p on p.id=ro.id where cast(year as int)>2003 - group by organization) ---return results_fair/all_results -select allresults.organization, result_fair.no_result_fair/allresults.no_allresults org_fairness -from allresults - join result_fair on result_fair.organization=allresults.organization; + group by ro.organization; -ANALYZE TABLE indi_org_fairness_pub_pr COMPUTE STATISTICS; +drop table if exists ${stats_db_name}.indi_org_fairness_pub_pr purge; -CREATE TEMPORARY table result_fair as - select year, ro.organization organization, count(distinct ro.id) no_result_fair from result_organization ro - join result p on p.id=ro.id +create table if not exists ${stats_db_name}.indi_org_fairness_pub_pr stored as parquet as +select ar.organization, rf.no_result_fair/ar.no_allresults org_fairness +from ${stats_db_name}.allresults ar + join ${stats_db_name}.result_fair rf on rf.organization=ar.organization; + +DROP table ${stats_db_name}.result_fair purge; +DROP table ${stats_db_name}.allresults purge; + +CREATE TEMPORARY table ${stats_db_name}.result_fair as + select year, ro.organization organization, count(distinct ro.id) no_result_fair from ${stats_db_name}.result_organization ro + join ${stats_db_name}.result p on p.id=ro.id where (title is not null) and (publisher is not null) and (abstract=true) and (year is not null) and (authors>0) and cast(year as int)>2003 group by ro.organization, year; -CREATE TEMPORARY TABLE allresults as select year, organization, count(distinct ro.id) no_allresults from result_organization ro - join result p on p.id=ro.id +CREATE TEMPORARY TABLE ${stats_db_name}.allresults as select year, ro.organization, count(distinct ro.id) no_allresults from ${stats_db_name}.result_organization ro + join ${stats_db_name}.result p on p.id=ro.id where cast(year as int)>2003 - group by organization, year; + group by ro.organization, year; -create table if not exists indi_org_fairness_pub_year stored as parquet as +drop table if exists ${stats_db_name}.indi_org_fairness_pub_year purge; + +create table if not exists ${stats_db_name}.indi_org_fairness_pub_year stored as parquet as select allresults.year, allresults.organization, result_fair.no_result_fair/allresults.no_allresults org_fairness -from allresults - join result_fair on result_fair.organization=allresults.organization and result_fair.year=allresults.year; +from ${stats_db_name}.allresults + join ${stats_db_name}.result_fair on result_fair.organization=allresults.organization and result_fair.year=allresults.year; -DROP table result_fair purge; -DROP table allresults purge; +DROP table ${stats_db_name}.result_fair purge; +DROP table ${stats_db_name}.allresults purge; -ANALYZE TABLE indi_org_fairness_pub_year COMPUTE STATISTICS; - -CREATE TEMPORARY TABLE result_fair as +CREATE TEMPORARY TABLE ${stats_db_name}.result_fair as select ro.organization organization, count(distinct ro.id) no_result_fair - from result_organization ro - join result p on p.id=ro.id + from ${stats_db_name}.result_organization ro + join ${stats_db_name}.result p on p.id=ro.id where (title is not null) and (publisher is not null) and (abstract=true) and (year is not null) and (authors>0) and cast(year as int)>2003 group by ro.organization; -CREATE TEMPORARY TABLE allresults as - select organization, count(distinct ro.id) no_allresults from result_organization ro - join result p on p.id=ro.id +CREATE TEMPORARY TABLE ${stats_db_name}.allresults as + select ro.organization, count(distinct ro.id) no_allresults from ${stats_db_name}.result_organization ro + join ${stats_db_name}.result p on p.id=ro.id where cast(year as int)>2003 - group by organization; + group by ro.organization; -create table if not exists indi_org_fairness_pub as -select allresults.organization, result_fair.no_result_fair/allresults.no_allresults org_fairness -from allresults join result_fair on result_fair.organization=allresults.organization; +drop table if exists ${stats_db_name}.indi_org_fairness_pub purge; -DROP table result_fair purge; -DROP table allresults purge; +create table if not exists ${stats_db_name}.indi_org_fairness_pub as +select ar.organization, rf.no_result_fair/ar.no_allresults org_fairness +from ${stats_db_name}.allresults ar join ${stats_db_name}.result_fair rf +on rf.organization=ar.organization; -ANALYZE TABLE indi_org_fairness_pub COMPUTE STATISTICS; +DROP table ${stats_db_name}.result_fair purge; +DROP table ${stats_db_name}.allresults purge; -CREATE TEMPORARY TABLE result_fair as - select year, ro.organization organization, count(distinct ro.id) no_result_fair from result_organization ro - join result r on r.id=ro.id - join result_pids rp on r.id=rp.id +CREATE TEMPORARY TABLE ${stats_db_name}.result_fair as + select year, ro.organization organization, count(distinct ro.id) no_result_fair from ${stats_db_name}.result_organization ro + join ${stats_db_name}.result r on r.id=ro.id + join ${stats_db_name}.result_pids rp on r.id=rp.id where (title is not null) and (publisher is not null) and (abstract=true) and (year is not null) and (authors>0) and cast(year as int)>2003 group by ro.organization, year; -CREATE TEMPORARY TABLE allresults as - select year, organization, count(distinct ro.id) no_allresults from result_organization ro - join result r on r.id=ro.id +CREATE TEMPORARY TABLE ${stats_db_name}.allresults as + select year, ro.organization, count(distinct ro.id) no_allresults from ${stats_db_name}.result_organization ro + join ${stats_db_name}.result r on r.id=ro.id where cast(year as int)>2003 - group by organization, year; + group by ro.organization, year; -create table if not exists indi_org_fairness_year stored as parquet as - select allresults.year, allresults.organization, result_fair.no_result_fair/allresults.no_allresults org_fairness - from allresults - join result_fair on result_fair.organization=allresults.organization and result_fair.year=allresults.year; +drop table if exists ${stats_db_name}.indi_org_fairness_year purge; -DROP table result_fair purge; -DROP table allresults purge; +create table if not exists ${stats_db_name}.indi_org_fairness_year stored as parquet as + select cast(allresults.year as int) year, allresults.organization, result_fair.no_result_fair/allresults.no_allresults org_fairness + from ${stats_db_name}.allresults + join ${stats_db_name}.result_fair on result_fair.organization=allresults.organization and cast(result_fair.year as int)=cast(allresults.year as int); -ANALYZE TABLE indi_org_fairness_year COMPUTE STATISTICS; +DROP table ${stats_db_name}.result_fair purge; +DROP table ${stats_db_name}.allresults purge; -CREATE TEMPORARY TABLE result_with_pid as - select year, ro.organization organization, count(distinct rp.id) no_result_with_pid from result_organization ro - join result_pids rp on rp.id=ro.id - join result r on r.id=rp.id +CREATE TEMPORARY TABLE ${stats_db_name}.result_with_pid as + select year, ro.organization, count(distinct rp.id) no_result_with_pid from ${stats_db_name}.result_organization ro + join ${stats_db_name}.result_pids rp on rp.id=ro.id + join ${stats_db_name}.result r on r.id=rp.id where cast(year as int) >2003 group by ro.organization, year; -CREATE TEMPORARY TABLE allresults as - select year, organization, count(distinct ro.id) no_allresults from result_organization ro - join result r on r.id=ro.id +CREATE TEMPORARY TABLE ${stats_db_name}.allresults as + select year, ro.organization, count(distinct ro.id) no_allresults from ${stats_db_name}.result_organization ro + join ${stats_db_name}.result r on r.id=ro.id where cast(year as int) >2003 - group by organization, year; + group by ro.organization, year; -create table if not exists indi_org_findable_year stored as parquet as -select allresults.year, allresults.organization, result_with_pid.no_result_with_pid/allresults.no_allresults org_findable -from allresults - join result_with_pid on result_with_pid.organization=allresults.organization and result_with_pid.year=allresults.year; +drop table if exists ${stats_db_name}.indi_org_findable_year purge; -DROP table result_with_pid purge; -DROP table allresults purge; +create table if not exists ${stats_db_name}.indi_org_findable_year stored as parquet as +select cast(allresults.year as int) year, allresults.organization, result_with_pid.no_result_with_pid/allresults.no_allresults org_findable +from ${stats_db_name}.allresults + join ${stats_db_name}.result_with_pid on result_with_pid.organization=allresults.organization and cast(result_with_pid.year as int)=cast(allresults.year as int); -ANALYZE TABLE indi_org_findable_year COMPUTE STATISTICS; +DROP table ${stats_db_name}.result_with_pid purge; +DROP table ${stats_db_name}.allresults purge; -CREATE TEMPORARY TABLE result_with_pid as -select ro.organization organization, count(distinct rp.id) no_result_with_pid from result_organization ro - join result_pids rp on rp.id=ro.id - join result r on r.id=rp.id +CREATE TEMPORARY TABLE ${stats_db_name}.result_with_pid as +select ro.organization, count(distinct rp.id) no_result_with_pid from ${stats_db_name}.result_organization ro + join ${stats_db_name}.result_pids rp on rp.id=ro.id + join ${stats_db_name}.result r on r.id=rp.id where cast(year as int) >2003 group by ro.organization; -CREATE TEMPORARY TABLE allresults as -select organization, count(distinct ro.id) no_allresults from result_organization ro - join result r on r.id=ro.id +CREATE TEMPORARY TABLE ${stats_db_name}.allresults as +select ro.organization, count(distinct ro.id) no_allresults from ${stats_db_name}.result_organization ro + join ${stats_db_name}.result r on r.id=ro.id where cast(year as int) >2003 - group by organization; + group by ro.organization; -create table if not exists indi_org_findable stored as parquet as +drop table if exists ${stats_db_name}.indi_org_findable purge; + +create table if not exists ${stats_db_name}.indi_org_findable stored as parquet as select allresults.organization, result_with_pid.no_result_with_pid/allresults.no_allresults org_findable -from allresults - join result_with_pid on result_with_pid.organization=allresults.organization; +from ${stats_db_name}.allresults + join ${stats_db_name}.result_with_pid on result_with_pid.organization=allresults.organization; -DROP table result_with_pid purge; -DROP table allresults purge; +DROP table ${stats_db_name}.result_with_pid purge; +DROP table ${stats_db_name}.allresults purge; -ANALYZE TABLE indi_org_findable COMPUTE STATISTICS; - -CREATE TEMPORARY TABLE pubs_oa as -SELECT ro.organization, count(distinct r.id) no_oapubs FROM publication r - join result_organization ro on ro.id=r.id - join result_instance ri on ri.id=r.id +CREATE TEMPORARY TABLE ${stats_db_name}.pubs_oa as +SELECT ro.organization, count(distinct r.id) no_oapubs FROM ${stats_db_name}.publication r + join ${stats_db_name}.result_organization ro on ro.id=r.id + join ${stats_db_name}.result_instance ri on ri.id=r.id where (ri.accessright = 'Open Access' or ri.accessright = 'Embargo' or ri.accessright = 'Open Source') and cast(r.year as int)>2003 group by ro.organization; -CREATE TEMPORARY TABLE datasets_oa as -SELECT ro.organization, count(distinct r.id) no_oadatasets FROM dataset r - join result_organization ro on ro.id=r.id - join result_instance ri on ri.id=r.id +CREATE TEMPORARY TABLE ${stats_db_name}.datasets_oa as +SELECT ro.organization, count(distinct r.id) no_oadatasets FROM ${stats_db_name}.dataset r + join ${stats_db_name}.result_organization ro on ro.id=r.id + join ${stats_db_name}.result_instance ri on ri.id=r.id where (ri.accessright = 'Open Access' or ri.accessright = 'Embargo' or ri.accessright = 'Open Source') and cast(r.year as int)>2003 group by ro.organization; -CREATE TEMPORARY TABLE software_oa as -SELECT ro.organization, count(distinct r.id) no_oasoftware FROM software r - join result_organization ro on ro.id=r.id - join result_instance ri on ri.id=r.id +CREATE TEMPORARY TABLE ${stats_db_name}.software_oa as +SELECT ro.organization, count(distinct r.id) no_oasoftware FROM ${stats_db_name}.software r + join ${stats_db_name}.result_organization ro on ro.id=r.id + join ${stats_db_name}.result_instance ri on ri.id=r.id where (ri.accessright = 'Open Access' or ri.accessright = 'Embargo' or ri.accessright = 'Open Source') and cast(r.year as int)>2003 group by ro.organization; -CREATE TEMPORARY TABLE allpubs as -SELECT ro.organization organization, count(ro.id) no_allpubs FROM result_organization ro - join publication ps on ps.id=ro.id +CREATE TEMPORARY TABLE ${stats_db_name}.allpubs as +SELECT ro.organization, count(ro.id) no_allpubs FROM ${stats_db_name}.result_organization ro + join ${stats_db_name}.publication ps on ps.id=ro.id where cast(ps.year as int)>2003 group by ro.organization; -CREATE TEMPORARY TABLE alldatasets as -SELECT ro.organization organization, count(ro.id) no_alldatasets FROM result_organization ro - join dataset ps on ps.id=ro.id +CREATE TEMPORARY TABLE ${stats_db_name}.alldatasets as +SELECT ro.organization, count(ro.id) no_alldatasets FROM ${stats_db_name}.result_organization ro + join ${stats_db_name}.dataset ps on ps.id=ro.id where cast(ps.year as int)>2003 group by ro.organization; -CREATE TEMPORARY TABLE allsoftware as -SELECT ro.organization organization, count(ro.id) no_allsoftware FROM result_organization ro - join software ps on ps.id=ro.id +CREATE TEMPORARY TABLE ${stats_db_name}.allsoftware as +SELECT ro.organization, count(ro.id) no_allsoftware FROM ${stats_db_name}.result_organization ro + join ${stats_db_name}.software ps on ps.id=ro.id where cast(ps.year as int)>2003 group by ro.organization; -CREATE TEMPORARY TABLE allpubsshare as -select pubs_oa.organization, pubs_oa.no_oapubs/allpubs.no_allpubs p from allpubs - join pubs_oa on allpubs.organization=pubs_oa.organization; +CREATE TEMPORARY TABLE ${stats_db_name}.allpubsshare as +select pubs_oa.organization, pubs_oa.no_oapubs/allpubs.no_allpubs p from ${stats_db_name}.allpubs + join ${stats_db_name}.pubs_oa on allpubs.organization=pubs_oa.organization; -CREATE TEMPORARY TABLE alldatasetssshare as +CREATE TEMPORARY TABLE ${stats_db_name}.alldatasetssshare as select datasets_oa.organization, datasets_oa.no_oadatasets/alldatasets.no_alldatasets d - from alldatasets - join datasets_oa on alldatasets.organization=datasets_oa.organization; + from ${stats_db_name}.alldatasets + join ${stats_db_name}.datasets_oa on alldatasets.organization=datasets_oa.organization; -CREATE TEMPORARY TABLE allsoftwaresshare as +CREATE TEMPORARY TABLE ${stats_db_name}.allsoftwaresshare as select software_oa.organization, software_oa.no_oasoftware/allsoftware.no_allsoftware s - from allsoftware - join software_oa on allsoftware.organization=software_oa.organization; + from ${stats_db_name}.allsoftware + join ${stats_db_name}.software_oa on allsoftware.organization=software_oa.organization; -create table if not exists indi_org_openess stored as parquet as +drop table if exists ${stats_db_name}.indi_org_openess purge; + +create table if not exists ${stats_db_name}.indi_org_openess stored as parquet as select allpubsshare.organization, (p+if(isnull(s),0,s)+if(isnull(d),0,d))/(1+(case when s is null then 0 else 1 end) +(case when d is null then 0 else 1 end)) - org_openess FROM allpubsshare + org_openess FROM ${stats_db_name}.allpubsshare left outer join (select organization,d from - alldatasetssshare) tmp1 + ${stats_db_name}.alldatasetssshare) tmp1 on tmp1.organization=allpubsshare.organization left outer join (select organization,s from - allsoftwaresshare) tmp2 + ${stats_db_name}.allsoftwaresshare) tmp2 on tmp2.organization=allpubsshare.organization; -DROP TABLE pubs_oa purge; -DROP TABLE datasets_oa purge; -DROP TABLE software_oa purge; -DROP TABLE allpubs purge; -DROP TABLE alldatasets purge; -DROP TABLE allsoftware purge; -DROP TABLE allpubsshare purge; -DROP TABLE alldatasetssshare purge; -DROP TABLE allsoftwaresshare purge; +DROP TABLE ${stats_db_name}.pubs_oa purge; +DROP TABLE ${stats_db_name}.datasets_oa purge; +DROP TABLE ${stats_db_name}.software_oa purge; +DROP TABLE ${stats_db_name}.allpubs purge; +DROP TABLE ${stats_db_name}.alldatasets purge; +DROP TABLE ${stats_db_name}.allsoftware purge; +DROP TABLE ${stats_db_name}.allpubsshare purge; +DROP TABLE ${stats_db_name}.alldatasetssshare purge; +DROP TABLE ${stats_db_name}.allsoftwaresshare purge; -ANALYZE TABLE indi_org_openess COMPUTE STATISTICS; - -CREATE TEMPORARY TABLE pubs_oa AS -SELECT r.year, ro.organization, count(distinct r.id) no_oapubs FROM publication r - join result_organization ro on ro.id=r.id - join result_instance ri on ri.id=r.id +CREATE TEMPORARY TABLE ${stats_db_name}.pubs_oa AS +SELECT r.year, ro.organization, count(distinct r.id) no_oapubs FROM ${stats_db_name}.publication r + join ${stats_db_name}.result_organization ro on ro.id=r.id + join ${stats_db_name}.result_instance ri on ri.id=r.id where (ri.accessright = 'Open Access' or ri.accessright = 'Embargo' or ri.accessright = 'Open Source') and cast(r.year as int)>2003 group by ro.organization,r.year; -CREATE TEMPORARY TABLE datasets_oa AS -SELECT r.year,ro.organization, count(distinct r.id) no_oadatasets FROM dataset r - join result_organization ro on ro.id=r.id - join result_instance ri on ri.id=r.id +CREATE TEMPORARY TABLE ${stats_db_name}.datasets_oa AS +SELECT r.year,ro.organization, count(distinct r.id) no_oadatasets FROM ${stats_db_name}.dataset r + join ${stats_db_name}.result_organization ro on ro.id=r.id + join ${stats_db_name}.result_instance ri on ri.id=r.id where (ri.accessright = 'Open Access' or ri.accessright = 'Embargo' or ri.accessright = 'Open Source') and cast(r.year as int)>2003 group by ro.organization, r.year; -CREATE TEMPORARY TABLE software_oa AS -SELECT r.year,ro.organization, count(distinct r.id) no_oasoftware FROM software r - join result_organization ro on ro.id=r.id - join result_instance ri on ri.id=r.id +CREATE TEMPORARY TABLE ${stats_db_name}.software_oa AS +SELECT r.year,ro.organization, count(distinct r.id) no_oasoftware FROM ${stats_db_name}.software r + join ${stats_db_name}.result_organization ro on ro.id=r.id + join ${stats_db_name}.result_instance ri on ri.id=r.id where (ri.accessright = 'Open Access' or ri.accessright = 'Embargo' or ri.accessright = 'Open Source') and cast(r.year as int)>2003 group by ro.organization, r.year; -CREATE TEMPORARY TABLE allpubs as -SELECT p.year,ro.organization organization, count(ro.id) no_allpubs FROM result_organization ro - join publication p on p.id=ro.id where cast(p.year as int)>2003 +CREATE TEMPORARY TABLE ${stats_db_name}.allpubs as +SELECT p.year,ro.organization organization, count(ro.id) no_allpubs FROM ${stats_db_name}.result_organization ro + join ${stats_db_name}.publication p on p.id=ro.id where cast(p.year as int)>2003 group by ro.organization, p.year; -CREATE TEMPORARY TABLE alldatasets as -SELECT d.year, ro.organization organization, count(ro.id) no_alldatasets FROM result_organization ro - join dataset d on d.id=ro.id where cast(d.year as int)>2003 +CREATE TEMPORARY TABLE ${stats_db_name}.alldatasets as +SELECT d.year, ro.organization organization, count(ro.id) no_alldatasets FROM ${stats_db_name}.result_organization ro + join ${stats_db_name}.dataset d on d.id=ro.id where cast(d.year as int)>2003 group by ro.organization, d.year; -CREATE TEMPORARY TABLE allsoftware as -SELECT s.year,ro.organization organization, count(ro.id) no_allsoftware FROM result_organization ro - join software s on s.id=ro.id where cast(s.year as int)>2003 +CREATE TEMPORARY TABLE ${stats_db_name}.allsoftware as +SELECT s.year,ro.organization organization, count(ro.id) no_allsoftware FROM ${stats_db_name}.result_organization ro + join ${stats_db_name}.software s on s.id=ro.id where cast(s.year as int)>2003 group by ro.organization, s.year; -CREATE TEMPORARY TABLE allpubsshare as -select allpubs.year, pubs_oa.organization, pubs_oa.no_oapubs/allpubs.no_allpubs p from allpubs - join pubs_oa on allpubs.organization=pubs_oa.organization where cast(allpubs.year as INT)=cast(pubs_oa.year as int); +CREATE TEMPORARY TABLE ${stats_db_name}.allpubsshare as +select allpubs.year, pubs_oa.organization, pubs_oa.no_oapubs/allpubs.no_allpubs p from ${stats_db_name}.allpubs + join ${stats_db_name}.pubs_oa on allpubs.organization=pubs_oa.organization where cast(allpubs.year as INT)=cast(pubs_oa.year as int); -CREATE TEMPORARY TABLE alldatasetssshare as +CREATE TEMPORARY TABLE ${stats_db_name}.alldatasetssshare as select alldatasets.year, datasets_oa.organization, datasets_oa.no_oadatasets/alldatasets.no_alldatasets d - from alldatasets - join datasets_oa on alldatasets.organization=datasets_oa.organization where cast(alldatasets.year as INT)=cast(datasets_oa.year as int); + from ${stats_db_name}.alldatasets + join ${stats_db_name}.datasets_oa on alldatasets.organization=datasets_oa.organization where cast(alldatasets.year as INT)=cast(datasets_oa.year as int); -CREATE TEMPORARY TABLE allsoftwaresshare as +CREATE TEMPORARY TABLE ${stats_db_name}.allsoftwaresshare as select allsoftware.year, software_oa.organization, software_oa.no_oasoftware/allsoftware.no_allsoftware s - from allsoftware - join software_oa on allsoftware.organization=software_oa.organization where cast(allsoftware.year as INT)=cast(software_oa.year as int); + from ${stats_db_name}.allsoftware + join ${stats_db_name}.software_oa on allsoftware.organization=software_oa.organization where cast(allsoftware.year as INT)=cast(software_oa.year as int); +drop table if exists ${stats_db_name}.indi_org_openess_year purge; -create table if not exists indi_org_openess_year stored as parquet as -select allpubsshare.year, allpubsshare.organization, +create table if not exists ${stats_db_name}.indi_org_openess_year stored as parquet as +select cast(allpubsshare.year as int) year, allpubsshare.organization, (p+if(isnull(s),0,s)+if(isnull(d),0,d))/(1+(case when s is null then 0 else 1 end) +(case when d is null then 0 else 1 end)) - org_openess FROM allpubsshare - left outer join (select year, organization,d from - alldatasetssshare) tmp1 + org_openess FROM ${stats_db_name}.allpubsshare + left outer join (select cast(year as int), organization,d from + ${stats_db_name}.alldatasetssshare) tmp1 on tmp1.organization=allpubsshare.organization and tmp1.year=allpubsshare.year - left outer join (select year, organization,s from - allsoftwaresshare) tmp2 - on tmp2.organization=allpubsshare.organization and tmp2.year=allpubsshare.year; + left outer join (select cast(year as int), organization,s from + ${stats_db_name}.allsoftwaresshare) tmp2 + on tmp2.organization=allpubsshare.organization and cast(tmp2.year as int)=cast(allpubsshare.year as int); -DROP TABLE pubs_oa purge; -DROP TABLE datasets_oa purge; -DROP TABLE software_oa purge; -DROP TABLE allpubs purge; -DROP TABLE alldatasets purge; -DROP TABLE allsoftware purge; -DROP TABLE allpubsshare purge; -DROP TABLE alldatasetssshare purge; -DROP TABLE allsoftwaresshare purge; +DROP TABLE ${stats_db_name}.pubs_oa purge; +DROP TABLE ${stats_db_name}.datasets_oa purge; +DROP TABLE ${stats_db_name}.software_oa purge; +DROP TABLE ${stats_db_name}.allpubs purge; +DROP TABLE ${stats_db_name}.alldatasets purge; +DROP TABLE ${stats_db_name}.allsoftware purge; +DROP TABLE ${stats_db_name}.allpubsshare purge; +DROP TABLE ${stats_db_name}.alldatasetssshare purge; +DROP TABLE ${stats_db_name}.allsoftwaresshare purge; -ANALYZE TABLE indi_org_openess_year COMPUTE STATISTICS; +drop table if exists ${stats_db_name}.indi_pub_has_preprint purge; -create table if not exists indi_pub_has_preprint stored as parquet as +create table if not exists ${stats_db_name}.indi_pub_has_preprint stored as parquet as select distinct p.id, coalesce(has_preprint, 0) as has_preprint -from publication_classifications p +from ${stats_db_name}.publication_classifications p left outer join ( select p.id, 1 as has_preprint - from publication_classifications p + from ${stats_db_name}.publication_classifications p where p.type='Preprint') tmp on p.id= tmp.id; +drop table if exists ${stats_db_name}.indi_pub_in_subscribed purge; -ANALYZE TABLE indi_pub_has_preprint COMPUTE STATISTICS; - -create table if not exists indi_pub_in_subscribed stored as parquet as +create table if not exists ${stats_db_name}.indi_pub_in_subscribed stored as parquet as select distinct p.id, coalesce(is_subscription, 0) as is_subscription -from publication p +from ${stats_db_name}.publication p left outer join( - select p.id, 1 as is_subscription from publication p - join indi_pub_gold_oa g on p.id=g.id - join indi_pub_hybrid h on p.id=h.id - join indi_pub_in_transformative t on p.id=t.id + select p.id, 1 as is_subscription from ${stats_db_name}.publication p + join ${stats_db_name}.indi_pub_gold_oa g on p.id=g.id + join ${stats_db_name}.indi_pub_hybrid h on p.id=h.id + join ${stats_db_name}.indi_pub_in_transformative t on p.id=t.id where g.is_gold=0 and h.is_hybrid=0 and t.is_transformative=0) tmp on p.id=tmp.id; -ANALYZE TABLE indi_pub_in_subscribed COMPUTE STATISTICS; +drop table if exists ${stats_db_name}.indi_result_with_pid purge; -create table if not exists indi_result_with_pid as +create table if not exists ${stats_db_name}.indi_result_with_pid as select distinct p.id, coalesce(result_with_pid, 0) as result_with_pid -from result p +from ${stats_db_name}.result p left outer join ( select p.id, 1 as result_with_pid - from result_pids p) tmp + from ${stats_db_name}.result_pids p) tmp on p.id= tmp.id; -ANALYZE TABLE indi_result_with_pid COMPUTE STATISTICS; - -CREATE TEMPORARY TABLE pub_fos_totals as -select rf.id, count(distinct lvl3) totals from result_fos rf +CREATE TEMPORARY TABLE ${stats_db_name}.pub_fos_totals as +select rf.id, count(distinct lvl3) totals from ${stats_db_name}.result_fos rf group by rf.id; -create table if not exists indi_pub_interdisciplinarity as +drop table if exists ${stats_db_name}.indi_pub_interdisciplinarity purge; + +create table if not exists ${stats_db_name}.indi_pub_interdisciplinarity as select distinct p.id as id, coalesce(is_interdisciplinary, 0) as is_interdisciplinary -from pub_fos_totals p +from ${stats_db_name}.pub_fos_totals p left outer join ( -select pub_fos_totals.id, 1 as is_interdisciplinary from pub_fos_totals +select pub_fos_totals.id, 1 as is_interdisciplinary from ${stats_db_name}.pub_fos_totals where totals>1) tmp on p.id=tmp.id; -drop table pub_fos_totals purge; +drop table ${stats_db_name}.pub_fos_totals purge; -ANALYZE TABLE indi_pub_interdisciplinarity COMPUTE STATISTICS; +drop table if exists ${stats_db_name}.indi_pub_bronze_oa purge; -create table if not exists indi_pub_bronze_oa stored as parquet as -select distinct p.id, coalesce(is_bronze_oa,0) as is_bronze_oa -from publication p -left outer join -(select p.id, 1 as is_bronze_oa from publication p -join indi_result_has_cc_licence cc on cc.id=p.id -join indi_pub_gold_oa ga on ga.id=p.id -where cc.has_cc_license=0 and ga.is_gold=0) tmp on tmp.id=p.id; +--create table if not exists ${stats_db_name}.indi_pub_bronze_oa stored as parquet as +--select distinct p.id, coalesce(is_bronze_oa,0) as is_bronze_oa +--from ${stats_db_name}.publication p +--left outer join +--(select p.id, 1 as is_bronze_oa from ${stats_db_name}.publication p +--join ${stats_db_name}.indi_result_has_cc_licence cc on cc.id=p.id +--join ${stats_db_name}.indi_pub_gold_oa ga on ga.id=p.id +--join ${stats_db_name}.result_instance ri on ri.id=p.id +--join ${stats_db_name}.datasource d on d.id=ri.hostedby +--where cc.has_cc_license=0 and ga.is_gold=0 +--and (d.type='Journal' or d.type='Journal Aggregator/Publisher') +--and ri.accessright='Open Access') tmp on tmp.id=p.id; --- create table if not exists indi_pub_bronze_oa stored as parquet as --- WITH hybrid_oa AS ( --- SELECT issn_l, journal_is_in_doaj, journal_is_oa, issn_print as issn --- FROM STATS_EXT.plan_s_jn --- WHERE issn_print != "" --- UNION ALL --- SELECT issn_l, journal_is_in_doaj, journal_is_oa, issn_online as issn --- FROM STATS_EXT.plan_s_jn --- WHERE issn_online != "" and (journal_is_in_doaj = FALSE OR journal_is_oa = FALSE)), --- issn AS ( --- SELECT * --- FROM ( --- SELECT id, issn_printed as issn --- FROM datasource --- WHERE issn_printed IS NOT NULL --- UNION ALL --- SELECT id,issn_online as issn --- FROM datasource --- WHERE issn_online IS NOT NULL ) as issn --- WHERE LENGTH(issn) > 7) ---SELECT DISTINCT pd.id, coalesce(is_bronze_oa, 0) as is_bronze_oa ---FROM publication_datasources pd --- LEFT OUTER JOIN ( --- SELECT pd.id, 1 as is_bronze_oa from publication_datasources pd --- JOIN datasource d on d.id=pd.datasource --- JOIN issn on issn.id=pd.datasource --- JOIN hybrid_oa ON issn.issn = hybrid_oa.issn --- JOIN indi_result_has_cc_licence cc on pd.id=cc.id --- JOIN indi_pub_gold_oa ga on pd.id=ga.id --- JOIN indi_pub_hybrid_oa_with_cc hy on hy.id=pd.id --- where cc.has_cc_license=0 and ga.is_gold=0 and hy.is_hybrid_oa=0) tmp on pd.id=tmp.id; +create table ${stats_db_name}.indi_pub_bronze stored as parquet as +select pd.id,coalesce(is_bronze_oa,0) is_bronze_oa from ${stats_db_name}.publication_datasources pd +left outer join (select pd.id, 1 as is_bronze_oa from ${stats_db_name}.publication_datasources pd +join ${stats_db_name}.datasource d on pd.datasource=d.id +join ${stats_db_name}.result_instance ri on ri.id=pd.id +join ${stats_db_name}.indi_pub_gold_oa indi_gold on indi_gold.id=pd.id +join ${stats_db_name}.result_accessroute ra on ra.id=pd.id +where d.type like '%Journal%' and ri.accessright!='Closed Access' and (ri.accessright_uw!='gold' +or indi_gold.is_gold=0) and (ra.accessroute='bronze' or ri.license is null)) tmp +on pd.id=tmp.id; + +CREATE TEMPORARY TABLE ${stats_db_name}.project_year_result_year as +select p.id project_id, acronym, r.id result_id, r.year, p.end_year +from ${stats_db_name}.project p +join ${stats_db_name}.result_projects rp on p.id=rp.project +join ${stats_db_name}.result r on r.id=rp.id +where p.end_year is NOT NULL and r.year is not null; + +drop table if exists ${stats_db_name}.indi_is_project_result_after purge; + +create table if not exists ${stats_db_name}.indi_is_project_result_after stored as parquet as +select pry.project_id, pry.acronym, pry.result_id, +coalesce(is_project_result_after, 0) as is_project_result_after +from ${stats_db_name}.project_year_result_year pry +left outer join (select pry.project_id, pry.acronym, pry.result_id, 1 as is_project_result_after +from ${stats_db_name}.project_year_result_year pry +where pry.year>pry.end_year) tmp on pry.result_id=tmp.result_id; + +drop table ${stats_db_name}.project_year_result_year purge; + +drop table ${stats_db_name}.indi_is_funder_plan_s purge; + +create table if not exists ${stats_db_name}.indi_is_funder_plan_s stored as parquet as +select distinct f.id, f.name, coalesce(is_funder_plan_s, 0) as is_funder_plan_s +from ${stats_db_name}.funder f + left outer join (select id, name, 1 as is_funder_plan_s from ${stats_db_name}.funder + join stats_ext.plan_s_short on c_o_alition_s_organisation_funder=name) tmp + on f.name= tmp.name; + +--Funder Fairness +drop table ${stats_db_name}.indi_funder_fairness purge; + +create table if not exists ${stats_db_name}.indi_funder_fairness stored as parquet as + with result_fair as + (select p.funder funder, count(distinct rp.id) no_result_fair from ${stats_db_name}.result_projects rp + join ${stats_db_name}.result r on r.id=rp.id + join ${stats_db_name}.project p on p.id=rp.project + where (r.title is not null) and (publisher is not null) and (abstract=true) and (year is not null) and (authors>0) and cast(year as int)>2003 + group by p.funder), + allresults as (select p.funder funder, count(distinct rp.id) no_allresults from ${stats_db_name}.result_projects rp + join ${stats_db_name}.result r on r.id=rp.id + join ${stats_db_name}.project p on p.id=rp.project + where cast(year as int)>2003 + group by p.funder) +select allresults.funder, result_fair.no_result_fair/allresults.no_allresults funder_fairness +from allresults + join result_fair on result_fair.funder=allresults.funder; + +--RIs Fairness +drop table ${stats_db_name}.indi_ris_fairness purge; + +create table if not exists ${stats_db_name}.indi_ris_fairness stored as parquet as +with result_contexts as +(select distinct rc.id, context.name ri_initiative from ${stats_db_name}.result_concepts rc +join ${stats_db_name}.concept on concept.id=rc.concept +join ${stats_db_name}.category on category.id=concept.category +join ${stats_db_name}.context on context.id=category.context), +result_fair as + (select rc.ri_initiative ri_initiative, count(distinct rc.id) no_result_fair from result_contexts rc + join ${stats_db_name}.result r on r.id=rc.id + where (title is not null) and (publisher is not null) and (abstract=true) and (year is not null) and (authors>0) and cast(year as int)>2003 + group by rc.ri_initiative), +allresults as +(select rc.ri_initiative ri_initiative, count(distinct rc.id) no_allresults from result_contexts rc + join ${stats_db_name}.result r on r.id=rc.id + where cast(year as int)>2003 + group by rc.ri_initiative) +select allresults.ri_initiative, result_fair.no_result_fair/allresults.no_allresults ris_fairness +from allresults + join result_fair on result_fair.ri_initiative=allresults.ri_initiative; + +--Funder Openess + +CREATE TEMPORARY TABLE ${stats_db_name}.pubs_oa as +select p.funder funder, count(distinct rp.id) no_oapubs from ${stats_db_name}.result_projects rp +join ${stats_db_name}.project p on p.id=rp.project +join ${stats_db_name}.publication r on r.id=rp.id +join ${stats_db_name}.result_instance ri on ri.id=r.id +where (ri.accessright = 'Open Access' or ri.accessright = 'Embargo' or ri.accessright = 'Open Source') +and cast(r.year as int)>2003 +group by p.funder; + + +CREATE TEMPORARY TABLE ${stats_db_name}.datasets_oa as +select p.funder funder, count(distinct rp.id) no_oadatasets from ${stats_db_name}.result_projects rp +join ${stats_db_name}.project p on p.id=rp.project +join ${stats_db_name}.dataset r on r.id=rp.id +join ${stats_db_name}.result_instance ri on ri.id=r.id +where (ri.accessright = 'Open Access' or ri.accessright = 'Embargo' or ri.accessright = 'Open Source') +and cast(r.year as int)>2003 +group by p.funder; + +CREATE TEMPORARY TABLE ${stats_db_name}.software_oa as +select p.funder funder, count(distinct rp.id) no_oasoftware from ${stats_db_name}.result_projects rp +join ${stats_db_name}.project p on p.id=rp.project +join ${stats_db_name}.software r on r.id=rp.id +join ${stats_db_name}.result_instance ri on ri.id=r.id +where (ri.accessright = 'Open Access' or ri.accessright = 'Embargo' or ri.accessright = 'Open Source') +and cast(r.year as int)>2003 +group by p.funder; + +CREATE TEMPORARY TABLE ${stats_db_name}.allpubs as +select p.funder funder, count(distinct rp.id) no_allpubs from ${stats_db_name}.result_projects rp +join ${stats_db_name}.project p on p.id=rp.project +join ${stats_db_name}.publication r on r.id=rp.id +where cast(r.year as int)>2003 +group by p.funder; + +CREATE TEMPORARY TABLE ${stats_db_name}.alldatasets as +select p.funder funder, count(distinct rp.id) no_alldatasets from ${stats_db_name}.result_projects rp +join ${stats_db_name}.project p on p.id=rp.project +join ${stats_db_name}.dataset r on r.id=rp.id +where cast(r.year as int)>2003 +group by p.funder; + +CREATE TEMPORARY TABLE ${stats_db_name}.allsoftware as +select p.funder funder, count(distinct rp.id) no_allsoftware from ${stats_db_name}.result_projects rp +join ${stats_db_name}.project p on p.id=rp.project +join ${stats_db_name}.software r on r.id=rp.id +where cast(r.year as int)>2003 +group by p.funder; + +CREATE TEMPORARY TABLE ${stats_db_name}.allpubsshare as +select pubs_oa.funder, pubs_oa.no_oapubs/allpubs.no_allpubs p from ${stats_db_name}.allpubs + join ${stats_db_name}.pubs_oa on allpubs.funder=pubs_oa.funder; + +CREATE TEMPORARY TABLE ${stats_db_name}.alldatasetssshare as +select datasets_oa.funder, datasets_oa.no_oadatasets/alldatasets.no_alldatasets d + from ${stats_db_name}.alldatasets + join ${stats_db_name}.datasets_oa on alldatasets.funder=datasets_oa.funder; + +CREATE TEMPORARY TABLE ${stats_db_name}.allsoftwaresshare as +select software_oa.funder, software_oa.no_oasoftware/allsoftware.no_allsoftware s + from ${stats_db_name}.allsoftware + join ${stats_db_name}.software_oa on allsoftware.funder=software_oa.funder; + +drop table ${stats_db_name}.indi_funder_openess purge; + +create table if not exists ${stats_db_name}.indi_funder_openess stored as parquet as +select allpubsshare.funder, + (p+if(isnull(s),0,s)+if(isnull(d),0,d))/(1+(case when s is null then 0 else 1 end) + +(case when d is null then 0 else 1 end)) + funder_openess FROM ${stats_db_name}.allpubsshare + left outer join (select funder,d from + ${stats_db_name}.alldatasetssshare) tmp1 + on tmp1.funder=allpubsshare.funder + left outer join (select funder,s from + ${stats_db_name}.allsoftwaresshare) tmp2 + on tmp2.funder=allpubsshare.funder; + +DROP TABLE ${stats_db_name}.pubs_oa purge; +DROP TABLE ${stats_db_name}.datasets_oa purge; +DROP TABLE ${stats_db_name}.software_oa purge; +DROP TABLE ${stats_db_name}.allpubs purge; +DROP TABLE ${stats_db_name}.alldatasets purge; +DROP TABLE ${stats_db_name}.allsoftware purge; +DROP TABLE ${stats_db_name}.allpubsshare purge; +DROP TABLE ${stats_db_name}.alldatasetssshare purge; +DROP TABLE ${stats_db_name}.allsoftwaresshare purge; + +--RIs Openess + +CREATE TEMPORARY TABLE ${stats_db_name}.result_contexts as +select distinct rc.id, context.name ri_initiative from ${stats_db_name}.result_concepts rc +join ${stats_db_name}.concept on concept.id=rc.concept +join ${stats_db_name}.category on category.id=concept.category +join ${stats_db_name}.context on context.id=category.context; + +CREATE TEMPORARY TABLE ${stats_db_name}.pubs_oa as +select rp.ri_initiative ri_initiative, count(distinct rp.id) no_oapubs from ${stats_db_name}.result_contexts rp +join ${stats_db_name}.publication r on r.id=rp.id +join ${stats_db_name}.result_instance ri on ri.id=r.id +where (ri.accessright = 'Open Access' or ri.accessright = 'Embargo' or ri.accessright = 'Open Source') +and cast(r.year as int)>2003 +group by rp.ri_initiative; + +CREATE TEMPORARY TABLE ${stats_db_name}.datasets_oa as +select rp.ri_initiative ri_initiative, count(distinct rp.id) no_oadatasets from ${stats_db_name}.result_contexts rp +join ${stats_db_name}.dataset r on r.id=rp.id +join ${stats_db_name}.result_instance ri on ri.id=r.id +where (ri.accessright = 'Open Access' or ri.accessright = 'Embargo' or ri.accessright = 'Open Source') +and cast(r.year as int)>2003 +group by rp.ri_initiative; + +CREATE TEMPORARY TABLE ${stats_db_name}.software_oa as +select rp.ri_initiative ri_initiative, count(distinct rp.id) no_oasoftware from ${stats_db_name}.result_contexts rp +join ${stats_db_name}.software r on r.id=rp.id +join ${stats_db_name}.result_instance ri on ri.id=r.id +where (ri.accessright = 'Open Access' or ri.accessright = 'Embargo' or ri.accessright = 'Open Source') +and cast(r.year as int)>2003 +group by rp.ri_initiative; + +CREATE TEMPORARY TABLE ${stats_db_name}.allpubs as +select rp.ri_initiative ri_initiative, count(distinct rp.id) no_allpubs from ${stats_db_name}.result_contexts rp +join ${stats_db_name}.publication r on r.id=rp.id +where cast(r.year as int)>2003 +group by rp.ri_initiative; + +CREATE TEMPORARY TABLE ${stats_db_name}.alldatasets as +select rp.ri_initiative ri_initiative, count(distinct rp.id) no_alldatasets from ${stats_db_name}.result_contexts rp +join ${stats_db_name}.dataset r on r.id=rp.id +where cast(r.year as int)>2003 +group by rp.ri_initiative; + +CREATE TEMPORARY TABLE ${stats_db_name}.allsoftware as +select rp.ri_initiative ri_initiative, count(distinct rp.id) no_allsoftware from ${stats_db_name}.result_contexts rp +join ${stats_db_name}.software r on r.id=rp.id +where cast(r.year as int)>2003 +group by rp.ri_initiative; + +CREATE TEMPORARY TABLE ${stats_db_name}.allpubsshare as +select pubs_oa.ri_initiative, pubs_oa.no_oapubs/allpubs.no_allpubs p from ${stats_db_name}.allpubs + join ${stats_db_name}.pubs_oa on allpubs.ri_initiative=pubs_oa.ri_initiative; + +CREATE TEMPORARY TABLE ${stats_db_name}.alldatasetssshare as +select datasets_oa.ri_initiative, datasets_oa.no_oadatasets/alldatasets.no_alldatasets d + from ${stats_db_name}.alldatasets + join ${stats_db_name}.datasets_oa on alldatasets.ri_initiative=datasets_oa.ri_initiative; + +CREATE TEMPORARY TABLE ${stats_db_name}.allsoftwaresshare as +select software_oa.ri_initiative, software_oa.no_oasoftware/allsoftware.no_allsoftware s + from ${stats_db_name}.allsoftware + join ${stats_db_name}.software_oa on allsoftware.ri_initiative=software_oa.ri_initiative; + +drop table ${stats_db_name}.indi_ris_openess purge; + +create table if not exists ${stats_db_name}.indi_ris_openess stored as parquet as +select allpubsshare.ri_initiative, + (p+if(isnull(s),0,s)+if(isnull(d),0,d))/(1+(case when s is null then 0 else 1 end) + +(case when d is null then 0 else 1 end)) + ris_openess FROM ${stats_db_name}.allpubsshare + left outer join (select ri_initiative,d from + ${stats_db_name}.alldatasetssshare) tmp1 + on tmp1.ri_initiative=allpubsshare.ri_initiative + left outer join (select ri_initiative,s from + ${stats_db_name}.allsoftwaresshare) tmp2 + on tmp2.ri_initiative=allpubsshare.ri_initiative; + +DROP TABLE ${stats_db_name}.result_contexts purge; +DROP TABLE ${stats_db_name}.pubs_oa purge; +DROP TABLE ${stats_db_name}.datasets_oa purge; +DROP TABLE ${stats_db_name}.software_oa purge; +DROP TABLE ${stats_db_name}.allpubs purge; +DROP TABLE ${stats_db_name}.alldatasets purge; +DROP TABLE ${stats_db_name}.allsoftware purge; +DROP TABLE ${stats_db_name}.allpubsshare purge; +DROP TABLE ${stats_db_name}.alldatasetssshare purge; +DROP TABLE ${stats_db_name}.allsoftwaresshare purge; + +--Funder Findability +drop table ${stats_db_name}.indi_funder_findable purge; + +create table if not exists ${stats_db_name}.indi_funder_findable stored as parquet as +with result_findable as + (select p.funder funder, count(distinct rp.id) no_result_findable from ${stats_db_name}.result_projects rp + join ${stats_db_name}.publication r on r.id=rp.id + join ${stats_db_name}.project p on p.id=rp.project + join ${stats_db_name}.result_pids rpi on rpi.id=r.id + where cast(year as int)>2003 + group by p.funder), + allresults as (select p.funder funder, count(distinct rp.id) no_allresults from ${stats_db_name}.result_projects rp + join ${stats_db_name}.result r on r.id=rp.id + join ${stats_db_name}.project p on p.id=rp.project + where cast(year as int)>2003 + group by p.funder) +select allresults.funder, result_findable.no_result_findable/allresults.no_allresults funder_findable +from allresults + join result_findable on result_findable.funder=allresults.funder; + +--RIs Findability +drop table ${stats_db_name}.indi_ris_findable purge; + +create table if not exists ${stats_db_name}.indi_ris_findable stored as parquet as +with result_contexts as +(select distinct rc.id, context.name ri_initiative from ${stats_db_name}.result_concepts rc +join ${stats_db_name}.concept on concept.id=rc.concept +join ${stats_db_name}.category on category.id=concept.category +join ${stats_db_name}.context on context.id=category.context), +result_findable as + (select rc.ri_initiative ri_initiative, count(distinct rc.id) no_result_findable from result_contexts rc + join ${stats_db_name}.result r on r.id=rc.id + join ${stats_db_name}.result_pids rp on rp.id=r.id + where cast(r.year as int)>2003 + group by rc.ri_initiative), +allresults as +(select rc.ri_initiative ri_initiative, count(distinct rc.id) no_allresults from result_contexts rc + join ${stats_db_name}.result r on r.id=rc.id + where cast(r.year as int)>2003 + group by rc.ri_initiative) +select allresults.ri_initiative, result_findable.no_result_findable/allresults.no_allresults ris_findable +from allresults + join result_findable on result_findable.ri_initiative=allresults.ri_initiative; -ANALYZE TABLE indi_pub_bronze_oa COMPUTE STATISTICS; \ 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/step20-createMonitorDB.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql index 9744d5aae..d5d242230 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql @@ -88,85 +88,88 @@ create view if not exists TARGET.doctoratestudents as select * from SOURCE.docto create view if not exists TARGET.totalresearchers as select * from SOURCE.totalresearchers; create view if not exists TARGET.totalresearchersft as select * from SOURCE.totalresearchersft; create view if not exists TARGET.hrrst as select * from SOURCE.hrrst; +create view if not exists TARGET.graduatedoctorates as select * from SOURCE.graduatedoctorates; create table TARGET.result_citations stored as parquet as select * from SOURCE.result_citations orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_citations COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_citations COMPUTE STATISTICS; create table TARGET.result_references_oc stored as parquet as select * from SOURCE.result_references_oc orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_references_oc COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_references_oc COMPUTE STATISTICS; create table TARGET.result_citations_oc stored as parquet as select * from SOURCE.result_citations_oc orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_citations_oc COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_citations_oc COMPUTE STATISTICS; create table TARGET.result_classifications stored as parquet as select * from SOURCE.result_classifications orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_classifications COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_classifications COMPUTE STATISTICS; create table TARGET.result_apc stored as parquet as select * from SOURCE.result_apc orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_apc COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_apc COMPUTE STATISTICS; create table TARGET.result_concepts stored as parquet as select * from SOURCE.result_concepts orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_concepts COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_concepts COMPUTE STATISTICS; create table TARGET.result_datasources stored as parquet as select * from SOURCE.result_datasources orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_datasources COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_datasources COMPUTE STATISTICS; create table TARGET.result_fundercount stored as parquet as select * from SOURCE.result_fundercount orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_fundercount COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_fundercount COMPUTE STATISTICS; create table TARGET.result_gold stored as parquet as select * from SOURCE.result_gold orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_gold COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_gold COMPUTE STATISTICS; create table TARGET.result_greenoa stored as parquet as select * from SOURCE.result_greenoa orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_greenoa COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_greenoa COMPUTE STATISTICS; create table TARGET.result_languages stored as parquet as select * from SOURCE.result_languages orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_languages COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_languages COMPUTE STATISTICS; create table TARGET.result_licenses stored as parquet as select * from SOURCE.result_licenses orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_licenses COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_licenses COMPUTE STATISTICS; create table TARGET.licenses_normalized STORED AS PARQUET as select * from SOURCE.licenses_normalized; -ANALYZE TABLE TARGET.licenses_normalized COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.licenses_normalized COMPUTE STATISTICS; create table TARGET.result_oids stored as parquet as select * from SOURCE.result_oids orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_oids COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_oids COMPUTE STATISTICS; create table TARGET.result_organization stored as parquet as select * from SOURCE.result_organization orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_organization COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_organization COMPUTE STATISTICS; create table TARGET.result_peerreviewed stored as parquet as select * from SOURCE.result_peerreviewed orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_peerreviewed COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_peerreviewed COMPUTE STATISTICS; create table TARGET.result_pids stored as parquet as select * from SOURCE.result_pids orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_pids COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_pids COMPUTE STATISTICS; create table TARGET.result_projectcount stored as parquet as select * from SOURCE.result_projectcount orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_projectcount COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_projectcount COMPUTE STATISTICS; create table TARGET.result_projects stored as parquet as select * from SOURCE.result_projects orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_projects COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_projects COMPUTE STATISTICS; create table TARGET.result_refereed stored as parquet as select * from SOURCE.result_refereed orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_refereed COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_refereed COMPUTE STATISTICS; create table TARGET.result_sources stored as parquet as select * from SOURCE.result_sources orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_sources COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_sources COMPUTE STATISTICS; create table TARGET.result_topics stored as parquet as select * from SOURCE.result_topics orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_topics COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_topics COMPUTE STATISTICS; create table TARGET.result_fos stored as parquet as select * from SOURCE.result_fos orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_fos COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_fos COMPUTE STATISTICS; create table TARGET.result_accessroute stored as parquet as select * from SOURCE.result_accessroute orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.result_accessroute COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_accessroute COMPUTE STATISTICS; + +create table TARGET.result_orcid stored as parquet as select * from SOURCE.result_orcid orig where exists (select 1 from TARGET.result r where r.id=orig.id); create view TARGET.foo1 as select * from SOURCE.result_result rr where rr.source in (select id from TARGET.result); create view TARGET.foo2 as select * from SOURCE.result_result rr where rr.target in (select id from TARGET.result); create table TARGET.result_result STORED AS PARQUET as select distinct * from (select * from TARGET.foo1 union all select * from TARGET.foo2) foufou; drop view TARGET.foo1; drop view TARGET.foo2; -ANALYZE TABLE TARGET.result_result COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.result_result COMPUTE STATISTICS; -- datasources create view if not exists TARGET.datasource as select * from SOURCE.datasource; @@ -175,7 +178,7 @@ create view if not exists TARGET.datasource_organizations as select * from SOURC create view if not exists TARGET.datasource_sources as select * from SOURCE.datasource_sources; create table TARGET.datasource_results stored as parquet as select id as result, datasource as id from TARGET.result_datasources; -ANALYZE TABLE TARGET.datasource_results COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.datasource_results COMPUTE STATISTICS; -- organizations create view if not exists TARGET.organization as select * from SOURCE.organization; @@ -193,28 +196,28 @@ create view if not exists TARGET.project_classification as select * from SOURCE. create view if not exists TARGET.project_organization_contribution as select * from SOURCE.project_organization_contribution; create table TARGET.project_results stored as parquet as select id as result, project as id from TARGET.result_projects; -ANALYZE TABLE TARGET.project_results COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.project_results COMPUTE STATISTICS; -- indicators -- Sprint 1 ---- create table TARGET.indi_pub_green_oa stored as parquet as select * from SOURCE.indi_pub_green_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_pub_green_oa COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_pub_green_oa COMPUTE STATISTICS; create table TARGET.indi_pub_grey_lit stored as parquet as select * from SOURCE.indi_pub_grey_lit orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_pub_grey_lit COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_pub_grey_lit COMPUTE STATISTICS; create table TARGET.indi_pub_doi_from_crossref stored as parquet as select * from SOURCE.indi_pub_doi_from_crossref orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_pub_doi_from_crossref COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_pub_doi_from_crossref COMPUTE STATISTICS; -- Sprint 2 ---- create table TARGET.indi_result_has_cc_licence stored as parquet as select * from SOURCE.indi_result_has_cc_licence orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_result_has_cc_licence COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_result_has_cc_licence COMPUTE STATISTICS; create table TARGET.indi_result_has_cc_licence_url stored as parquet as select * from SOURCE.indi_result_has_cc_licence_url orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_result_has_cc_licence_url COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_result_has_cc_licence_url COMPUTE STATISTICS; create table TARGET.indi_pub_has_abstract stored as parquet as select * from SOURCE.indi_pub_has_abstract orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_pub_has_abstract COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_pub_has_abstract COMPUTE STATISTICS; create table TARGET.indi_result_with_orcid stored as parquet as select * from SOURCE.indi_result_with_orcid orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_result_with_orcid COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_result_with_orcid COMPUTE STATISTICS; ---- Sprint 3 ---- create table TARGET.indi_funded_result_with_fundref stored as parquet as select * from SOURCE.indi_funded_result_with_fundref orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_funded_result_with_fundref COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_funded_result_with_fundref COMPUTE STATISTICS; create view TARGET.indi_result_org_collab as select * from SOURCE.indi_result_org_collab; create view TARGET.indi_result_org_country_collab as select * from SOURCE.indi_result_org_country_collab; create view TARGET.indi_project_collab_org as select * from SOURCE.indi_project_collab_org; @@ -223,32 +226,32 @@ create view TARGET.indi_funder_country_collab as select * from SOURCE.indi_funde create view TARGET.indi_result_country_collab as select * from SOURCE.indi_result_country_collab; ---- Sprint 4 ---- create table TARGET.indi_pub_diamond stored as parquet as select * from SOURCE.indi_pub_diamond orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_pub_diamond COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_pub_diamond COMPUTE STATISTICS; create table TARGET.indi_pub_in_transformative stored as parquet as select * from SOURCE.indi_pub_in_transformative orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_pub_in_transformative COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_pub_in_transformative COMPUTE STATISTICS; create table TARGET.indi_pub_closed_other_open stored as parquet as select * from SOURCE.indi_pub_closed_other_open orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_pub_closed_other_open COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_pub_closed_other_open COMPUTE STATISTICS; ---- Sprint 5 ---- create table TARGET.indi_result_no_of_copies stored as parquet as select * from SOURCE.indi_result_no_of_copies orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_result_no_of_copies COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_result_no_of_copies COMPUTE STATISTICS; ---- Sprint 6 ---- create table TARGET.indi_pub_hybrid_oa_with_cc stored as parquet as select * from SOURCE.indi_pub_hybrid_oa_with_cc orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_pub_hybrid_oa_with_cc COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_pub_hybrid_oa_with_cc COMPUTE STATISTICS; create table TARGET.indi_pub_bronze_oa stored as parquet as select * from SOURCE.indi_pub_bronze_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_pub_bronze_oa COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_pub_bronze_oa COMPUTE STATISTICS; create table TARGET.indi_pub_downloads stored as parquet as select * from SOURCE.indi_pub_downloads orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); -ANALYZE TABLE TARGET.indi_pub_downloads COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_pub_downloads COMPUTE STATISTICS; create table TARGET.indi_pub_downloads_datasource stored as parquet as select * from SOURCE.indi_pub_downloads_datasource orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); -ANALYZE TABLE TARGET.indi_pub_downloads_datasource COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_pub_downloads_datasource COMPUTE STATISTICS; create table TARGET.indi_pub_downloads_year stored as parquet as select * from SOURCE.indi_pub_downloads_year orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); -ANALYZE TABLE TARGET.indi_pub_downloads_year COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_pub_downloads_year COMPUTE STATISTICS; create table TARGET.indi_pub_downloads_datasource_year stored as parquet as select * from SOURCE.indi_pub_downloads_datasource_year orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); -ANALYZE TABLE TARGET.indi_pub_downloads_datasource_year COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_pub_downloads_datasource_year COMPUTE STATISTICS; ---- Sprint 7 ---- create table TARGET.indi_pub_gold_oa stored as parquet as select * from SOURCE.indi_pub_gold_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_pub_gold_oa COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_pub_gold_oa COMPUTE STATISTICS; create table TARGET.indi_pub_hybrid stored as parquet as select * from SOURCE.indi_pub_hybrid orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_pub_hybrid COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_pub_hybrid COMPUTE STATISTICS; create view TARGET.indi_org_fairness as select * from SOURCE.indi_org_fairness; create view TARGET.indi_org_fairness_pub_pr as select * from SOURCE.indi_org_fairness_pub_pr; create view TARGET.indi_org_fairness_pub_year as select * from SOURCE.indi_org_fairness_pub_year; @@ -259,12 +262,22 @@ create view TARGET.indi_org_findable as select * from SOURCE.indi_org_findable; create view TARGET.indi_org_openess as select * from SOURCE.indi_org_openess; create view TARGET.indi_org_openess_year as select * from SOURCE.indi_org_openess_year; create table TARGET.indi_pub_has_preprint stored as parquet as select * from SOURCE.indi_pub_has_preprint orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_pub_has_preprint COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_pub_has_preprint COMPUTE STATISTICS; create table TARGET.indi_pub_in_subscribed stored as parquet as select * from SOURCE.indi_pub_in_subscribed orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_pub_in_subscribed COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_pub_in_subscribed COMPUTE STATISTICS; create table TARGET.indi_result_with_pid stored as parquet as select * from SOURCE.indi_result_with_pid orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_result_with_pid COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_result_with_pid COMPUTE STATISTICS; create table TARGET.indi_impact_measures stored as parquet as select * from SOURCE.indi_impact_measures orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_impact_measures COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_impact_measures COMPUTE STATISTICS; create table TARGET.indi_pub_interdisciplinarity stored as parquet as select * from SOURCE.indi_pub_interdisciplinarity orig where exists (select 1 from TARGET.result r where r.id=orig.id); -ANALYZE TABLE TARGET.indi_pub_interdisciplinarity COMPUTE STATISTICS; +--ANALYZE TABLE TARGET.indi_pub_interdisciplinarity COMPUTE STATISTICS; +create table TARGET.result_apc_affiliations stored as parquet as select * from SOURCE.result_apc_affiliations orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_apc_affiliations COMPUTE STATISTICS; +create table TARGET.indi_is_project_result_after stored as parquet as select * from SOURCE.indi_is_project_result_after orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); +create table TARGET.indi_is_funder_plan_s stored as parquet as select * from SOURCE.indi_is_funder_plan_s orig where exists (select 1 from TARGET.result r where r.id=orig.id); +create view TARGET.indi_funder_fairness as select * from SOURCE.indi_funder_fairness; +create view TARGET.indi_funder_openess as select * from SOURCE.indi_funder_openess; +create view TARGET.indi_funder_findable as select * from SOURCE.indi_funder_findable; +create view TARGET.indi_ris_fairness as select * from SOURCE.indi_ris_fairness; +create view TARGET.indi_ris_openess as select * from SOURCE.indi_ris_openess; +create view TARGET.indi_ris_findable as select * from SOURCE.indi_ris_findable; \ 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/step20-createMonitorDBAll.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql new file mode 100644 index 000000000..df4795e3e --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql @@ -0,0 +1,292 @@ +drop database if exists TARGET cascade; +create database if not exists TARGET; + +create view if not exists TARGET.category as select * from SOURCE.category; +create view if not exists TARGET.concept as select * from SOURCE.concept; +create view if not exists TARGET.context as select * from SOURCE.context; +create view if not exists TARGET.country as select * from SOURCE.country; +create view if not exists TARGET.countrygdp as select * from SOURCE.countrygdp; +create view if not exists TARGET.creation_date as select * from SOURCE.creation_date; +create view if not exists TARGET.funder as select * from SOURCE.funder; +create view if not exists TARGET.fundref as select * from SOURCE.fundref; +create view if not exists TARGET.rndexpenditure as select * from SOURCE.rndexpediture; +create view if not exists TARGET.rndgdpexpenditure as select * from SOURCE.rndgdpexpenditure; +create view if not exists TARGET.doctoratestudents as select * from SOURCE.doctoratestudents; +create view if not exists TARGET.totalresearchers as select * from SOURCE.totalresearchers; +create view if not exists TARGET.totalresearchersft as select * from SOURCE.totalresearchersft; +create view if not exists TARGET.hrrst as select * from SOURCE.hrrst; +create view if not exists TARGET.graduatedoctorates as select * from SOURCE.graduatedoctorates; + +create table TARGET.result stored as parquet as + select distinct * from ( + select * from SOURCE.result r where exists (select 1 from SOURCE.result_projects rp join SOURCE.project p on rp.project=p.id where rp.id=r.id) + union all + select * from SOURCE.result r where exists (select 1 from SOURCE.result_concepts rc where rc.id=r.id) + union all + select * from SOURCE.result r where exists (select 1 from SOURCE.result_organization ro where ro.id=r.id and ro.organization in ( + 'openorgs____::b84450f9864182c67b8611b5593f4250', --"Athena Research and Innovation Center In Information Communication & Knowledge Technologies', --ARC" + 'openorgs____::d41cf6bd4ab1b1362a44397e0b95c975', --National Research Council + 'openorgs____::d2a09b9d5eabb10c95f9470e172d05d2', --??? Not exists ?? + 'openorgs____::d169c7407dd417152596908d48c11460', --Masaryk University + 'openorgs____::1ec924b1759bb16d0a02f2dad8689b21', --University of Belgrade + 'openorgs____::0ae431b820e4c33db8967fbb2b919150', --University of Helsinki + 'openorgs____::759d59f05d77188faee99b7493b46805', --University of Minho + 'openorgs____::cad284878801b9465fa51a95b1d779db', --Universidad Politécnica de Madrid + 'openorgs____::eadc8da90a546e98c03f896661a2e4d4', --University of Göttingen + 'openorgs____::c0286313e36479eff8676dba9b724b40', --National and Kapodistrian University of Athens + -- 'openorgs____::c80a8243a5e5c620d7931c88d93bf17a', --Université Paris Diderot + 'openorgs____::c08634f0a6b0081c3dc6e6c93a4314f3', --Bielefeld University + 'openorgs____::6fc85e4a8f7ecaf4b0c738d010e967ea', --University of Southern Denmark + 'openorgs____::3d6122f87f9a97a99d8f6e3d73313720', --Humboldt-Universität zu Berlin + 'openorgs____::16720ada63d0fa8ca41601feae7d1aa5', --TU Darmstadt + 'openorgs____::ccc0a066b56d2cfaf90c2ae369df16f5', --KU Leuven + 'openorgs____::4c6f119632adf789746f0a057ed73e90', --University of the Western Cape + 'openorgs____::ec3665affa01aeafa28b7852c4176dbd', --Rudjer Boskovic Institute + 'openorgs____::5f31346d444a7f06a28c880fb170b0f6', --Ghent University + 'openorgs____::2dbe47117fd5409f9c61620813456632', --University of Luxembourg + 'openorgs____::6445d7758d3a40c4d997953b6632a368', --National Institute of Informatics (NII) + 'openorgs____::b77c01aa15de3675da34277d48de2ec1', -- Valencia Catholic University Saint Vincent Martyr + 'openorgs____::7fe2f66cdc43983c6b24816bfe9cf6a0', -- Unviersity of Warsaw + 'openorgs____::15e7921fc50d9aa1229a82a84429419e', -- University Of Thessaly + 'openorgs____::11f7919dadc8f8a7251af54bba60c956', -- Technical University of Crete + 'openorgs____::84f0c5f5dbb6daf42748485924efde4b', -- University of Piraeus + 'openorgs____::4ac562f0376fce3539504567649cb373', -- University of Patras + 'openorgs____::3e8d1f8c3f6cd7f418b09f1f58b4873b', -- Aristotle University of Thessaloniki + 'openorgs____::3fcef6e1c469c10f2a84b281372c9814', -- World Bank + 'openorgs____::1698a2eb1885ef8adb5a4a969e745ad3', -- École des Ponts ParisTech + 'openorgs____::e15adb13c4dadd49de4d35c39b5da93a', -- Nanyang Technological University + 'openorgs____::4b34103bde246228fcd837f5f1bf4212', -- Autonomous University of Barcelona + 'openorgs____::72ec75fcfc4e0df1a76dc4c49007fceb', -- McMaster University + 'openorgs____::51c7fc556e46381734a25a6fbc3fd398', -- University of Modena and Reggio Emilia + 'openorgs____::235d7f9ad18ecd7e6dc62ea4990cb9db', -- Bilkent University + 'openorgs____::31f2fa9e05b49d4cf40a19c3fed8eb06', -- Saints Cyril and Methodius University of Skopje + 'openorgs____::db7686f30f22cbe73a4fde872ce812a6', -- University of Milan + 'openorgs____::b8b8ca674452579f3f593d9f5e557483', -- University College Cork + 'openorgs____::38d7097854736583dde879d12dacafca', -- Brown University + 'openorgs____::57784c9e047e826fefdb1ef816120d92', --Arts et Métiers ParisTech + 'openorgs____::2530baca8a15936ba2e3297f2bce2e7e', -- University of Cape Town + 'openorgs____::d11f981828c485cd23d93f7f24f24db1', -- Technological University Dublin + 'openorgs____::5e6bf8962665cdd040341171e5c631d8', -- Delft University of Technology + 'openorgs____::846cb428d3f52a445f7275561a7beb5d', -- University of Manitoba + 'openorgs____::eb391317ed0dc684aa81ac16265de041', -- Universitat Rovira i Virgili + 'openorgs____::66aa9fc2fceb271423dfabcc38752dc0', -- Lund University + 'openorgs____::3cff625a4370d51e08624cc586138b2f', -- IMT Atlantique + 'openorgs____::c0b262bd6eab819e4c994914f9c010e2', -- National Institute of Geophysics and Volcanology + 'openorgs____::1624ff7c01bb641b91f4518539a0c28a', -- Vrije Universiteit Amsterdam + 'openorgs____::4d4051b56708688235252f1d8fddb8c1', --Iscte - Instituto Universitário de Lisboa + 'openorgs____::ab4ac74c35fa5dada770cf08e5110fab', -- Universidade Católica Portuguesa + 'openorgs____::4d4051b56708688235252f1d8fddb8c1', -- Iscte - Instituto Universitário de Lisboa + 'openorgs____::5d55fb216b14691cf68218daf5d78cd9', -- Munster Technological University + 'openorgs____::0fccc7640f0cb44d5cd1b06b312a06b9', -- Cardiff University + 'openorgs____::8839b55dae0c84d56fd533f52d5d483a' -- Leibniz Institute of Ecological Urban and Regional Development + ) )) foo; + +--ANALYZE TABLE TARGET.result COMPUTE STATISTICS; + +create view if not exists TARGET.category as select * from SOURCE.category; +create view if not exists TARGET.concept as select * from SOURCE.concept; +create view if not exists TARGET.context as select * from SOURCE.context; +create view if not exists TARGET.country as select * from SOURCE.country; +create view if not exists TARGET.countrygdp as select * from SOURCE.countrygdp; +create view if not exists TARGET.creation_date as select * from SOURCE.creation_date; +create view if not exists TARGET.funder as select * from SOURCE.funder; +create view if not exists TARGET.fundref as select * from SOURCE.fundref; +create view if not exists TARGET.rndexpenditure as select * from SOURCE.rndexpediture; +create view if not exists TARGET.rndgdpexpenditure as select * from SOURCE.rndgdpexpenditure; +create view if not exists TARGET.doctoratestudents as select * from SOURCE.doctoratestudents; +create view if not exists TARGET.totalresearchers as select * from SOURCE.totalresearchers; +create view if not exists TARGET.totalresearchersft as select * from SOURCE.totalresearchersft; +create view if not exists TARGET.hrrst as select * from SOURCE.hrrst; +--create view if not exists TARGET.graduatedoctorates as select * from SOURCE.graduatedoctorates; + +create table TARGET.result_citations stored as parquet as select * from SOURCE.result_citations orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_citations COMPUTE STATISTICS; + +create table TARGET.result_references_oc stored as parquet as select * from SOURCE.result_references_oc orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_references_oc COMPUTE STATISTICS; + +create table TARGET.result_citations_oc stored as parquet as select * from SOURCE.result_citations_oc orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_citations_oc COMPUTE STATISTICS; + +create table TARGET.result_classifications stored as parquet as select * from SOURCE.result_classifications orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_classifications COMPUTE STATISTICS; + +create table TARGET.result_apc stored as parquet as select * from SOURCE.result_apc orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_apc COMPUTE STATISTICS; + +create table TARGET.result_concepts stored as parquet as select * from SOURCE.result_concepts orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_concepts COMPUTE STATISTICS; + +create table TARGET.result_datasources stored as parquet as select * from SOURCE.result_datasources orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_datasources COMPUTE STATISTICS; + +create table TARGET.result_fundercount stored as parquet as select * from SOURCE.result_fundercount orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_fundercount COMPUTE STATISTICS; + +create table TARGET.result_gold stored as parquet as select * from SOURCE.result_gold orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_gold COMPUTE STATISTICS; + +create table TARGET.result_greenoa stored as parquet as select * from SOURCE.result_greenoa orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_greenoa COMPUTE STATISTICS; + +create table TARGET.result_languages stored as parquet as select * from SOURCE.result_languages orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_languages COMPUTE STATISTICS; + +create table TARGET.result_licenses stored as parquet as select * from SOURCE.result_licenses orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_licenses COMPUTE STATISTICS; + +create table TARGET.licenses_normalized STORED AS PARQUET as select * from SOURCE.licenses_normalized; +--ANALYZE TABLE TARGET.licenses_normalized COMPUTE STATISTICS; + +create table TARGET.result_oids stored as parquet as select * from SOURCE.result_oids orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_oids COMPUTE STATISTICS; + +create table TARGET.result_organization stored as parquet as select * from SOURCE.result_organization orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_organization COMPUTE STATISTICS; + +create table TARGET.result_peerreviewed stored as parquet as select * from SOURCE.result_peerreviewed orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_peerreviewed COMPUTE STATISTICS; + +create table TARGET.result_pids stored as parquet as select * from SOURCE.result_pids orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_pids COMPUTE STATISTICS; + +create table TARGET.result_projectcount stored as parquet as select * from SOURCE.result_projectcount orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_projectcount COMPUTE STATISTICS; + +create table TARGET.result_projects stored as parquet as select * from SOURCE.result_projects orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_projects COMPUTE STATISTICS; + +create table TARGET.result_refereed stored as parquet as select * from SOURCE.result_refereed orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_refereed COMPUTE STATISTICS; + +create table TARGET.result_sources stored as parquet as select * from SOURCE.result_sources orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_sources COMPUTE STATISTICS; + +create table TARGET.result_topics stored as parquet as select * from SOURCE.result_topics orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_topics COMPUTE STATISTICS; + +create table TARGET.result_fos stored as parquet as select * from SOURCE.result_fos orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_fos COMPUTE STATISTICS; + +create table TARGET.result_accessroute stored as parquet as select * from SOURCE.result_accessroute orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_accessroute COMPUTE STATISTICS; + +create view TARGET.foo1 as select * from SOURCE.result_result rr where rr.source in (select id from TARGET.result); +create view TARGET.foo2 as select * from SOURCE.result_result rr where rr.target in (select id from TARGET.result); +create table TARGET.result_result STORED AS PARQUET as select distinct * from (select * from TARGET.foo1 union all select * from TARGET.foo2) foufou; +drop view TARGET.foo1; +drop view TARGET.foo2; +--ANALYZE TABLE TARGET.result_result COMPUTE STATISTICS; + +-- datasources +create view if not exists TARGET.datasource as select * from SOURCE.datasource; +create view if not exists TARGET.datasource_oids as select * from SOURCE.datasource_oids; +create view if not exists TARGET.datasource_organizations as select * from SOURCE.datasource_organizations; +create view if not exists TARGET.datasource_sources as select * from SOURCE.datasource_sources; + +create table TARGET.datasource_results stored as parquet as select id as result, datasource as id from TARGET.result_datasources; +--ANALYZE TABLE TARGET.datasource_results COMPUTE STATISTICS; + +-- organizations +create view if not exists TARGET.organization as select * from SOURCE.organization; +create view if not exists TARGET.organization_datasources as select * from SOURCE.organization_datasources; +create view if not exists TARGET.organization_pids as select * from SOURCE.organization_pids; +create view if not exists TARGET.organization_projects as select * from SOURCE.organization_projects; +create view if not exists TARGET.organization_sources as select * from SOURCE.organization_sources; + +-- projects +create view if not exists TARGET.project as select * from SOURCE.project; +create view if not exists TARGET.project_oids as select * from SOURCE.project_oids; +create view if not exists TARGET.project_organizations as select * from SOURCE.project_organizations; +create view if not exists TARGET.project_resultcount as select * from SOURCE.project_resultcount; +create view if not exists TARGET.project_classification as select * from SOURCE.project_classification; +create view if not exists TARGET.project_organization_contribution as select * from SOURCE.project_organization_contribution; + +create table TARGET.project_results stored as parquet as select id as result, project as id from TARGET.result_projects; +--ANALYZE TABLE TARGET.project_results COMPUTE STATISTICS; + +-- indicators +-- Sprint 1 ---- +create table TARGET.indi_pub_green_oa stored as parquet as select * from SOURCE.indi_pub_green_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_pub_green_oa COMPUTE STATISTICS; +create table TARGET.indi_pub_grey_lit stored as parquet as select * from SOURCE.indi_pub_grey_lit orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_pub_grey_lit COMPUTE STATISTICS; +create table TARGET.indi_pub_doi_from_crossref stored as parquet as select * from SOURCE.indi_pub_doi_from_crossref orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_pub_doi_from_crossref COMPUTE STATISTICS; +-- Sprint 2 ---- +create table TARGET.indi_result_has_cc_licence stored as parquet as select * from SOURCE.indi_result_has_cc_licence orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_result_has_cc_licence COMPUTE STATISTICS; +create table TARGET.indi_result_has_cc_licence_url stored as parquet as select * from SOURCE.indi_result_has_cc_licence_url orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_result_has_cc_licence_url COMPUTE STATISTICS; +create table TARGET.indi_pub_has_abstract stored as parquet as select * from SOURCE.indi_pub_has_abstract orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_pub_has_abstract COMPUTE STATISTICS; +create table TARGET.indi_result_with_orcid stored as parquet as select * from SOURCE.indi_result_with_orcid orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_result_with_orcid COMPUTE STATISTICS; +---- Sprint 3 ---- +create table TARGET.indi_funded_result_with_fundref stored as parquet as select * from SOURCE.indi_funded_result_with_fundref orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_funded_result_with_fundref COMPUTE STATISTICS; +create view TARGET.indi_result_org_collab as select * from SOURCE.indi_result_org_collab; +create view TARGET.indi_result_org_country_collab as select * from SOURCE.indi_result_org_country_collab; +create view TARGET.indi_project_collab_org as select * from SOURCE.indi_project_collab_org; +create view TARGET.indi_project_collab_org_country as select * from SOURCE.indi_project_collab_org_country; +create view TARGET.indi_funder_country_collab as select * from SOURCE.indi_funder_country_collab; +create view TARGET.indi_result_country_collab as select * from SOURCE.indi_result_country_collab; +---- Sprint 4 ---- +create table TARGET.indi_pub_diamond stored as parquet as select * from SOURCE.indi_pub_diamond orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_pub_diamond COMPUTE STATISTICS; +create table TARGET.indi_pub_in_transformative stored as parquet as select * from SOURCE.indi_pub_in_transformative orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_pub_in_transformative COMPUTE STATISTICS; +create table TARGET.indi_pub_closed_other_open stored as parquet as select * from SOURCE.indi_pub_closed_other_open orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_pub_closed_other_open COMPUTE STATISTICS; +---- Sprint 5 ---- +create table TARGET.indi_result_no_of_copies stored as parquet as select * from SOURCE.indi_result_no_of_copies orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_result_no_of_copies COMPUTE STATISTICS; +---- Sprint 6 ---- +create table TARGET.indi_pub_hybrid_oa_with_cc stored as parquet as select * from SOURCE.indi_pub_hybrid_oa_with_cc orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_pub_hybrid_oa_with_cc COMPUTE STATISTICS; +create table TARGET.indi_pub_bronze_oa stored as parquet as select * from SOURCE.indi_pub_bronze_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_pub_bronze_oa COMPUTE STATISTICS; +create table TARGET.indi_pub_downloads stored as parquet as select * from SOURCE.indi_pub_downloads orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); +--ANALYZE TABLE TARGET.indi_pub_downloads COMPUTE STATISTICS; +create table TARGET.indi_pub_downloads_datasource stored as parquet as select * from SOURCE.indi_pub_downloads_datasource orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); +--ANALYZE TABLE TARGET.indi_pub_downloads_datasource COMPUTE STATISTICS; +create table TARGET.indi_pub_downloads_year stored as parquet as select * from SOURCE.indi_pub_downloads_year orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); +--ANALYZE TABLE TARGET.indi_pub_downloads_year COMPUTE STATISTICS; +create table TARGET.indi_pub_downloads_datasource_year stored as parquet as select * from SOURCE.indi_pub_downloads_datasource_year orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); +--ANALYZE TABLE TARGET.indi_pub_downloads_datasource_year COMPUTE STATISTICS; +---- Sprint 7 ---- +create table TARGET.indi_pub_gold_oa stored as parquet as select * from SOURCE.indi_pub_gold_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_pub_gold_oa COMPUTE STATISTICS; +create table TARGET.indi_pub_hybrid stored as parquet as select * from SOURCE.indi_pub_hybrid orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_pub_hybrid COMPUTE STATISTICS; +create view TARGET.indi_org_fairness as select * from SOURCE.indi_org_fairness; +create view TARGET.indi_org_fairness_pub_pr as select * from SOURCE.indi_org_fairness_pub_pr; +create view TARGET.indi_org_fairness_pub_year as select * from SOURCE.indi_org_fairness_pub_year; +create view TARGET.indi_org_fairness_pub as select * from SOURCE.indi_org_fairness_pub; +create view TARGET.indi_org_fairness_year as select * from SOURCE.indi_org_fairness_year; +create view TARGET.indi_org_findable_year as select * from SOURCE.indi_org_findable_year; +create view TARGET.indi_org_findable as select * from SOURCE.indi_org_findable; +create view TARGET.indi_org_openess as select * from SOURCE.indi_org_openess; +create view TARGET.indi_org_openess_year as select * from SOURCE.indi_org_openess_year; +create table TARGET.indi_pub_has_preprint stored as parquet as select * from SOURCE.indi_pub_has_preprint orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_pub_has_preprint COMPUTE STATISTICS; +create table TARGET.indi_pub_in_subscribed stored as parquet as select * from SOURCE.indi_pub_in_subscribed orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_pub_in_subscribed COMPUTE STATISTICS; +create table TARGET.indi_result_with_pid stored as parquet as select * from SOURCE.indi_result_with_pid orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_result_with_pid COMPUTE STATISTICS; +create table TARGET.indi_impact_measures stored as parquet as select * from SOURCE.indi_impact_measures orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_impact_measures COMPUTE STATISTICS; +create table TARGET.indi_pub_interdisciplinarity stored as parquet as select * from SOURCE.indi_pub_interdisciplinarity orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.indi_pub_interdisciplinarity COMPUTE STATISTICS; +create table TARGET.result_apc_affiliations stored as parquet as select * from SOURCE.result_apc_affiliations orig where exists (select 1 from TARGET.result r where r.id=orig.id); +--ANALYZE TABLE TARGET.result_apc_affiliations COMPUTE STATISTICS; +create table TARGET.indi_is_project_result_after stored as parquet as select * from SOURCE.indi_is_project_result_after orig where exists (select 1 from TARGET.result r where r.id=orig.id); +create table TARGET.indi_is_funder_plan_s stored as parquet as select * from SOURCE.indi_is_funder_plan_s orig where exists (select 1 from TARGET.result r where r.id=orig.id); +create view TARGET.indi_funder_fairness as select * from SOURCE.indi_funder_fairness; +create view TARGET.indi_funder_openess as select * from SOURCE.indi_funder_openess; +create view TARGET.indi_funder_findable as select * from SOURCE.indi_funder_findable; +create view TARGET.indi_ris_fairness as select * from SOURCE.indi_ris_fairness; +create view TARGET.indi_ris_openess as select * from SOURCE.indi_ris_openess; +create view TARGET.indi_ris_findable as select * from SOURCE.indi_ris_findable; + + diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs.sql index 92b40405d..9a9407c2d 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs.sql @@ -12,4 +12,4 @@ create table TARGET.result stored as parquet as -- join SOURCE.result where rc.id=r.id and conc.category like CONTEXT) ) foo; -ANALYZE TABLE TARGET.result COMPUTE STATISTICS; \ No newline at end of file +--ANALYZE TABLE TARGET.result COMPUTE STATISTICS; \ 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/step20-createMonitorDB_RIs_tail.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs_tail.sql index ef6d08d79..bad18efde 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs_tail.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs_tail.sql @@ -12,4 +12,4 @@ create table TARGET.result stored as parquet as -- join SOURCE.result where rc.id=r.id and conc.category not in (CONTEXTS)) ) foo; -ANALYZE TABLE TARGET.result COMPUTE STATISTICS; \ No newline at end of file +--ANALYZE TABLE TARGET.result COMPUTE STATISTICS; \ 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/step20-createMonitorDB_funded.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_funded.sql index 8d8739c74..b8d3c0242 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_funded.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_funded.sql @@ -6,4 +6,4 @@ create table TARGET.result stored as parquet as select * from SOURCE.result r where exists (select 1 from SOURCE.result_projects rp join SOURCE.project p on rp.project=p.id where rp.id=r.id) ) foo; -ANALYZE TABLE TARGET.result COMPUTE STATISTICS; \ No newline at end of file +--ANALYZE TABLE TARGET.result COMPUTE STATISTICS; \ 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/step20-createMonitorDB_institutions.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql index 442e623cd..7bfba92a8 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql @@ -34,16 +34,16 @@ create table TARGET.result stored as parquet as 'openorgs____::3e8d1f8c3f6cd7f418b09f1f58b4873b', -- Aristotle University of Thessaloniki 'openorgs____::3fcef6e1c469c10f2a84b281372c9814', -- World Bank 'openorgs____::1698a2eb1885ef8adb5a4a969e745ad3', -- École des Ponts ParisTech - 'openorgs____::e15adb13c4dadd49de4d35c39b5da93a', -- Nanyang Technological University - 'openorgs____::4b34103bde246228fcd837f5f1bf4212', -- Autonomous University of Barcelona - 'openorgs____::72ec75fcfc4e0df1a76dc4c49007fceb', -- McMaster University - 'openorgs____::51c7fc556e46381734a25a6fbc3fd398', -- University of Modena and Reggio Emilia - 'openorgs____::235d7f9ad18ecd7e6dc62ea4990cb9db', -- Bilkent University - 'openorgs____::31f2fa9e05b49d4cf40a19c3fed8eb06', -- Saints Cyril and Methodius University of Skopje + 'openorgs____::e15adb13c4dadd49de4d35c39b5da93a', -- Nanyang Technological University + 'openorgs____::4b34103bde246228fcd837f5f1bf4212', -- Autonomous University of Barcelona + 'openorgs____::72ec75fcfc4e0df1a76dc4c49007fceb', -- McMaster University + 'openorgs____::51c7fc556e46381734a25a6fbc3fd398', -- University of Modena and Reggio Emilia + 'openorgs____::235d7f9ad18ecd7e6dc62ea4990cb9db', -- Bilkent University + 'openorgs____::31f2fa9e05b49d4cf40a19c3fed8eb06', -- Saints Cyril and Methodius University of Skopje 'openorgs____::db7686f30f22cbe73a4fde872ce812a6', -- University of Milan - 'openorgs____::b8b8ca674452579f3f593d9f5e557483', -- University College Cork - 'openorgs____::38d7097854736583dde879d12dacafca' -- Brown University - 'openorgs____::57784c9e047e826fefdb1ef816120d92', --Arts et Métiers ParisTech + 'openorgs____::b8b8ca674452579f3f593d9f5e557483', -- University College Cork + 'openorgs____::38d7097854736583dde879d12dacafca', -- Brown University + 'openorgs____::57784c9e047e826fefdb1ef816120d92', --Arts et Métiers ParisTech 'openorgs____::2530baca8a15936ba2e3297f2bce2e7e', -- University of Cape Town 'openorgs____::d11f981828c485cd23d93f7f24f24db1', -- Technological University Dublin 'openorgs____::5e6bf8962665cdd040341171e5c631d8', -- Delft University of Technology @@ -52,7 +52,13 @@ create table TARGET.result stored as parquet as 'openorgs____::66aa9fc2fceb271423dfabcc38752dc0', -- Lund University 'openorgs____::3cff625a4370d51e08624cc586138b2f', -- IMT Atlantique 'openorgs____::c0b262bd6eab819e4c994914f9c010e2', -- National Institute of Geophysics and Volcanology - 'openorgs____::1624ff7c01bb641b91f4518539a0c28a' -- Vrije Universiteit Amsterdam + 'openorgs____::1624ff7c01bb641b91f4518539a0c28a', -- Vrije Universiteit Amsterdam + 'openorgs____::4d4051b56708688235252f1d8fddb8c1', --Iscte - Instituto Universitário de Lisboa + 'openorgs____::ab4ac74c35fa5dada770cf08e5110fab', -- Universidade Católica Portuguesa + 'openorgs____::4d4051b56708688235252f1d8fddb8c1', -- Iscte - Instituto Universitário de Lisboa + 'openorgs____::5d55fb216b14691cf68218daf5d78cd9', -- Munster Technological University + 'openorgs____::0fccc7640f0cb44d5cd1b06b312a06b9', -- Cardiff University + 'openorgs____::8839b55dae0c84d56fd533f52d5d483a' -- Leibniz Institute of Ecological Urban and Regional Development ))) foo; -ANALYZE TABLE TARGET.result COMPUTE STATISTICS; \ No newline at end of file +--ANALYZE TABLE TARGET.result COMPUTE STATISTICS; \ 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/step21-createObservatoryDB.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql index 2d7d572b3..b7e421813 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql @@ -8,7 +8,7 @@ from ${stats_db_name}.result r group by rl.id ) rln on rln.id=r.id; -ANALYZE TABLE ${observatory_db_name}.result_cc_licence COMPUTE STATISTICS; +--ANALYZE TABLE ${observatory_db_name}.result_cc_licence COMPUTE STATISTICS; create table ${observatory_db_name}.result_affiliated_country stored as parquet as select @@ -39,7 +39,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, c.code, c.name; -ANALYZE TABLE ${observatory_db_name}.result_affiliated_country COMPUTE STATISTICS; +--ANALYZE TABLE ${observatory_db_name}.result_affiliated_country COMPUTE STATISTICS; create table ${observatory_db_name}.result_affiliated_year stored as parquet as select @@ -70,7 +70,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year; -ANALYZE TABLE ${observatory_db_name}.result_affiliated_year COMPUTE STATISTICS; +--ANALYZE TABLE ${observatory_db_name}.result_affiliated_year COMPUTE STATISTICS; create table ${observatory_db_name}.result_affiliated_year_country stored as parquet as select @@ -101,7 +101,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year, c.code, c.name; -ANALYZE TABLE ${observatory_db_name}.result_affiliated_year_country COMPUTE STATISTICS; +--ANALYZE TABLE ${observatory_db_name}.result_affiliated_year_country COMPUTE STATISTICS; create table ${observatory_db_name}.result_affiliated_datasource stored as parquet as select @@ -134,7 +134,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name; -ANALYZE TABLE ${observatory_db_name}.result_affiliated_datasource COMPUTE STATISTICS; +--ANALYZE TABLE ${observatory_db_name}.result_affiliated_datasource COMPUTE STATISTICS; create table ${observatory_db_name}.result_affiliated_datasource_country stored as parquet as select @@ -167,7 +167,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name, c.code, c.name; -ANALYZE TABLE ${observatory_db_name}.result_affiliated_datasource_country COMPUTE STATISTICS; +--ANALYZE TABLE ${observatory_db_name}.result_affiliated_datasource_country COMPUTE STATISTICS; create table ${observatory_db_name}.result_affiliated_organization stored as parquet as select @@ -198,7 +198,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name; -ANALYZE TABLE ${observatory_db_name}.result_affiliated_organization COMPUTE STATISTICS; +--ANALYZE TABLE ${observatory_db_name}.result_affiliated_organization COMPUTE STATISTICS; create table ${observatory_db_name}.result_affiliated_organization_country stored as parquet as select @@ -229,7 +229,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name, c.code, c.name; -ANALYZE TABLE ${observatory_db_name}.result_affiliated_organization_country COMPUTE STATISTICS; +--ANALYZE TABLE ${observatory_db_name}.result_affiliated_organization_country COMPUTE STATISTICS; create table ${observatory_db_name}.result_affiliated_funder stored as parquet as select @@ -262,7 +262,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder; -ANALYZE TABLE ${observatory_db_name}.result_affiliated_funder COMPUTE STATISTICS; +--ANALYZE TABLE ${observatory_db_name}.result_affiliated_funder COMPUTE STATISTICS; create table ${observatory_db_name}.result_affiliated_funder_country stored as parquet as select @@ -295,7 +295,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder, c.code, c.name; -ANALYZE TABLE ${observatory_db_name}.result_affiliated_funder_country COMPUTE STATISTICS; +--ANALYZE TABLE ${observatory_db_name}.result_affiliated_funder_country COMPUTE STATISTICS; create table ${observatory_db_name}.result_deposited_country stored as parquet as select @@ -328,7 +328,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, c.code, c.name; -ANALYZE TABLE ${observatory_db_name}.result_deposited_country COMPUTE STATISTICS; +--ANALYZE TABLE ${observatory_db_name}.result_deposited_country COMPUTE STATISTICS; create table ${observatory_db_name}.result_deposited_year stored as parquet as select @@ -361,7 +361,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year; -ANALYZE TABLE ${observatory_db_name}.result_deposited_year COMPUTE STATISTICS; +--ANALYZE TABLE ${observatory_db_name}.result_deposited_year COMPUTE STATISTICS; create table ${observatory_db_name}.result_deposited_year_country stored as parquet as select @@ -394,7 +394,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year, c.code, c.name; -ANALYZE TABLE ${observatory_db_name}.result_deposited_year_country COMPUTE STATISTICS; +--ANALYZE TABLE ${observatory_db_name}.result_deposited_year_country COMPUTE STATISTICS; create table ${observatory_db_name}.result_deposited_datasource stored as parquet as select @@ -427,7 +427,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name; -ANALYZE TABLE ${observatory_db_name}.result_deposited_datasource COMPUTE STATISTICS; +--ANALYZE TABLE ${observatory_db_name}.result_deposited_datasource COMPUTE STATISTICS; create table ${observatory_db_name}.result_deposited_datasource_country stored as parquet as select @@ -460,7 +460,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name, c.code, c.name; -ANALYZE TABLE ${observatory_db_name}.result_deposited_datasource_country COMPUTE STATISTICS; +--ANALYZE TABLE ${observatory_db_name}.result_deposited_datasource_country COMPUTE STATISTICS; create table ${observatory_db_name}.result_deposited_organization stored as parquet as select @@ -493,7 +493,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name; -ANALYZE TABLE ${observatory_db_name}.result_deposited_organization COMPUTE STATISTICS; +--ANALYZE TABLE ${observatory_db_name}.result_deposited_organization COMPUTE STATISTICS; create table ${observatory_db_name}.result_deposited_organization_country stored as parquet as select @@ -526,7 +526,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name, c.code, c.name; -ANALYZE TABLE ${observatory_db_name}.result_deposited_organization_country COMPUTE STATISTICS; +--ANALYZE TABLE ${observatory_db_name}.result_deposited_organization_country COMPUTE STATISTICS; create table ${observatory_db_name}.result_deposited_funder stored as parquet as select @@ -561,7 +561,7 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder; -ANALYZE TABLE ${observatory_db_name}.result_deposited_funder COMPUTE STATISTICS; +--ANALYZE TABLE ${observatory_db_name}.result_deposited_funder COMPUTE STATISTICS; create table ${observatory_db_name}.result_deposited_funder_country stored as parquet as select @@ -596,4 +596,4 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder, c.code, c.name; -ANALYZE TABLE ${observatory_db_name}.result_deposited_funder_country COMPUTE STATISTICS; \ No newline at end of file +--ANALYZE TABLE ${observatory_db_name}.result_deposited_funder_country COMPUTE STATISTICS; \ 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/step5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql old mode 100644 new mode 100755 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 2ab50fb29..aa991730b 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 @@ -317,15 +317,12 @@ - - ${jobTracker} - ${nameNode} - indicators.sh - ${stats_db_name} - ${external_stats_db_name} - ${wf:appPath()}/scripts/step16-createIndicatorsTables.sql - indicators.sh - + + ${hive_jdbc_url} + + stats_db_name=${stats_db_name} + external_stats_db_name=${external_stats_db_name} + @@ -378,6 +375,7 @@ ${wf:appPath()}/scripts/step20-createMonitorDB_institutions.sql ${wf:appPath()}/scripts/step20-createMonitorDB_RIs.sql ${wf:appPath()}/scripts/step20-createMonitorDB_RIs_tail.sql + ${wf:appPath()}/scripts/step20-createMonitorDBAll.sql monitor.sh diff --git a/dhp-workflows/dhp-swh/pom.xml b/dhp-workflows/dhp-swh/pom.xml new file mode 100644 index 000000000..80fff4587 --- /dev/null +++ b/dhp-workflows/dhp-swh/pom.xml @@ -0,0 +1,110 @@ + + + 4.0.0 + + eu.dnetlib.dhp + dhp-workflows + 1.2.5-SNAPSHOT + + dhp-swh + + + + org.apache.spark + spark-core_${scala.binary.version} + + + + org.apache.spark + spark-sql_${scala.binary.version} + + + + eu.dnetlib.dhp + dhp-common + ${project.version} + + + net.sf.saxon + Saxon-HE + + + + + + dom4j + dom4j + + + + xml-apis + xml-apis + + + + jaxen + jaxen + + + + org.apache.hadoop + hadoop-distcp + + + + eu.dnetlib + dnet-actionmanager-api + + + eu.dnetlib + dnet-actionmanager-common + + + eu.dnetlib + dnet-openaireplus-mapping-utils + + + saxonica + saxon + + + saxonica + saxon-dom + + + jgrapht + jgrapht + + + net.sf.ehcache + ehcache + + + org.springframework + spring-test + + + org.apache.* + * + + + apache + * + + + + + + org.apache.httpcomponents + httpclient + 4.5.13 + + + org.datanucleus + datanucleus-core + 3.2.10 + compile + + + + diff --git a/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/ArchiveRepositoryURLs.java b/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/ArchiveRepositoryURLs.java new file mode 100644 index 000000000..baa510346 --- /dev/null +++ b/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/ArchiveRepositoryURLs.java @@ -0,0 +1,176 @@ + +package eu.dnetlib.dhp.swh; + +import static eu.dnetlib.dhp.utils.DHPUtils.getHadoopConfiguration; + +import java.io.IOException; +import java.net.URL; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.Optional; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.cli.ParseException; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +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.collection.CollectorException; +import eu.dnetlib.dhp.common.collection.HttpClientParams; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.utils.GraphCleaningFunctions; +import eu.dnetlib.dhp.swh.models.LastVisitData; +import eu.dnetlib.dhp.swh.utils.SWHConnection; +import eu.dnetlib.dhp.swh.utils.SWHConstants; +import eu.dnetlib.dhp.swh.utils.SWHUtils; + +/** + * Sends archive requests to the SWH API for those software repository URLs that are missing from them + * + * @author Serafeim Chatzopoulos + */ +public class ArchiveRepositoryURLs { + + private static final Logger log = LoggerFactory.getLogger(ArchiveRepositoryURLs.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static SWHConnection swhConnection = null; + + public static void main(final String[] args) throws IOException, ParseException { + final ArgumentApplicationParser argumentParser = new ArgumentApplicationParser( + IOUtils + .toString( + CollectLastVisitRepositoryData.class + .getResourceAsStream( + "/eu/dnetlib/dhp/swh/input_archive_repository_urls.json"))); + argumentParser.parseArgument(args); + + final String hdfsuri = argumentParser.get("namenode"); + log.info("hdfsURI: {}", hdfsuri); + + final String inputPath = argumentParser.get("lastVisitsPath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = argumentParser.get("archiveRequestsPath"); + log.info("outputPath: {}", outputPath); + + final Integer archiveThresholdInDays = Integer.parseInt(argumentParser.get("archiveThresholdInDays")); + log.info("archiveThresholdInDays: {}", archiveThresholdInDays); + + final String apiAccessToken = argumentParser.get("apiAccessToken"); + log.info("apiAccessToken: {}", apiAccessToken); + + final HttpClientParams clientParams = SWHUtils.getClientParams(argumentParser); + + swhConnection = new SWHConnection(clientParams, apiAccessToken); + + final FileSystem fs = FileSystem.get(getHadoopConfiguration(hdfsuri)); + + archive(fs, inputPath, outputPath, archiveThresholdInDays); + + } + + private static void archive(FileSystem fs, String inputPath, String outputPath, Integer archiveThresholdInDays) + throws IOException { + + SequenceFile.Reader fr = SWHUtils.getSequenceFileReader(fs, inputPath); + SequenceFile.Writer fw = SWHUtils.getSequenceFileWriter(fs, outputPath); + + // Create key and value objects to hold data + Text repoUrl = new Text(); + Text lastVisitData = new Text(); + + // Read key-value pairs from the SequenceFile and handle appropriately + while (fr.next(repoUrl, lastVisitData)) { + + String response = null; + try { + response = handleRecord(repoUrl.toString(), lastVisitData.toString(), archiveThresholdInDays); + } catch (java.text.ParseException e) { + log.error("Could not handle record with repo Url: {}", repoUrl.toString()); + throw new RuntimeException(e); + } + + // response is equal to null when no need for request + if (response != null) { + SWHUtils.appendToSequenceFile(fw, repoUrl.toString(), response); + } + + } + + // Close readers + fw.close(); + fr.close(); + } + + public static String handleRecord(String repoUrl, String lastVisitData, Integer archiveThresholdInDays) + throws IOException, java.text.ParseException { + + log.info("{ Key: {}, Value: {} }", repoUrl, lastVisitData); + + LastVisitData lastVisit = OBJECT_MAPPER.readValue(lastVisitData, LastVisitData.class); + + // a previous attempt for archival has been made, and repository URL was not found + // avoid performing the same archive request again + if (lastVisit.getStatus() != null && + lastVisit.getStatus().equals(SWHConstants.VISIT_STATUS_NOT_FOUND)) { + + log.info("Avoid request -- previous archive request returned NOT_FOUND"); + return null; + } + + // if we have last visit data + if (lastVisit.getSnapshot() != null) { + + String cleanDate = GraphCleaningFunctions.cleanDate(lastVisit.getDate()); + + // and the last visit date can be parsed + if (cleanDate != null) { + + SimpleDateFormat formatter = new SimpleDateFormat(ModelSupport.DATE_FORMAT); + Date lastVisitDate = formatter.parse(cleanDate); + + // OR last visit time < (now() - archiveThresholdInDays) + long diffInMillies = Math.abs((new Date()).getTime() - lastVisitDate.getTime()); + long diffInDays = TimeUnit.DAYS.convert(diffInMillies, TimeUnit.MILLISECONDS); + log.info("Date diff from now (in days): {}", diffInDays); + + // do not perform a request, if the last visit date is no older than $archiveThresholdInDays + if (archiveThresholdInDays >= diffInDays) { + log.info("Avoid request -- no older than {} days", archiveThresholdInDays); + return null; + } + } + } + + // ELSE perform an archive request + log.info("Perform archive request for: {}", repoUrl); + + // if last visit data are available, re-use version control type, + // else use the default one (i.e., git) + String visitType = Optional + .ofNullable(lastVisit.getType()) + .orElse(SWHConstants.DEFAULT_VISIT_TYPE); + + URL url = new URL(String.format(SWHConstants.SWH_ARCHIVE_URL, visitType, repoUrl.trim())); + + log.info("Sending archive request: {}", url); + + String response; + try { + response = swhConnection.call(url.toString()); + } catch (CollectorException e) { + log.error("Error in request: {}", url); + response = "{}"; + } + + return response; + } + +} diff --git a/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/CollectLastVisitRepositoryData.java b/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/CollectLastVisitRepositoryData.java new file mode 100644 index 000000000..ebb9176ff --- /dev/null +++ b/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/CollectLastVisitRepositoryData.java @@ -0,0 +1,119 @@ + +package eu.dnetlib.dhp.swh; + +import static eu.dnetlib.dhp.utils.DHPUtils.getHadoopConfiguration; + +import java.io.BufferedReader; +import java.io.IOException; +import java.net.URL; + +import org.apache.commons.cli.ParseException; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.SequenceFile; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.collection.CollectorException; +import eu.dnetlib.dhp.common.collection.HttpClientParams; +import eu.dnetlib.dhp.swh.utils.SWHConnection; +import eu.dnetlib.dhp.swh.utils.SWHConstants; +import eu.dnetlib.dhp.swh.utils.SWHUtils; + +/** + * Given a file with software repository URLs, this class + * collects last visit data from the Software Heritage API. + * + * @author Serafeim Chatzopoulos + */ +public class CollectLastVisitRepositoryData { + + private static final Logger log = LoggerFactory.getLogger(CollectLastVisitRepositoryData.class); + private static SWHConnection swhConnection = null; + + public static void main(final String[] args) + throws IOException, ParseException { + final ArgumentApplicationParser argumentParser = new ArgumentApplicationParser( + IOUtils + .toString( + CollectLastVisitRepositoryData.class + .getResourceAsStream( + "/eu/dnetlib/dhp/swh/input_collect_last_visit_repository_data.json"))); + argumentParser.parseArgument(args); + + log.info("Java Xmx: {}m", Runtime.getRuntime().maxMemory() / (1024 * 1024)); + + final String hdfsuri = argumentParser.get("namenode"); + log.info("hdfsURI: {}", hdfsuri); + + final String inputPath = argumentParser.get("softwareCodeRepositoryURLs"); + log.info("inputPath: {}", inputPath); + + final String outputPath = argumentParser.get("lastVisitsPath"); + log.info("outputPath: {}", outputPath); + + final String apiAccessToken = argumentParser.get("apiAccessToken"); + log.info("apiAccessToken: {}", apiAccessToken); + + final HttpClientParams clientParams = SWHUtils.getClientParams(argumentParser); + + swhConnection = new SWHConnection(clientParams, apiAccessToken); + + final FileSystem fs = FileSystem.get(getHadoopConfiguration(hdfsuri)); + + collect(fs, inputPath, outputPath); + + fs.close(); + } + + private static void collect(FileSystem fs, String inputPath, String outputPath) + throws IOException { + + SequenceFile.Writer fw = SWHUtils.getSequenceFileWriter(fs, outputPath); + + // Specify the HDFS directory path you want to read + Path directoryPath = new Path(inputPath); + + // List all files in the directory + FileStatus[] partStatuses = fs.listStatus(directoryPath); + + for (FileStatus partStatus : partStatuses) { + + // Check if it's a file (not a directory) + if (partStatus.isFile()) { + handleFile(fs, partStatus.getPath(), fw); + } + + } + + fw.close(); + } + + private static void handleFile(FileSystem fs, Path partInputPath, SequenceFile.Writer fw) + throws IOException { + + BufferedReader br = SWHUtils.getFileReader(fs, partInputPath); + + String repoUrl; + while ((repoUrl = br.readLine()) != null) { + + URL url = new URL(String.format(SWHConstants.SWH_LATEST_VISIT_URL, repoUrl.trim())); + + String response; + try { + response = swhConnection.call(url.toString()); + } catch (CollectorException e) { + log.error("Error in request: {}", url); + response = "{}"; + } + + SWHUtils.appendToSequenceFile(fw, repoUrl, response); + } + + br.close(); + } + +} diff --git a/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/CollectSoftwareRepositoryURLs.java b/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/CollectSoftwareRepositoryURLs.java new file mode 100644 index 000000000..abd51bc5b --- /dev/null +++ b/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/CollectSoftwareRepositoryURLs.java @@ -0,0 +1,93 @@ + +package eu.dnetlib.dhp.swh; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +import java.util.Optional; + +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Result; + +/** + * Collects unique software repository URLs in the Graph using Hive + * + * @author Serafeim Chatzopoulos + */ +public class CollectSoftwareRepositoryURLs { + + private static final Logger log = LoggerFactory.getLogger(CollectSoftwareRepositoryURLs.class); + + public static void main(String[] args) throws Exception { + + String jsonConfiguration = IOUtils + .toString( + CollectSoftwareRepositoryURLs.class + .getResourceAsStream("/eu/dnetlib/dhp/swh/input_collect_software_repository_urls.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + parser.parseArgument(args); + + final Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + final String hiveDbName = parser.get("hiveDbName"); + log.info("hiveDbName: {}", hiveDbName); + + final String outputPath = parser.get("softwareCodeRepositoryURLs"); + log.info("softwareCodeRepositoryURLs: {}", outputPath); + + final String hiveMetastoreUris = parser.get("hiveMetastoreUris"); + log.info("hiveMetastoreUris: {}", hiveMetastoreUris); + + final Integer softwareLimit = Integer.parseInt(parser.get("softwareLimit")); + log.info("softwareLimit: {}", softwareLimit); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", hiveMetastoreUris); + + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + doRun(spark, hiveDbName, softwareLimit, outputPath); + }); + } + + private static void doRun(SparkSession spark, String hiveDbName, Integer limit, + String outputPath) { + + String queryTemplate = "SELECT distinct coderepositoryurl.value " + + "FROM %s.software " + + "WHERE coderepositoryurl.value IS NOT NULL " + + "AND datainfo.deletedbyinference = FALSE " + + "AND datainfo.invisible = FALSE "; + + if (limit != null) { + queryTemplate += String.format("LIMIT %s", limit); + } + + String query = String.format(queryTemplate, hiveDbName); + + log.info("Hive query to fetch software code URLs: {}", query); + + Dataset df = spark.sql(query); + + // write distinct repository URLs + df + .write() + .mode(SaveMode.Overwrite) + .csv(outputPath); + } +} diff --git a/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/PrepareSWHActionsets.java b/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/PrepareSWHActionsets.java new file mode 100644 index 000000000..2691d4b7e --- /dev/null +++ b/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/PrepareSWHActionsets.java @@ -0,0 +1,185 @@ + +package eu.dnetlib.dhp.swh; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; +import static org.apache.spark.sql.functions.col; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.hadoop.mapred.SequenceFileOutputFormat; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.*; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; +import eu.dnetlib.dhp.swh.models.LastVisitData; +import eu.dnetlib.dhp.swh.utils.SWHConstants; +import scala.Tuple2; + +/** + * Creates action sets for Software Heritage data + * + * @author Serafeim Chatzopoulos + */ +public class PrepareSWHActionsets { + + private static final Logger log = LoggerFactory.getLogger(PrepareSWHActionsets.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws Exception { + + String jsonConfiguration = IOUtils + .toString( + PrepareSWHActionsets.class + .getResourceAsStream( + "/eu/dnetlib/dhp/swh/input_prepare_swh_actionsets.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + parser.parseArgument(args); + + final Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + final String inputPath = parser.get("lastVisitsPath"); + log.info("inputPath: {}", inputPath); + + final String softwareInputPath = parser.get("softwareInputPath"); + log.info("softwareInputPath: {}", softwareInputPath); + + final String outputPath = parser.get("actionsetsPath"); + log.info("outputPath: {}", outputPath); + + SparkConf conf = new SparkConf(); + + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + JavaPairRDD softwareRDD = prepareActionsets(spark, inputPath, softwareInputPath); + softwareRDD + .saveAsHadoopFile( + outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class); + }); + } + + private static Dataset loadSWHData(SparkSession spark, String inputPath) { + + JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + // read from file and transform to tuples + // Note: snapshot id is the SWH id for us + JavaRDD swhRDD = sc + .sequenceFile(inputPath, Text.class, Text.class) + .map(t -> t._2().toString()) + .map(t -> OBJECT_MAPPER.readValue(t, LastVisitData.class)) + .filter(t -> t.getOrigin() != null && t.getSnapshot() != null) // response from SWH API is empty if repo URL + // was not found + .map(item -> RowFactory.create(item.getOrigin(), item.getSnapshot())); + + // convert RDD to 2-column DF + List fields = Arrays + .asList( + DataTypes.createStructField("repoUrl", DataTypes.StringType, true), + DataTypes.createStructField("swhId", DataTypes.StringType, true)); + StructType schema = DataTypes.createStructType(fields); + + return spark.createDataFrame(swhRDD, schema); + } + + private static Dataset loadGraphSoftwareData(SparkSession spark, String softwareInputPath) { + return spark + .read() + .textFile(softwareInputPath) + .map( + (MapFunction) t -> OBJECT_MAPPER.readValue(t, Software.class), + Encoders.bean(Software.class)) + .filter(t -> t.getCodeRepositoryUrl() != null) + .select(col("id"), col("codeRepositoryUrl.value").as("repoUrl")); + } + + private static JavaPairRDD prepareActionsets(SparkSession spark, String inputPath, + String softwareInputPath) { + + Dataset swhDF = loadSWHData(spark, inputPath); +// swhDF.show(false); + + Dataset graphSoftwareDF = loadGraphSoftwareData(spark, softwareInputPath); +// graphSoftwareDF.show(5); + + Dataset joinedDF = graphSoftwareDF.join(swhDF, "repoUrl").select("id", "swhid"); +// joinedDF.show(false); + + return joinedDF.map((MapFunction) row -> { + + Software s = new Software(); + + // set openaire id + s.setId(row.getString(row.fieldIndex("id"))); + + // set swh id + Qualifier qualifier = OafMapperUtils + .qualifier( + SWHConstants.SWHID, + SWHConstants.SWHID_CLASSNAME, + ModelConstants.DNET_PID_TYPES, + ModelConstants.DNET_PID_TYPES); + + DataInfo dataInfo = OafMapperUtils + .dataInfo( + false, + null, + false, + false, + ModelConstants.PROVENANCE_ACTION_SET_QUALIFIER, + ""); + + s + .setPid( + Arrays + .asList( + OafMapperUtils + .structuredProperty( + String.format("swh:1:snp:%s", row.getString(row.fieldIndex("swhid"))), + qualifier, + dataInfo))); + + // add SWH in the `collectedFrom` field + KeyValue kv = new KeyValue(); + kv.setKey(SWHConstants.SWH_ID); + kv.setValue(SWHConstants.SWH_NAME); + + s.setCollectedfrom(Arrays.asList(kv)); + + return s; + }, Encoders.bean(Software.class)) + .toJavaRDD() + .map(p -> new AtomicAction(Software.class, p)) + .mapToPair( + aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), + new Text(OBJECT_MAPPER.writeValueAsString(aa)))); + } +} diff --git a/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/models/LastVisitData.java b/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/models/LastVisitData.java new file mode 100644 index 000000000..5e705716c --- /dev/null +++ b/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/models/LastVisitData.java @@ -0,0 +1,71 @@ + +package eu.dnetlib.dhp.swh.models; + +import java.io.Serializable; + +import com.cloudera.com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; + +@JsonIgnoreProperties(ignoreUnknown = true) +public class LastVisitData implements Serializable { + + private String origin; + private String type; + private String date; + + @JsonProperty("snapshot") + private String snapshotId; + + private String status; + + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + public String getDate() { + return date; + } + + public void setDate(String date) { + this.date = date; + } + + public String getSnapshot() { + return snapshotId; + } + + public void setSnapshot(String snapshotId) { + this.snapshotId = snapshotId; + } + + public String getStatus() { + return status; + } + + public void setStatus(String status) { + this.status = status; + } + + public String getOrigin() { + return origin; + } + + public void setOrigin(String origin) { + this.origin = origin; + } + + @Override + public String toString() { + return "LastVisitData{" + + "origin='" + origin + '\'' + + ", type='" + type + '\'' + + ", date='" + date + '\'' + + ", snapshotId='" + snapshotId + '\'' + + ", status='" + status + '\'' + + '}'; + } +} diff --git a/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/utils/SWHConnection.java b/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/utils/SWHConnection.java new file mode 100644 index 000000000..80249e816 --- /dev/null +++ b/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/utils/SWHConnection.java @@ -0,0 +1,40 @@ + +package eu.dnetlib.dhp.swh.utils; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.http.HttpHeaders; + +import eu.dnetlib.dhp.common.collection.CollectorException; +import eu.dnetlib.dhp.common.collection.HttpClientParams; +import eu.dnetlib.dhp.common.collection.HttpConnector2; + +public class SWHConnection { + + HttpConnector2 conn; + + public SWHConnection(HttpClientParams clientParams, String accessToken) { + + // set custom headers + Map headers = new HashMap() { + { + put(HttpHeaders.ACCEPT, "application/json"); + if (accessToken != null) { + put(HttpHeaders.AUTHORIZATION, String.format("Bearer %s", accessToken)); + } + } + }; + + clientParams.setHeaders(headers); + + // create http connector + conn = new HttpConnector2(clientParams); + + } + + public String call(String url) throws CollectorException { + return conn.getInputSource(url); + } + +} diff --git a/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/utils/SWHConstants.java b/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/utils/SWHConstants.java new file mode 100644 index 000000000..2a0403044 --- /dev/null +++ b/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/utils/SWHConstants.java @@ -0,0 +1,21 @@ + +package eu.dnetlib.dhp.swh.utils; + +public class SWHConstants { + public static final String SWH_LATEST_VISIT_URL = "https://archive.softwareheritage.org/api/1/origin/%s/visit/latest/"; + + public static final String SWH_ARCHIVE_URL = "https://archive.softwareheritage.org/api/1/origin/save/%s/url/%s/"; + + public static final String DEFAULT_VISIT_TYPE = "git"; + + public static final String VISIT_STATUS_NOT_FOUND = "not_found"; + + public static final String SWHID = "swhid"; + + public static final String SWHID_CLASSNAME = "Software Hash Identifier"; + + public static final String SWH_ID = "10|openaire____::dbfd07503aaa1ed31beed7dec942f3f4"; + + public static final String SWH_NAME = "Software Heritage"; + +} diff --git a/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/utils/SWHUtils.java b/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/utils/SWHUtils.java new file mode 100644 index 000000000..405ce51e4 --- /dev/null +++ b/dhp-workflows/dhp-swh/src/main/java/eu/dnetlib/dhp/swh/utils/SWHUtils.java @@ -0,0 +1,95 @@ + +package eu.dnetlib.dhp.swh.utils; + +import static eu.dnetlib.dhp.common.Constants.*; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.util.Optional; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.collection.HttpClientParams; + +public class SWHUtils { + + private static final Logger log = LoggerFactory.getLogger(SWHUtils.class); + + public static HttpClientParams getClientParams(ArgumentApplicationParser argumentParser) { + + final HttpClientParams clientParams = new HttpClientParams(); + clientParams + .setMaxNumberOfRetry( + Optional + .ofNullable(argumentParser.get(MAX_NUMBER_OF_RETRY)) + .map(Integer::parseInt) + .orElse(HttpClientParams._maxNumberOfRetry)); + log.info("maxNumberOfRetry is {}", clientParams.getMaxNumberOfRetry()); + + clientParams + .setRequestDelay( + Optional + .ofNullable(argumentParser.get(REQUEST_DELAY)) + .map(Integer::parseInt) + .orElse(HttpClientParams._requestDelay)); + log.info("requestDelay is {}", clientParams.getRequestDelay()); + + clientParams + .setRetryDelay( + Optional + .ofNullable(argumentParser.get(RETRY_DELAY)) + .map(Integer::parseInt) + .orElse(HttpClientParams._retryDelay)); + log.info("retryDelay is {}", clientParams.getRetryDelay()); + + clientParams + .setRequestMethod( + Optional + .ofNullable(argumentParser.get(REQUEST_METHOD)) + .orElse(HttpClientParams._requestMethod)); + log.info("requestMethod is {}", clientParams.getRequestMethod()); + + return clientParams; + } + + public static BufferedReader getFileReader(FileSystem fs, Path inputPath) throws IOException { + FSDataInputStream inputStream = fs.open(inputPath); + return new BufferedReader( + new InputStreamReader(inputStream, StandardCharsets.UTF_8)); + } + + public static SequenceFile.Writer getSequenceFileWriter(FileSystem fs, String outputPath) throws IOException { + return SequenceFile + .createWriter( + fs.getConf(), + SequenceFile.Writer.file(new Path(outputPath)), + SequenceFile.Writer.keyClass(Text.class), + SequenceFile.Writer.valueClass(Text.class)); + } + + public static SequenceFile.Reader getSequenceFileReader(FileSystem fs, String inputPath) throws IOException { + Path filePath = new Path(inputPath); + SequenceFile.Reader.Option fileOption = SequenceFile.Reader.file(filePath); + + return new SequenceFile.Reader(fs.getConf(), fileOption); + } + + public static void appendToSequenceFile(SequenceFile.Writer fw, String keyStr, String valueStr) throws IOException { + Text key = new Text(); + key.set(keyStr); + + Text value = new Text(); + value.set(valueStr); + + fw.append(key, value); + } +} diff --git a/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/input_archive_repository_urls.json b/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/input_archive_repository_urls.json new file mode 100644 index 000000000..e8671f71b --- /dev/null +++ b/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/input_archive_repository_urls.json @@ -0,0 +1,56 @@ +[ + { + "paramName": "n", + "paramLongName": "namenode", + "paramDescription": "the Name Node URI", + "paramRequired": true + }, + { + "paramName": "lv", + "paramLongName": "lastVisitsPath", + "paramDescription": "the URL where to store last visits data", + "paramRequired": true + }, + { + "paramName": "arp", + "paramLongName": "archiveRequestsPath", + "paramDescription": "the URL where to store the responses of the archive requests", + "paramRequired": true + }, + { + "paramName": "mnr", + "paramLongName": "maxNumberOfRetry", + "paramDescription": "the maximum number of admitted connection retries", + "paramRequired": false + }, + { + "paramName": "rqd", + "paramLongName": "requestDelay", + "paramDescription": "the delay (ms) between requests", + "paramRequired": false + }, + { + "paramName": "rtd", + "paramLongName": "retryDelay", + "paramDescription": "the delay (ms) between retries", + "paramRequired": false + }, + { + "paramName": "rm", + "paramLongName": "requestMethod", + "paramDescription": "the method of the requests to perform", + "paramRequired": false + }, + { + "paramName": "atid", + "paramLongName": "archiveThresholdInDays", + "paramDescription": "the thershold (in days) required to issue an archive request", + "paramRequired": false + }, + { + "paramName": "aat", + "paramLongName": "apiAccessToken", + "paramDescription": "the API access token of the SWH API", + "paramRequired": false + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/input_collect_last_visit_repository_data.json b/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/input_collect_last_visit_repository_data.json new file mode 100644 index 000000000..662582dfe --- /dev/null +++ b/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/input_collect_last_visit_repository_data.json @@ -0,0 +1,50 @@ +[ + { + "paramName": "n", + "paramLongName": "namenode", + "paramDescription": "the Name Node URI", + "paramRequired": true + }, + { + "paramName": "scr", + "paramLongName": "softwareCodeRepositoryURLs", + "paramDescription": "the URL from where to read software repository URLs", + "paramRequired": true + }, + { + "paramName": "lv", + "paramLongName": "lastVisitsPath", + "paramDescription": "the URL where to store last visits data", + "paramRequired": true + }, + { + "paramName": "mnr", + "paramLongName": "maxNumberOfRetry", + "paramDescription": "the maximum number of admitted connection retries", + "paramRequired": false + }, + { + "paramName": "rqd", + "paramLongName": "requestDelay", + "paramDescription": "the delay (ms) between requests", + "paramRequired": false + }, + { + "paramName": "rtd", + "paramLongName": "retryDelay", + "paramDescription": "the delay (ms) between retries", + "paramRequired": false + }, + { + "paramName": "rm", + "paramLongName": "requestMethod", + "paramDescription": "the method of the requests to perform", + "paramRequired": false + }, + { + "paramName": "aat", + "paramLongName": "apiAccessToken", + "paramDescription": "the API access token of the SWH API", + "paramRequired": false + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/input_collect_software_repository_urls.json b/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/input_collect_software_repository_urls.json new file mode 100644 index 000000000..4459fe9df --- /dev/null +++ b/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/input_collect_software_repository_urls.json @@ -0,0 +1,32 @@ +[ + { + "paramName": "issm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "when true will stop SparkSession after job execution", + "paramRequired": false + }, + { + "paramName": "scr", + "paramLongName": "softwareCodeRepositoryURLs", + "paramDescription": "the URL where to store software repository URLs", + "paramRequired": true + }, + { + "paramName": "db", + "paramLongName": "hiveDbName", + "paramDescription": "the target hive database name", + "paramRequired": true + }, + { + "paramName": "hmu", + "paramLongName": "hiveMetastoreUris", + "paramDescription": "the hive metastore uris", + "paramRequired": true + }, + { + "paramName": "slim", + "paramLongName": "softwareLimit", + "paramDescription": "limit on the number of software repo URL to fetch", + "paramRequired": false + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/input_prepare_swh_actionsets.json b/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/input_prepare_swh_actionsets.json new file mode 100644 index 000000000..07ab0b1f4 --- /dev/null +++ b/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/input_prepare_swh_actionsets.json @@ -0,0 +1,26 @@ +[ + { + "paramName": "issm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "when true will stop SparkSession after job execution", + "paramRequired": false + }, + { + "paramName": "lv", + "paramLongName": "lastVisitsPath", + "paramDescription": "the URL where to store last visits data", + "paramRequired": true + }, + { + "paramName": "ap", + "paramLongName": "actionsetsPath", + "paramDescription": "the URL path where to store actionsets", + "paramRequired": true + }, + { + "paramName": "sip", + "paramLongName": "softwareInputPath", + "paramDescription": "the URL path of the software in the graph", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/job.properties b/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/job.properties new file mode 100644 index 000000000..35c068286 --- /dev/null +++ b/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/job.properties @@ -0,0 +1,19 @@ +# hive +hiveDbName=openaire_prod_20230914 + +# input/output files +softwareCodeRepositoryURLs=${workingDir}/1_code_repo_urls.csv +lastVisitsPath=${workingDir}/2_last_visits.seq +archiveRequestsPath=${workingDir}/3_archive_requests.seq +actionsetsPath=${workingDir}/4_actionsets +graphPath=/tmp/prod_provision/graph/18_graph_blacklisted + +apiAccessToken=eyJhbGciOiJIUzI1NiIsInR5cCIgOiAiSldUIiwia2lkIiA6ICJhMTMxYTQ1My1hM2IyLTQwMTUtODQ2Ny05MzAyZjk3MTFkOGEifQ.eyJpYXQiOjE2OTQ2MzYwMjAsImp0aSI6IjkwZjdkNTNjLTQ5YTktNGFiMy1hY2E0LTcwMTViMjEyZTNjNiIsImlzcyI6Imh0dHBzOi8vYXV0aC5zb2Z0d2FyZWhlcml0YWdlLm9yZy9hdXRoL3JlYWxtcy9Tb2Z0d2FyZUhlcml0YWdlIiwiYXVkIjoiaHR0cHM6Ly9hdXRoLnNvZnR3YXJlaGVyaXRhZ2Uub3JnL2F1dGgvcmVhbG1zL1NvZnR3YXJlSGVyaXRhZ2UiLCJzdWIiOiIzMTY5OWZkNC0xNmE0LTQxOWItYTdhMi00NjI5MDY4ZjI3OWEiLCJ0eXAiOiJPZmZsaW5lIiwiYXpwIjoic3doLXdlYiIsInNlc3Npb25fc3RhdGUiOiIzMjYzMzEwMS00ZDRkLTQwMjItODU2NC1iMzNlMTJiNTE3ZDkiLCJzY29wZSI6Im9wZW5pZCBvZmZsaW5lX2FjY2VzcyBwcm9maWxlIGVtYWlsIn0.XHj1VIZu1dZ4Ej32-oU84mFmaox9cLNjXosNxwZM0Xs + +maxNumberOfRetry=2 +retryDelay=1 +requestDelay=100 + +softwareLimit=500 + +resume=collect-software-repository-urls diff --git a/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/oozie_app/config-default.xml b/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/oozie_app/config-default.xml new file mode 100644 index 000000000..3e45a53fa --- /dev/null +++ b/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/oozie_app/config-default.xml @@ -0,0 +1,54 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + + hiveMetastoreUris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + + spark2YarnHistoryServerAddress + http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089 + + + spark2EventLogDir + /user/spark/spark2ApplicationHistory + + + spark2ExtraListeners + "com.cloudera.spark.lineage.NavigatorAppListener" + + + spark2SqlQueryExecutionListeners + "com.cloudera.spark.lineage.NavigatorQueryListener" + + + oozieActionShareLibForSpark2 + spark2 + + + resourceManager + http://iis-cdh5-test-m2.ocean.icm.edu.pl:8088/cluster + + + oozie.launcher.mapreduce.user.classpath.first + true + + + sparkSqlWarehouseDir + /user/hive/warehouse + + \ No newline at end of file diff --git a/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/oozie_app/workflow.xml b/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/oozie_app/workflow.xml new file mode 100644 index 000000000..c625fcb5b --- /dev/null +++ b/dhp-workflows/dhp-swh/src/main/resources/eu/dnetlib/dhp/swh/oozie_app/workflow.xml @@ -0,0 +1,183 @@ + + + + + + hiveDbName + The name of the Hive DB to be used + + + softwareCodeRepositoryURLs + The path in the HDFS to save the software repository URLs + + + lastVisitsPath + The path in the HDFS to save the responses of the last visit requests + + + archiveRequestsPath + The path in the HDFS to save the responses of the archive requests + + + actionsetsPath + The path in the HDFS to save the action sets + + + graphPath + The path in the HDFS to the base folder of the graph + + + maxNumberOfRetry + Max number of retries for failed API calls + + + retryDelay + Retry delay for failed requests (in sec) + + + requestDelay + Delay between API requests (in ms) + + + apiAccessToken + The API Key of the SWH API + + + softwareLimit + Limit on the number of repo URLs to use (Optional); for debug purposes + + + resumeFrom + Variable that indicates the step to start from + + + + + + ${jobTracker} + ${nameNode} + + + oozie.action.sharelib.for.spark + ${oozieActionShareLibForSpark2} + + + actionsetsPath + ${actionsetsPath} + + + apiAccessToken + ${apiAccessToken} + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + ${wf:conf('resumeFrom') eq 'collect-software-repository-urls'} + ${wf:conf('resumeFrom') eq 'create-swh-actionsets'} + + + + + + + yarn + cluster + Collect software repository URLs + eu.dnetlib.dhp.swh.CollectSoftwareRepositoryURLs + dhp-swh-${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.warehouse.dir=${sparkSqlWarehouseDir} + + + --softwareCodeRepositoryURLs${softwareCodeRepositoryURLs} + --hiveDbName${hiveDbName} + --hiveMetastoreUris${hiveMetastoreUris} + --softwareLimit${softwareLimit} + + + + + + + + eu.dnetlib.dhp.swh.CollectLastVisitRepositoryData + + --namenode${nameNode} + --softwareCodeRepositoryURLs${softwareCodeRepositoryURLs} + --lastVisitsPath${lastVisitsPath} + + --maxNumberOfRetry${maxNumberOfRetry} + --requestDelay${requestDelay} + --retryDelay${retryDelay} + --requestMethodGET + --apiAccessToken${apiAccessToken} + + + + + + + + + eu.dnetlib.dhp.swh.ArchiveRepositoryURLs + + --namenode${nameNode} + --lastVisitsPath${lastVisitsPath} + --archiveRequestsPath${archiveRequestsPath} + --archiveThresholdInDays365 + + --maxNumberOfRetry${maxNumberOfRetry} + --requestDelay${requestDelay} + --retryDelay${retryDelay} + --requestMethodPOST + --apiAccessToken${apiAccessToken} + + + + + + + + + yarn + cluster + Create actionsets for SWH data + eu.dnetlib.dhp.swh.PrepareSWHActionsets + dhp-swh-${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.warehouse.dir=${sparkSqlWarehouseDir} + + + --lastVisitsPath${lastVisitsPath} + --actionsetsPath${actionsetsPath} + --softwareInputPath${graphPath}/software + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-swh/src/test/java/eu/dnetlib/dhp/swh/ArchiveRepositoryURLsTest.java b/dhp-workflows/dhp-swh/src/test/java/eu/dnetlib/dhp/swh/ArchiveRepositoryURLsTest.java new file mode 100644 index 000000000..e069e9655 --- /dev/null +++ b/dhp-workflows/dhp-swh/src/test/java/eu/dnetlib/dhp/swh/ArchiveRepositoryURLsTest.java @@ -0,0 +1,38 @@ + +package eu.dnetlib.dhp.swh; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.text.ParseException; +import java.util.Arrays; + +import org.apache.hadoop.fs.FileSystem; +import org.junit.jupiter.api.Test; + +import eu.dnetlib.dhp.swh.utils.SWHUtils; + +public class ArchiveRepositoryURLsTest { + + @Test + void testArchive() throws IOException, ParseException { + String inputPath = getClass() + .getResource("/eu/dnetlib/dhp/swh/lastVisitDataToArchive.csv") + .getPath(); + + File file = new File(inputPath); + FileReader fr = new FileReader(file); + BufferedReader br = new BufferedReader(fr); // creates a buffering character input stream + + String line; + while ((line = br.readLine()) != null) { + String[] tokens = line.split("\t"); + + String response = ArchiveRepositoryURLs.handleRecord(tokens[0], tokens[1], 365); + System.out.println(tokens[0] + "\t" + response); + System.out.println(); + } + fr.close(); + } +} diff --git a/dhp-workflows/dhp-swh/src/test/java/eu/dnetlib/dhp/swh/PrepareSWHActionsetsTest.java b/dhp-workflows/dhp-swh/src/test/java/eu/dnetlib/dhp/swh/PrepareSWHActionsetsTest.java new file mode 100644 index 000000000..ffcb7aaee --- /dev/null +++ b/dhp-workflows/dhp-swh/src/test/java/eu/dnetlib/dhp/swh/PrepareSWHActionsetsTest.java @@ -0,0 +1,97 @@ + +package eu.dnetlib.dhp.swh; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.io.Text; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions; +import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; + +public class PrepareSWHActionsetsTest { + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static SparkSession spark; + + private static Path workingDir; + + private static final Logger log = LoggerFactory + .getLogger(PrepareSWHActionsetsTest.class); + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(PrepareSWHActionsetsTest.class.getSimpleName()); + + log.info("Using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(PrepareSWHActionsetsTest.class.getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + + spark = SparkSession + .builder() + .appName(PrepareSWHActionsetsTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + void testRun() throws Exception { + + String lastVisitsPath = getClass() + .getResource("/eu/dnetlib/dhp/swh/last_visits_data.seq") + .getPath(); + + String outputPath = workingDir.toString() + "/actionSet"; + + String softwareInputPath = getClass() + .getResource("/eu/dnetlib/dhp/swh/software.json.gz") + .getPath(); + + PrepareSWHActionsets + .main( + new String[] { + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-lastVisitsPath", lastVisitsPath, + "-softwareInputPath", softwareInputPath, + "-actionsetsPath", outputPath + }); + + } +} diff --git a/dhp-workflows/dhp-swh/src/test/java/eu/dnetlib/dhp/swh/SWHConnectionTest.java b/dhp-workflows/dhp-swh/src/test/java/eu/dnetlib/dhp/swh/SWHConnectionTest.java new file mode 100644 index 000000000..b19e0e7ac --- /dev/null +++ b/dhp-workflows/dhp-swh/src/test/java/eu/dnetlib/dhp/swh/SWHConnectionTest.java @@ -0,0 +1,58 @@ + +package eu.dnetlib.dhp.swh; + +import java.io.IOException; +import java.net.MalformedURLException; +import java.net.URL; + +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.common.collection.CollectorException; +import eu.dnetlib.dhp.common.collection.HttpClientParams; +import eu.dnetlib.dhp.swh.utils.SWHConnection; +import eu.dnetlib.dhp.swh.utils.SWHConstants; + +//import org.apache.hadoop.hdfs.MiniDFSCluster; + +public class SWHConnectionTest { + private static final Logger log = LoggerFactory.getLogger(SWHConnectionTest.class); + + @Test + void testGetCall() throws IOException { + + HttpClientParams clientParams = new HttpClientParams(); + clientParams.setRequestMethod("GET"); + + SWHConnection swhConnection = new SWHConnection(clientParams, null); + + String repoUrl = "https://github.com/stanford-futuredata/FAST"; + URL url = new URL(String.format(SWHConstants.SWH_LATEST_VISIT_URL, repoUrl)); + String response = null; + try { + response = swhConnection.call(url.toString()); + } catch (CollectorException e) { + System.out.println("Error in request: " + url); + } + System.out.println(response); + } + + @Test + void testPostCall() throws MalformedURLException { + HttpClientParams clientParams = new HttpClientParams(); + clientParams.setRequestMethod("POST"); + + SWHConnection swhConnection = new SWHConnection(clientParams, null); + + String repoUrl = "https://github.com/stanford-futuredata/FAST"; + URL url = new URL(String.format(SWHConstants.SWH_ARCHIVE_URL, SWHConstants.DEFAULT_VISIT_TYPE, repoUrl)); + String response = null; + try { + response = swhConnection.call(url.toString()); + } catch (CollectorException e) { + System.out.println("Error in request: " + url); + } + System.out.println(response); + } +} diff --git a/dhp-workflows/dhp-swh/src/test/resources/eu/dnetlib/dhp/swh/lastVisitDataToArchive.csv b/dhp-workflows/dhp-swh/src/test/resources/eu/dnetlib/dhp/swh/lastVisitDataToArchive.csv new file mode 100644 index 000000000..568ccf482 --- /dev/null +++ b/dhp-workflows/dhp-swh/src/test/resources/eu/dnetlib/dhp/swh/lastVisitDataToArchive.csv @@ -0,0 +1,7 @@ +https://bitbucket.org/samskillman/yt-stokes {"origin":"https://bitbucket.org/samskillman/yt-stokes","visit":43,"date":"2021-09-13T21:59:27.125171+00:00","status":"failed","snapshot":null,"type":"hg","metadata":{},"origin_url":"https://archive.softwareheritage.org/api/1/origin/https://bitbucket.org/samskillman/yt-stokes/get/","snapshot_url":null} +https://github.com/bioinsilico/BIPSPI {"origin":"https://github.com/bioinsilico/BIPSPI","visit":1,"date":"2020-03-18T14:50:21.541822+00:00","status":"full","snapshot":"c6c69d2cd73ce89811448da5f031611df6f63bdb","type":"git","metadata":{},"origin_url":"https://archive.softwareheritage.org/api/1/origin/https://github.com/bioinsilico/BIPSPI/get/","snapshot_url":"https://archive.softwareheritage.org/api/1/snapshot/c6c69d2cd73ce89811448da5f031611df6f63bdb/"} +https://github.com/mloop/kdiff-type1-error-rate/blob/master/analysis/simulation.R {} +https://github.com/schwanbeck/YSMR {"origin":"https://github.com/schwanbeck/YSMR","visit":6,"date":"2023-08-02T15:25:02.650676+00:00","status":"full","snapshot":"a9d1c5f0bca2def198b89f65bc9f7da3be8439ed","type":"git","metadata":{},"origin_url":"https://archive.softwareheritage.org/api/1/origin/https://github.com/schwanbeck/YSMR/get/","snapshot_url":"https://archive.softwareheritage.org/api/1/snapshot/a9d1c5f0bca2def198b89f65bc9f7da3be8439ed/"} +https://github.com/lvclark/TASSELGBS_combine {"origin":"https://github.com/lvclark/TASSELGBS_combine","visit":1,"date":"2020-04-12T20:44:09.405589+00:00","status":"full","snapshot":"ffa6fefd3f5becefbea9fe0e6d5d93859c95c071","type":"git","metadata":{},"origin_url":"https://archive.softwareheritage.org/api/1/origin/https://github.com/lvclark/TASSELGBS_combine/get/","snapshot_url":"https://archive.softwareheritage.org/api/1/snapshot/ffa6fefd3f5becefbea9fe0e6d5d93859c95c071/"} +https://github.com/PRIDE-Toolsuite/inspector-example-files {"origin":"https://github.com/PRIDE-Toolsuite/inspector-example-files","visit":12,"date":"2021-01-25T08:54:13.394674+00:00","status":"full","snapshot":"0b56eb0ad07cf778df6dabefc4b73636e0ae8b37","type":"git","metadata":{},"origin_url":"https://archive.softwareheritage.org/api/1/origin/https://github.com/PRIDE-Toolsuite/inspector-example-files/get/","snapshot_url":"https://archive.softwareheritage.org/api/1/snapshot/0b56eb0ad07cf778df6dabefc4b73636e0ae8b37/"} +https://bitbucket.org/matwey/chelyabinsk {"origin":"https://bitbucket.org/matwey/chelyabinsk","visit":6,"date":"2021-09-24T19:32:43.322909+00:00","status":"full","snapshot":"215913858c3ee0e61e1aaea18241c5ee006da1b0","type":"hg","metadata":{},"origin_url":"https://archive.softwareheritage.org/api/1/origin/https://bitbucket.org/matwey/chelyabinsk/get/","snapshot_url":"https://archive.softwareheritage.org/api/1/snapshot/215913858c3ee0e61e1aaea18241c5ee006da1b0/"} \ No newline at end of file diff --git a/dhp-workflows/dhp-swh/src/test/resources/eu/dnetlib/dhp/swh/last_visits_data.seq b/dhp-workflows/dhp-swh/src/test/resources/eu/dnetlib/dhp/swh/last_visits_data.seq new file mode 100644 index 000000000..683fc0e69 Binary files /dev/null and b/dhp-workflows/dhp-swh/src/test/resources/eu/dnetlib/dhp/swh/last_visits_data.seq differ diff --git a/dhp-workflows/dhp-swh/src/test/resources/eu/dnetlib/dhp/swh/software.json.gz b/dhp-workflows/dhp-swh/src/test/resources/eu/dnetlib/dhp/swh/software.json.gz new file mode 100644 index 000000000..3a62c0615 Binary files /dev/null and b/dhp-workflows/dhp-swh/src/test/resources/eu/dnetlib/dhp/swh/software.json.gz differ diff --git a/dhp-workflows/pom.xml b/dhp-workflows/pom.xml index 44e3cdde8..369c71b5b 100644 --- a/dhp-workflows/pom.xml +++ b/dhp-workflows/pom.xml @@ -38,6 +38,7 @@ dhp-broker-events dhp-doiboost dhp-impact-indicators + dhp-swh diff --git a/pom.xml b/pom.xml index c6b65e27a..f361a266c 100644 --- a/pom.xml +++ b/pom.xml @@ -112,6 +112,16 @@ https://maven.d4science.org/nexus/content/repositories/dnet-deps default + + maven-restlet + Restlet repository + https://maven.restlet.talend.com + + + conjars + conjars + https://conjars.wensel.net/repo/ + @@ -878,7 +888,7 @@ 3.3.3 3.4.2 [2.12,3.0) - [3.17.1] + [3.17.2] [4.0.3] [6.0.5] [3.1.6]