master #59
|
@ -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 ) {}
|
|
@ -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)
|
||||||
|
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -82,7 +82,7 @@ object BioDBToOAF {
|
||||||
"Springer Nature"
|
"Springer Nature"
|
||||||
)
|
)
|
||||||
val EBICollectedFrom: KeyValue = OafMapperUtils.keyValue(
|
val EBICollectedFrom: KeyValue = OafMapperUtils.keyValue(
|
||||||
"10|opendoar____::83e60e09c222f206c725385f53d7e567c",
|
"10|opendoar____::3e60e09c222f206c725385f53d7e567c",
|
||||||
"EMBL-EBIs Protein Data Bank in Europe (PDBe)"
|
"EMBL-EBIs Protein Data Bank in Europe (PDBe)"
|
||||||
)
|
)
|
||||||
val pubmedCollectedFrom: KeyValue =
|
val pubmedCollectedFrom: KeyValue =
|
||||||
|
|
|
@ -4,7 +4,10 @@ package eu.dnetlib.dhp.oa.dedup;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Optional;
|
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.io.IOUtils;
|
||||||
|
import org.apache.commons.lang3.StringUtils;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.JavaPairRDD;
|
import org.apache.spark.api.java.JavaPairRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
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.application.ArgumentApplicationParser;
|
||||||
import eu.dnetlib.dhp.oa.dedup.model.Block;
|
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.schema.oaf.Relation;
|
||||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||||
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
|
||||||
|
@ -73,9 +74,16 @@ public class SparkCreateSimRels extends AbstractSparkAction {
|
||||||
log.info("actionSetId: '{}'", actionSetId);
|
log.info("actionSetId: '{}'", actionSetId);
|
||||||
log.info("workingPath: '{}'", workingPath);
|
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 each dedup configuration
|
||||||
for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) {
|
for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) {
|
||||||
|
|
||||||
|
final long start = System.currentTimeMillis();
|
||||||
|
|
||||||
final String entity = dedupConf.getWf().getEntityType();
|
final String entity = dedupConf.getWf().getEntityType();
|
||||||
final String subEntity = dedupConf.getWf().getSubEntityValue();
|
final String subEntity = dedupConf.getWf().getSubEntityValue();
|
||||||
log.info("Creating simrels for: '{}'", subEntity);
|
log.info("Creating simrels for: '{}'", subEntity);
|
||||||
|
@ -85,6 +93,7 @@ public class SparkCreateSimRels extends AbstractSparkAction {
|
||||||
|
|
||||||
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
||||||
|
|
||||||
|
|
||||||
JavaPairRDD<String, MapDocument> mapDocuments = sc
|
JavaPairRDD<String, MapDocument> mapDocuments = sc
|
||||||
.textFile(DedupUtility.createEntityPath(graphBasePath, subEntity))
|
.textFile(DedupUtility.createEntityPath(graphBasePath, subEntity))
|
||||||
.repartition(numPartitions)
|
.repartition(numPartitions)
|
||||||
|
@ -109,6 +118,13 @@ public class SparkCreateSimRels extends AbstractSparkAction {
|
||||||
Encoders.bean(Relation.class));
|
Encoders.bean(Relation.class));
|
||||||
|
|
||||||
saveParquet(simRels, outputPath, SaveMode.Overwrite);
|
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);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -28,5 +28,17 @@
|
||||||
"paramLongName": "numPartitions",
|
"paramLongName": "numPartitions",
|
||||||
"paramDescription": "number of partitions for the similarity relations intermediate phases",
|
"paramDescription": "number of partitions for the similarity relations intermediate phases",
|
||||||
"paramRequired": false
|
"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
|
||||||
}
|
}
|
||||||
]
|
]
|
|
@ -52,10 +52,11 @@
|
||||||
"paramLongName": "iterations",
|
"paramLongName": "iterations",
|
||||||
"paramDescription": "the number of iterations to be computed",
|
"paramDescription": "the number of iterations to be computed",
|
||||||
"paramRequired": false
|
"paramRequired": false
|
||||||
},{
|
},
|
||||||
"paramName": "pop",
|
{
|
||||||
"paramLongName": "projectOrganizationPath",
|
"paramName": "pop",
|
||||||
"paramDescription": "the number of iterations to be computed",
|
"paramLongName": "projectOrganizationPath",
|
||||||
"paramRequired": true
|
"paramDescription": "the number of iterations to be computed",
|
||||||
}
|
"paramRequired": true
|
||||||
|
}
|
||||||
]
|
]
|
|
@ -181,6 +181,7 @@
|
||||||
<arg>--leavesPath</arg><arg>${workingDir}/preparedInfo/leavesPath</arg>
|
<arg>--leavesPath</arg><arg>${workingDir}/preparedInfo/leavesPath</arg>
|
||||||
<arg>--childParentPath</arg><arg>${workingDir}/preparedInfo/childParentPath</arg>
|
<arg>--childParentPath</arg><arg>${workingDir}/preparedInfo/childParentPath</arg>
|
||||||
<arg>--resultOrgPath</arg><arg>${workingDir}/preparedInfo/resultOrgPath</arg>
|
<arg>--resultOrgPath</arg><arg>${workingDir}/preparedInfo/resultOrgPath</arg>
|
||||||
|
<arg>--projectOrganizationPath</arg><arg>${workingDir}/preparedInfo/projectOrganizationPath</arg>
|
||||||
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
<arg>--hive_metastore_uris</arg><arg>${hive_metastore_uris}</arg>
|
||||||
<arg>--workingDir</arg><arg>${workingDir}/working</arg>
|
<arg>--workingDir</arg><arg>${workingDir}/working</arg>
|
||||||
<arg>--iterations</arg><arg>${iterations}</arg>
|
<arg>--iterations</arg><arg>${iterations}</arg>
|
||||||
|
|
|
@ -18,7 +18,7 @@ SELECT
|
||||||
ARRAY(SELECT DISTINCT pid FROM unnest(array_cat(
|
ARRAY(SELECT DISTINCT pid FROM unnest(array_cat(
|
||||||
array_agg(DISTINCT i.otherid || '###' || i.type || '@@@dnet:pid_types'),
|
array_agg(DISTINCT i.otherid || '###' || i.type || '@@@dnet:pid_types'),
|
||||||
array_agg(DISTINCT idup.otherid || '###' || idup.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_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_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,
|
(array_remove(array_cat(ARRAY[o.ec_nonprofit], array_agg(od.ec_nonprofit)), NULL))[1] AS ecnonprofit,
|
||||||
|
|
|
@ -10,7 +10,7 @@ export SOURCE=$1
|
||||||
export PRODUCTION=$2
|
export PRODUCTION=$2
|
||||||
|
|
||||||
echo "Updating ${PRODUCTION} database"
|
echo "Updating ${PRODUCTION} database"
|
||||||
impala-shell -q "create database if not exists ${PRODUCTION}"
|
impala-shell -i impala-cluster-dn1.openaire.eu -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 -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 -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 -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!"
|
echo "Production db ready!"
|
|
@ -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
|
|
@ -80,10 +80,10 @@
|
||||||
<shell xmlns="uri:oozie:shell-action:0.1">
|
<shell xmlns="uri:oozie:shell-action:0.1">
|
||||||
<job-tracker>${jobTracker}</job-tracker>
|
<job-tracker>${jobTracker}</job-tracker>
|
||||||
<name-node>${nameNode}</name-node>
|
<name-node>${nameNode}</name-node>
|
||||||
<exec>updateProductionViews.sh</exec>
|
<exec>updateProductionViewsMonitor.sh</exec>
|
||||||
<argument>${monitor_db_name}</argument>
|
<argument>${monitor_db_name}</argument>
|
||||||
<argument>${monitor_db_production_name}</argument>
|
<argument>${monitor_db_production_name}</argument>
|
||||||
<file>updateProductionViews.sh</file>
|
<file>updateProductionViewsMonitor.sh</file>
|
||||||
</shell>
|
</shell>
|
||||||
<ok to="updateObservatoryViews"/>
|
<ok to="updateObservatoryViews"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
|
|
@ -7,7 +7,7 @@ then
|
||||||
fi
|
fi
|
||||||
|
|
||||||
#export HADOOP_USER_NAME="dimitris.pierrakos"
|
#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"
|
export PROD_USAGE_STATS_DB="openaire_prod_usage_stats"
|
||||||
function copydb() {
|
function copydb() {
|
||||||
db=$1
|
db=$1
|
||||||
|
@ -21,8 +21,22 @@ function copydb() {
|
||||||
# change ownership to impala
|
# change ownership to impala
|
||||||
hdfs dfs -conf /etc/impala_cluster/hdfs-site.xml -chmod -R 777 /tmp/$FILE/${db}.db
|
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";
|
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 -i impala-cluster-dn1.openaire.eu -q "create database ${db}";
|
||||||
|
|
||||||
impala-shell --user $HADOOP_USER_NAME -q "INVALIDATE METADATA"
|
impala-shell --user $HADOOP_USER_NAME -q "INVALIDATE METADATA"
|
||||||
|
|
|
@ -10,6 +10,12 @@ function createShadowDB() {
|
||||||
SOURCE=$1
|
SOURCE=$1
|
||||||
SHADOW=$2
|
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 "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 -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 -
|
# 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 -
|
||||||
|
|
|
@ -49,5 +49,5 @@ FROM (
|
||||||
WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on o.datasource = d.id;
|
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
|
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;
|
lateral view explode (instance.accessright.openaccessroute) openaccessroute as accessroute;
|
||||||
|
|
|
@ -34,3 +34,11 @@ union all
|
||||||
select * from ${stats_db_name}.software_refereed
|
select * from ${stats_db_name}.software_refereed
|
||||||
union all
|
union all
|
||||||
select * from ${stats_db_name}.otherresearchproduct_refereed;
|
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;
|
||||||
|
|
|
@ -1,21 +1,21 @@
|
||||||
-------------------------------------------
|
-------------------------------------------
|
||||||
--- Extra tables, mostly used by indicators
|
--- 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
|
select r.id, count(distinct p.id) as count
|
||||||
from ${stats_db_name}.result r
|
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}.result_projects rp on rp.id=r.id
|
||||||
left outer join ${stats_db_name}.project p on p.id=rp.project
|
left outer join ${stats_db_name}.project p on p.id=rp.project
|
||||||
group by r.id;
|
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
|
select r.id, count(distinct p.funder) as count
|
||||||
from ${stats_db_name}.result r
|
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}.result_projects rp on rp.id=r.id
|
||||||
left outer join ${stats_db_name}.project p on p.id=rp.project
|
left outer join ${stats_db_name}.project p on p.id=rp.project
|
||||||
group by r.id;
|
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 (
|
with rcount as (
|
||||||
select p.id as pid, count(distinct r.id) as `count`, r.type as type
|
select p.id as pid, count(distinct r.id) as `count`, r.type as type
|
||||||
from ${stats_db_name}.project p
|
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
|
from rcount
|
||||||
group by rcount.pid;
|
group by rcount.pid;
|
||||||
|
|
||||||
create view ${stats_db_name}.rndexpenditure as select * from stats_ext.rndexpediture;
|
create or replace view ${stats_db_name}.rndexpenditure as select * from stats_ext.rndexpediture;
|
||||||
create view ${stats_db_name}.rndgdpexpenditure as select * from stats_ext.rndgdpexpenditure;
|
create or replace view ${stats_db_name}.rndgdpexpenditure as select * from stats_ext.rndgdpexpenditure;
|
||||||
create view ${stats_db_name}.doctoratestudents as select * from stats_ext.doctoratestudents;
|
create or replace view ${stats_db_name}.doctoratestudents as select * from stats_ext.doctoratestudents;
|
||||||
create view ${stats_db_name}.totalresearchers as select * from stats_ext.totalresearchers;
|
create or replace view ${stats_db_name}.totalresearchers as select * from stats_ext.totalresearchers;
|
||||||
create view ${stats_db_name}.totalresearchersft as select * from stats_ext.totalresearchersft;
|
create or replace 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}.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.*
|
select distinct r.*
|
||||||
from (
|
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,
|
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
|
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;
|
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
|
select r.id, r.amount, r.currency
|
||||||
from (
|
from (
|
||||||
select substr(r.id, 4) as id, cast(inst.processingchargeamount.value as float) as amount, inst.processingchargecurrency.value as currency
|
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
|
join ${stats_db_name}.result res on res.id=r.id
|
||||||
where r.amount is not null;
|
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;
|
create or replace view ${stats_db_name}.issn_gold_oa_dataset as select * from ${external_stats_db_name}.issn_gold_oa_dataset;
|
|
@ -342,40 +342,6 @@ FROM publication_datasources pd
|
||||||
|
|
||||||
ANALYZE TABLE indi_pub_hybrid_oa_with_cc COMPUTE STATISTICS;
|
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
|
create table if not exists indi_pub_hybrid stored as parquet as
|
||||||
WITH gold_oa AS ( SELECT
|
WITH gold_oa AS ( SELECT
|
||||||
issn_l,
|
issn_l,
|
||||||
|
@ -775,26 +741,61 @@ from result p
|
||||||
|
|
||||||
ANALYZE TABLE indi_result_with_pid COMPUTE STATISTICS;
|
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
|
CREATE TEMPORARY TABLE pub_fos_totals as
|
||||||
select rf.id, count(distinct lvl3) totals from result_fos rf
|
select rf.id, count(distinct lvl3) totals from result_fos rf
|
||||||
group by rf.id;
|
group by rf.id;
|
||||||
|
|
||||||
create table if not exists indi_pub_interdisciplinarity as
|
create table if not exists indi_pub_interdisciplinarity as
|
||||||
select distinct p.id, coalesce(indi_pub_is_interdisciplinary, 0)
|
select distinct p.id as id, coalesce(is_interdisciplinary, 0)
|
||||||
as indi_pub_is_interdisciplinary
|
as is_interdisciplinary
|
||||||
from pub_fos_totals p
|
from pub_fos_totals p
|
||||||
left outer join (
|
left outer join (
|
||||||
select pub_fos_totals.id, 1 as indi_pub_is_interdisciplinary from pub_fos_totals
|
select pub_fos_totals.id, 1 as is_interdisciplinary from pub_fos_totals
|
||||||
where totals>10) tmp on p.id=tmp.id;
|
where totals>1) tmp on p.id=tmp.id;
|
||||||
|
|
||||||
drop table pub_fos_totals purge;
|
drop table pub_fos_totals purge;
|
||||||
|
|
||||||
ANALYZE TABLE indi_pub_interdisciplinarity COMPUTE STATISTICS;
|
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;
|
|
@ -50,7 +50,9 @@ create table TARGET.result stored as parquet as
|
||||||
'openorgs____::846cb428d3f52a445f7275561a7beb5d', -- University of Manitoba
|
'openorgs____::846cb428d3f52a445f7275561a7beb5d', -- University of Manitoba
|
||||||
'openorgs____::eb391317ed0dc684aa81ac16265de041', -- Universitat Rovira i Virgili
|
'openorgs____::eb391317ed0dc684aa81ac16265de041', -- Universitat Rovira i Virgili
|
||||||
'openorgs____::66aa9fc2fceb271423dfabcc38752dc0', -- Lund University
|
'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;
|
))) foo;
|
||||||
|
|
||||||
ANALYZE TABLE TARGET.result COMPUTE STATISTICS;
|
ANALYZE TABLE TARGET.result COMPUTE STATISTICS;
|
Loading…
Reference in New Issue