diff --git a/dhp-common/src/main/scala/eu/dnetlib/dhp/application/dedup/log/DedupLogModel.scala b/dhp-common/src/main/scala/eu/dnetlib/dhp/application/dedup/log/DedupLogModel.scala new file mode 100644 index 000000000..c1473e7df --- /dev/null +++ b/dhp-common/src/main/scala/eu/dnetlib/dhp/application/dedup/log/DedupLogModel.scala @@ -0,0 +1,3 @@ +package eu.dnetlib.dhp.application.dedup.log + +case class DedupLogModel(tag:String, configuration:String, entity:String, startTS:Long, endTS:Long, totalMs:Long ) {} diff --git a/dhp-common/src/main/scala/eu/dnetlib/dhp/application/dedup/log/DedupLogWriter.scala b/dhp-common/src/main/scala/eu/dnetlib/dhp/application/dedup/log/DedupLogWriter.scala new file mode 100644 index 000000000..3060a13ae --- /dev/null +++ b/dhp-common/src/main/scala/eu/dnetlib/dhp/application/dedup/log/DedupLogWriter.scala @@ -0,0 +1,16 @@ +package eu.dnetlib.dhp.application.dedup.log + +import org.apache.spark.sql.{SaveMode, SparkSession} + +class DedupLogWriter (path:String) { + + + def appendLog(dedupLogModel: DedupLogModel, spark:SparkSession): Unit = { + import spark.implicits._ + val df = spark.createDataset[DedupLogModel](data = List(dedupLogModel)) + df.write.mode(SaveMode.Append).save(path) + + + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala index 670323598..c079f7537 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala @@ -82,7 +82,7 @@ object BioDBToOAF { "Springer Nature" ) val EBICollectedFrom: KeyValue = OafMapperUtils.keyValue( - "10|opendoar____::83e60e09c222f206c725385f53d7e567c", + "10|opendoar____::3e60e09c222f206c725385f53d7e567c", "EMBL-EBIs Protein Data Bank in Europe (PDBe)" ) val pubmedCollectedFrom: KeyValue = 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 3aa8f241d..c015078c5 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 @@ -4,7 +4,10 @@ package eu.dnetlib.dhp.oa.dedup; import java.io.IOException; import java.util.Optional; +import eu.dnetlib.dhp.application.dedup.log.DedupLogModel; +import eu.dnetlib.dhp.application.dedup.log.DedupLogWriter; import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -20,8 +23,6 @@ import org.xml.sax.SAXException; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.oa.dedup.model.Block; -import eu.dnetlib.dhp.schema.common.ModelConstants; -import eu.dnetlib.dhp.schema.oaf.DataInfo; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; @@ -73,9 +74,16 @@ public class SparkCreateSimRels extends AbstractSparkAction { log.info("actionSetId: '{}'", actionSetId); log.info("workingPath: '{}'", workingPath); + + final String dfLogPath = parser.get("dataframeLog"); + final String runTag = Optional.ofNullable(parser.get("runTAG")).orElse("UNKNOWN"); + + // for each dedup configuration for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) { + final long start = System.currentTimeMillis(); + final String entity = dedupConf.getWf().getEntityType(); final String subEntity = dedupConf.getWf().getSubEntityValue(); log.info("Creating simrels for: '{}'", subEntity); @@ -85,6 +93,7 @@ public class SparkCreateSimRels extends AbstractSparkAction { JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + JavaPairRDD mapDocuments = sc .textFile(DedupUtility.createEntityPath(graphBasePath, subEntity)) .repartition(numPartitions) @@ -109,6 +118,13 @@ public class SparkCreateSimRels extends AbstractSparkAction { Encoders.bean(Relation.class)); saveParquet(simRels, outputPath, SaveMode.Overwrite); + final long end = System.currentTimeMillis(); + if (StringUtils.isNotBlank(dfLogPath)) { + final DedupLogModel model = new DedupLogModel(runTag, dedupConf.toString(),entity, start, end, end-start); + new DedupLogWriter(dfLogPath).appendLog(model, spark); + + } + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json index 09f4365d3..c97de3482 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json @@ -28,5 +28,17 @@ "paramLongName": "numPartitions", "paramDescription": "number of partitions for the similarity relations intermediate phases", "paramRequired": false + }, + { + "paramName": "dl", + "paramLongName": "dataframeLog", + "paramDescription": "the path of the dataframe Log", + "paramRequired": false + }, + { + "paramName": "rt", + "paramLongName": "runTAG", + "paramDescription": "the label of the current running", + "paramRequired": false } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/entitytoorganizationfromsemrel/input_propagation_parameter.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/entitytoorganizationfromsemrel/input_propagation_parameter.json index 5a8597f38..66a7f5b2f 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/entitytoorganizationfromsemrel/input_propagation_parameter.json +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/entitytoorganizationfromsemrel/input_propagation_parameter.json @@ -52,10 +52,11 @@ "paramLongName": "iterations", "paramDescription": "the number of iterations to be computed", "paramRequired": false - },{ - "paramName": "pop", - "paramLongName": "projectOrganizationPath", - "paramDescription": "the number of iterations to be computed", - "paramRequired": true -} + }, + { + "paramName": "pop", + "paramLongName": "projectOrganizationPath", + "paramDescription": "the number of iterations to be computed", + "paramRequired": true + } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/entitytoorganizationfromsemrel/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/entitytoorganizationfromsemrel/oozie_app/workflow.xml index ff6ec8f37..16c8c4e19 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/entitytoorganizationfromsemrel/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/entitytoorganizationfromsemrel/oozie_app/workflow.xml @@ -181,6 +181,7 @@ --leavesPath${workingDir}/preparedInfo/leavesPath --childParentPath${workingDir}/preparedInfo/childParentPath --resultOrgPath${workingDir}/preparedInfo/resultOrgPath + --projectOrganizationPath${workingDir}/preparedInfo/projectOrganizationPath --hive_metastore_uris${hive_metastore_uris} --workingDir${workingDir}/working --iterations${iterations} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsForProvision.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsForProvision.sql index cba08d466..0ec303939 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsForProvision.sql +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsForProvision.sql @@ -18,7 +18,7 @@ SELECT ARRAY(SELECT DISTINCT pid FROM unnest(array_cat( array_agg(DISTINCT i.otherid || '###' || i.type || '@@@dnet:pid_types'), array_agg(DISTINCT idup.otherid || '###' || idup.type || '@@@dnet:pid_types') - )) as t(pid) where pid IS NOT NULL), + )) as t(pid) where pid IS NOT NULL) AS pid, (array_remove(array_cat(ARRAY[o.ec_legalbody], array_agg(od.ec_legalbody)), NULL))[1] AS eclegalbody, (array_remove(array_cat(ARRAY[o.ec_legalperson], array_agg(od.ec_legalperson)), NULL))[1] AS eclegalperson, (array_remove(array_cat(ARRAY[o.ec_nonprofit], array_agg(od.ec_nonprofit)), NULL))[1] AS ecnonprofit, diff --git a/dhp-workflows/dhp-stats-promote/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/updateProductionViews.sh b/dhp-workflows/dhp-stats-promote/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/updateProductionViews.sh index 3e510e87e..a6d7b289d 100644 --- a/dhp-workflows/dhp-stats-promote/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/updateProductionViews.sh +++ b/dhp-workflows/dhp-stats-promote/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/updateProductionViews.sh @@ -10,7 +10,7 @@ export SOURCE=$1 export PRODUCTION=$2 echo "Updating ${PRODUCTION} database" -impala-shell -q "create database if not exists ${PRODUCTION}" -impala-shell -d ${PRODUCTION} -q "show tables" --delimited | sed "s/^/drop view if exists ${PRODUCTION}./" | sed "s/$/;/" | impala-shell -c -f - -impala-shell -d ${SOURCE} -q "show tables" --delimited | sed "s/\(.*\)/create view ${PRODUCTION}.\1 as select * from ${SOURCE}.\1;/" | impala-shell -c -f - +impala-shell -i impala-cluster-dn1.openaire.eu -q "create database if not exists ${PRODUCTION}" +impala-shell -i impala-cluster-dn1.openaire.eu -d ${PRODUCTION} -q "show tables" --delimited | sed "s/^/drop view if exists ${PRODUCTION}./" | sed "s/$/;/" | impala-shell -c -f - +impala-shell -i impala-cluster-dn1.openaire.eu -d ${SOURCE} -q "show tables" --delimited | sed "s/\(.*\)/create view ${PRODUCTION}.\1 as select * from ${SOURCE}.\1;/" | impala-shell -c -f - echo "Production db ready!" \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-promote/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/updateProductionViewsMonitor.sh b/dhp-workflows/dhp-stats-promote/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/updateProductionViewsMonitor.sh new file mode 100644 index 000000000..68844b14c --- /dev/null +++ b/dhp-workflows/dhp-stats-promote/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/updateProductionViewsMonitor.sh @@ -0,0 +1,38 @@ +export PYTHON_EGG_CACHE=/home/$(whoami)/.python-eggs +export link_folder=/tmp/impala-shell-python-egg-cache-$(whoami) +if ! [ -L $link_folder ] +then + rm -Rf "$link_folder" + ln -sfn ${PYTHON_EGG_CACHE}${link_folder} ${link_folder} +fi + +export SOURCE=$1 +export PRODUCTION=$2 + +echo "Updating ${PRODUCTION}'_funded' database" +impala-shell -i impala-cluster-dn1.openaire.eu -q "create database if not exists ${PRODUCTION}'_funded'" +impala-shell -i impala-cluster-dn1.openaire.eu -d ${PRODUCTION}'_funded' -q "show tables" --delimited | sed "s/^/drop view if exists ${PRODUCTION}'_funded'./" | sed "s/$/;/" | impala-shell -c -f - +impala-shell -i impala-cluster-dn1.openaire.eu -d ${SOURCE}'_funded' -q "show tables" --delimited | sed "s/\(.*\)/create view ${PRODUCTION}'_funded'.\1 as select * from ${SOURCE}'_funded'.\1;/" | impala-shell -c -f - +echo "Production funded db ready!" + +echo "Updating ${PRODUCTION}'_institutions' database" +impala-shell -i impala-cluster-dn1.openaire.eu -q "create database if not exists ${PRODUCTION}'_institutions'" +impala-shell -i impala-cluster-dn1.openaire.eu -d ${PRODUCTION}'_institutions' -q "show tables" --delimited | sed "s/^/drop view if exists ${PRODUCTION}'_institutions'./" | sed "s/$/;/" | impala-shell -c -f - +impala-shell -i impala-cluster-dn1.openaire.eu -d ${SOURCE}'_institutions' -q "show tables" --delimited | sed "s/\(.*\)/create view ${PRODUCTION}'_institutions'.\1 as select * from ${SOURCE}'_institutions'.\1;/" | impala-shell -c -f - +echo "Production insitutions db ready!" + +echo "Updating ${PRODUCTION}'_ris_tail' database" +impala-shell -i impala-cluster-dn1.openaire.eu -q "create database if not exists ${PRODUCTION}'_ris_tail'" +impala-shell -i impala-cluster-dn1.openaire.eu -d ${PRODUCTION}'_ris_tail' -q "show tables" --delimited | sed "s/^/drop view if exists ${PRODUCTION}'_ris_tail'./" | sed "s/$/;/" | impala-shell -c -f - +impala-shell -i impala-cluster-dn1.openaire.eu -d ${SOURCE}'_RIs_tail' -q "show tables" --delimited | sed "s/\(.*\)/create view ${PRODUCTION}'_ris_tail'.\1 as select * from ${SOURCE}'_ris_tail'.\1;/" | impala-shell -c -f - +echo "Production RIS tail db ready!" + +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'` + impala-shell -i impala-cluster-dn1.openaire.eu -q "create database if not exists ${PRODUCTION}'_'${tmp}" + impala-shell -i impala-cluster-dn1.openaire.eu -d ${PRODUCTION}'_'${tmp} -q "show tables" --delimited | sed "s/^/drop view if exists ${PRODUCTION}'_'${tmp}./" | sed "s/$/;/" | impala-shell -c -f - + impala-shell -i impala-cluster-dn1.openaire.eu -d ${SOURCE}'_'${tmp} -q "show tables" --delimited | sed "s/\(.*\)/create view ${PRODUCTION}'_'${tmp}.\1 as select * from ${SOURCE}'_'${tmp}.\1;/" | impala-shell -c -f - + echo "Production ${tmp} db ready!" +done diff --git a/dhp-workflows/dhp-stats-promote/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-promote/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index 8286e5039..605c86ac9 100644 --- a/dhp-workflows/dhp-stats-promote/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-promote/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml @@ -80,10 +80,10 @@ ${jobTracker} ${nameNode} - updateProductionViews.sh + updateProductionViewsMonitor.sh ${monitor_db_name} ${monitor_db_production_name} - updateProductionViews.sh + updateProductionViewsMonitor.sh 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 b937eea25..87294f6e9 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 @@ -7,7 +7,7 @@ then fi #export HADOOP_USER_NAME="dimitris.pierrakos" -export HADOOP_USER_NAME=$5 +export HADOOP_USER_NAME=$6 export PROD_USAGE_STATS_DB="openaire_prod_usage_stats" function copydb() { db=$1 @@ -21,8 +21,22 @@ function copydb() { # change ownership to impala hdfs dfs -conf /etc/impala_cluster/hdfs-site.xml -chmod -R 777 /tmp/$FILE/${db}.db - # create the databases + # 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;"`; + 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;"`; + done + + # delete the database impala-shell --user $HADOOP_USER_NAME -i impala-cluster-dn1.openaire.eu -q "drop database if exists ${db} cascade"; + + # create the databases impala-shell --user $HADOOP_USER_NAME -i impala-cluster-dn1.openaire.eu -q "create database ${db}"; impala-shell --user $HADOOP_USER_NAME -q "INVALIDATE METADATA" 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 a77b5a113..857635b6c 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 @@ -10,6 +10,12 @@ function createShadowDB() { SOURCE=$1 SHADOW=$2 + # 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;"`; + done + impala-shell -i impala-cluster-dn1.openaire.eu -q "drop database ${SHADOW} CASCADE"; impala-shell -i impala-cluster-dn1.openaire.eu -q "create database if not exists ${SHADOW}"; # impala-shell -i impala-cluster-dn1.openaire.eu -d ${SHADOW} -q "show tables" | sed "s/^/drop view if exists ${SHADOW}./" | sed "s/$/;/" | impala-shell -i impala-cluster-dn1.openaire.eu -f - diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql index dc9e6c1f9..39755d68e 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql @@ -49,5 +49,5 @@ FROM ( WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on o.datasource = d.id; CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_accessroute STORED AS PARQUET as -select distinct substr(id,4),id, accessroute from ${openaire_db_name}.result +select distinct substr(id,4) as id, accessroute from ${openaire_db_name}.result lateral view explode (instance.accessright.openaccessroute) openaccessroute as accessroute; 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 cec22cd3e..132cb482e 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 @@ -33,4 +33,12 @@ select * from ${stats_db_name}.dataset_refereed union all select * from ${stats_db_name}.software_refereed union all -select * from ${stats_db_name}.otherresearchproduct_refereed; \ No newline at end of file +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 +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; 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 beec31082..f39ff2afd 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 @@ -1,21 +1,21 @@ ------------------------------------------- --- Extra tables, mostly used by indicators -create table ${stats_db_name}.result_projectcount STORED AS PARQUET as +create table if not exists ${stats_db_name}.result_projectcount STORED AS PARQUET as select r.id, count(distinct p.id) as count from ${stats_db_name}.result r left outer join ${stats_db_name}.result_projects rp on rp.id=r.id left outer join ${stats_db_name}.project p on p.id=rp.project group by r.id; -create table ${stats_db_name}.result_fundercount STORED AS PARQUET as +create table if not exists ${stats_db_name}.result_fundercount STORED AS PARQUET as select r.id, count(distinct p.funder) as count from ${stats_db_name}.result r left outer join ${stats_db_name}.result_projects rp on rp.id=r.id left outer join ${stats_db_name}.project p on p.id=rp.project group by r.id; -create table ${stats_db_name}.project_resultcount STORED AS PARQUET as +create table if not exists ${stats_db_name}.project_resultcount STORED AS PARQUET as with rcount as ( select p.id as pid, count(distinct r.id) as `count`, r.type as type from ${stats_db_name}.project p @@ -29,14 +29,14 @@ select rcount.pid, sum(case when rcount.type='publication' then rcount.count els from rcount group by rcount.pid; -create view ${stats_db_name}.rndexpenditure as select * from stats_ext.rndexpediture; -create view ${stats_db_name}.rndgdpexpenditure as select * from stats_ext.rndgdpexpenditure; -create view ${stats_db_name}.doctoratestudents as select * from stats_ext.doctoratestudents; -create view ${stats_db_name}.totalresearchers as select * from stats_ext.totalresearchers; -create view ${stats_db_name}.totalresearchersft as select * from stats_ext.totalresearchersft; -create view ${stats_db_name}.hrrst as select * from stats_ext.hrrst; +create or replace view ${stats_db_name}.rndexpenditure as select * from stats_ext.rndexpediture; +create or replace view ${stats_db_name}.rndgdpexpenditure as select * from stats_ext.rndgdpexpenditure; +create or replace view ${stats_db_name}.doctoratestudents as select * from stats_ext.doctoratestudents; +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 table ${stats_db_name}.result_instance stored as parquet as +create table if not exists ${stats_db_name}.result_instance stored as parquet as select distinct r.* from ( select substr(r.id, 4) as id, inst.accessright.classname as accessright, inst.accessright.openaccessroute as accessright_uw, substr(inst.collectedfrom.key, 4) as collectedfrom, @@ -44,7 +44,7 @@ from ( from ${openaire_db_name}.result r lateral view explode(r.instance) instances as inst lateral view explode(inst.pid) pids as p) r join ${stats_db_name}.result res on res.id=r.id; -create table ${stats_db_name}.result_apc STORED AS PARQUET as +create table if not exists ${stats_db_name}.result_apc STORED AS PARQUET as select r.id, r.amount, r.currency from ( select substr(r.id, 4) as id, cast(inst.processingchargeamount.value as float) as amount, inst.processingchargecurrency.value as currency @@ -52,4 +52,4 @@ from ( join ${stats_db_name}.result res on res.id=r.id where r.amount is not null; -create view ${stats_db_name}.issn_gold_oa_dataset as select * from ${external_stats_db_name}.issn_gold_oa_dataset; \ No newline at end of file +create or replace view ${stats_db_name}.issn_gold_oa_dataset as select * from ${external_stats_db_name}.issn_gold_oa_dataset; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql index 21d4efbf7..36b34cc3c 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 @@ -342,40 +342,6 @@ FROM publication_datasources pd ANALYZE TABLE indi_pub_hybrid_oa_with_cc COMPUTE STATISTICS; - 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_hybrid_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; - -ANALYZE TABLE indi_pub_bronze_oa COMPUTE STATISTICS; - create table if not exists indi_pub_hybrid stored as parquet as WITH gold_oa AS ( SELECT issn_l, @@ -775,26 +741,61 @@ from result p ANALYZE TABLE indi_result_with_pid COMPUTE STATISTICS; -create table if not exists indi_impact_measures as -select distinct substr(id, 4), measures_ids.id impactmetric, measures_ids.unit.value[0] score, -cast(measures_ids.unit.value[0] as decimal(6,3)) score_dec, measures_ids.unit.value[1] class -from 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 TEMPORARY TABLE pub_fos_totals as select rf.id, count(distinct lvl3) totals from result_fos rf group by rf.id; create table if not exists indi_pub_interdisciplinarity as -select distinct p.id, coalesce(indi_pub_is_interdisciplinary, 0) -as indi_pub_is_interdisciplinary +select distinct p.id as id, coalesce(is_interdisciplinary, 0) +as is_interdisciplinary from pub_fos_totals p left outer join ( -select pub_fos_totals.id, 1 as indi_pub_is_interdisciplinary from pub_fos_totals -where totals>10) tmp on p.id=tmp.id; +select pub_fos_totals.id, 1 as is_interdisciplinary from pub_fos_totals +where totals>1) tmp on p.id=tmp.id; drop table pub_fos_totals purge; -ANALYZE TABLE indi_pub_interdisciplinarity COMPUTE STATISTICS; \ No newline at end of file +ANALYZE TABLE indi_pub_interdisciplinarity COMPUTE STATISTICS; + +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 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; + +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_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 121ee6e7f..442e623cd 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 @@ -50,7 +50,9 @@ create table TARGET.result stored as parquet as 'openorgs____::846cb428d3f52a445f7275561a7beb5d', -- University of Manitoba 'openorgs____::eb391317ed0dc684aa81ac16265de041', -- Universitat Rovira i Virgili 'openorgs____::66aa9fc2fceb271423dfabcc38752dc0', -- Lund University - 'openorgs____::3cff625a4370d51e08624cc586138b2f' -- IMT Atlantique + 'openorgs____::3cff625a4370d51e08624cc586138b2f', -- IMT Atlantique + 'openorgs____::c0b262bd6eab819e4c994914f9c010e2', -- National Institute of Geophysics and Volcanology + 'openorgs____::1624ff7c01bb641b91f4518539a0c28a' -- Vrije Universiteit Amsterdam ))) foo; ANALYZE TABLE TARGET.result COMPUTE STATISTICS; \ No newline at end of file