Merge branch 'beta' into orcid_import

This commit is contained in:
Claudio Atzori 2023-12-01 15:05:35 +01:00
commit 09d061e90b
35 changed files with 908 additions and 276 deletions

View File

@ -0,0 +1,75 @@
<?xml version="1.0" encoding="UTF-8"?>
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-workflows</artifactId>
<version>1.2.5-SNAPSHOT</version>
</parent>
<artifactId>dhp-stats-actionsets</artifactId>
<dependencies>
<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_2.11</artifactId>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_2.11</artifactId>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-common</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>net.sf.saxon</groupId>
<artifactId>Saxon-HE</artifactId>
</dependency>
<dependency>
<groupId>dom4j</groupId>
<artifactId>dom4j</artifactId>
</dependency>
<dependency>
<groupId>xml-apis</groupId>
<artifactId>xml-apis</artifactId>
</dependency>
<dependency>
<groupId>jaxen</groupId>
<artifactId>jaxen</artifactId>
</dependency>
<dependency>
<groupId>org.json</groupId>
<artifactId>json</artifactId>
</dependency>
<!-- https://mvnrepository.com/artifact/org.apache.poi/poi-ooxml -->
<dependency>
<groupId>org.apache.poi</groupId>
<artifactId>poi-ooxml</artifactId>
</dependency>
<!-- https://mvnrepository.com/artifact/org.apache.commons/commons-compress -->
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-compress</artifactId>
</dependency>
<dependency>
<groupId>org.mongodb</groupId>
<artifactId>mongo-java-driver</artifactId>
</dependency>
</dependencies>
</project>

View File

@ -0,0 +1,162 @@
package eu.dnetlib.dhp.actionmanager.stats_actionsets;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession;
import java.io.Serializable;
import java.util.ArrayList;
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.api.java.function.MapGroupsFunction;
import org.apache.spark.sql.*;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.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.common.HdfsSupport;
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 scala.Tuple2;
/**
* created the Atomic Action for each type of results
*/
public class StatsAtomicActionsJob implements Serializable {
private static final Logger log = LoggerFactory.getLogger(StatsAtomicActionsJob.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
public static <I extends Result> void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils
.toString(
StatsAtomicActionsJob.class
.getResourceAsStream(
"/eu/dnetlib/dhp/actionmanager/stats_actionsets/input_actionset_parameter.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 outputPath = parser.get("outputPath");
log.info("outputPath {}: ", outputPath);
SparkConf conf = new SparkConf();
conf.set("hive.metastore.uris", parser.get("hive_metastore_uris"));
conf.set("spark.speculation", "false");
conf.set("spark.hadoop.mapreduce.map.speculative", "false");
conf.set("spark.hadoop.mapreduce.reduce.speculative", "false");
final String dbname = parser.get("statsDB");
final String workingPath = parser.get("workingPath");
runWithSparkHiveSession(
conf,
isSparkSessionManaged,
spark -> {
removeOutputDir(spark, outputPath);
prepareResultEnhancement(dbname, spark, workingPath + "/resultEnhancements", "id");
writeActionSet(spark, workingPath, outputPath);
});
}
private static void prepareResultEnhancement(String dbname, SparkSession spark, String workingPath,
String resultAttributeName) {
spark
.sql(
String
.format(
"select r.%s as id, is_gold, is_bronze_oa, is_hybrid,green_oa, in_diamond_journal,f.publicly_funded as publicly_funded "
+
"from %s.publication r " +
"left outer join %s.indi_pub_bronze_oa b on r.id=b.id " +
"left outer join %s.indi_pub_gold_oa g on r.id=g.id " +
"left outer join %s.indi_pub_hybrid h on r.id=h.id " +
"left outer join %s.indi_pub_green_oa gr on r.id=gr.id " +
"left outer join %s.indi_pub_diamond d on b.id=d.id " +
"left outer join %s.indi_pub_publicly_funded f on r.id=f.id ",
resultAttributeName, dbname, dbname, dbname, dbname, dbname, dbname, dbname))
.as(Encoders.bean(StatsResultEnhancementModel.class))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(workingPath);
}
public static void writeActionSet(SparkSession spark, String inputPath, String outputPath) {
getResultEnhancements(spark, inputPath + "/resultEnhancements")
.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,
GzipCodec.class);
}
private static Dataset<Result> getResultEnhancements(SparkSession spark, String inputPath) {
return readPath(spark, inputPath, StatsResultEnhancementModel.class)
.map((MapFunction<StatsResultEnhancementModel, Result>) usm -> {
Result r = new Result();
r.setId("50|" + usm.getId());
r.setIsInDiamondJournal(usm.isIn_diamond_journal());
r.setIsGreen(usm.isGreen_oa());
r.setPubliclyFunded(usm.isPublicly_funded());
if (usm.isIs_gold())
r.setOpenAccessColor(OpenAccessColor.gold);
else if (usm.isIs_hybrid())
r.setOpenAccessColor(OpenAccessColor.hybrid);
else if (usm.isIs_bronze_oa())
r.setOpenAccessColor(OpenAccessColor.bronze);
return r;
}, Encoders.bean(Result.class));
}
private static void removeOutputDir(SparkSession spark, String path) {
HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration());
}
public static <R> Dataset<R> readPath(
SparkSession spark, String inputPath, Class<R> clazz) {
return spark
.read()
.textFile(inputPath)
.map((MapFunction<String, R>) value -> OBJECT_MAPPER.readValue(value, clazz), Encoders.bean(clazz));
}
}

View File

@ -0,0 +1,76 @@
package eu.dnetlib.dhp.actionmanager.stats_actionsets;
import java.io.Serializable;
import eu.dnetlib.dhp.schema.oaf.*;
/**
* @author dimitris.pierrakos
* @Date 30/10/23
*/
public class StatsResultEnhancementModel implements Serializable {
private String id;
private Boolean is_gold;
private Boolean is_bronze_oa;
private Boolean is_hybrid;
private boolean in_diamond_journal;
private boolean green_oa;
private boolean publicly_funded;
public String getId() {
return id;
}
public void setId(String id) {
this.id = id;
}
public Boolean isIs_gold() {
return is_gold;
}
public void setIs_gold(Boolean is_gold) {
this.is_gold = is_gold;
}
public Boolean isIs_bronze_oa() {
return is_bronze_oa;
}
public void setIs_bronze_oa(Boolean is_bronze_oa) {
this.is_bronze_oa = is_bronze_oa;
}
public Boolean isIs_hybrid() {
return is_hybrid;
}
public void setIs_hybrid(Boolean is_hybrid) {
this.is_hybrid = is_hybrid;
}
public boolean isIn_diamond_journal() {
return in_diamond_journal;
}
public void setIn_diamond_journal(boolean in_diamond_journal) {
this.in_diamond_journal = in_diamond_journal;
}
public boolean isGreen_oa() {
return green_oa;
}
public void setGreen_oa(boolean green_oa) {
this.green_oa = green_oa;
}
public boolean isPublicly_funded() {
return publicly_funded;
}
public void setPublicly_funded(boolean publicly_funded) {
this.publicly_funded = publicly_funded;
}
}

View File

@ -0,0 +1,32 @@
[
{
"paramName": "issm",
"paramLongName": "isSparkSessionManaged",
"paramDescription": "when true will stop SparkSession after job execution",
"paramRequired": false
},
{
"paramName": "hmu",
"paramLongName": "hive_metastore_uris",
"paramDescription": "the URI for the hive metastore",
"paramRequired": true
},
{
"paramName": "o",
"paramLongName": "outputPath",
"paramDescription": "the path of the new ActionSet",
"paramRequired": true
},
{
"paramName": "sdb",
"paramLongName": "statsDB",
"paramDescription": "the name of the stats db to be used",
"paramRequired": true
},
{
"paramName": "wp",
"paramLongName": "workingPath",
"paramDescription": "the workingPath where to save the content of the usage_stats table",
"paramRequired": true
}
]

View File

@ -0,0 +1,30 @@
<configuration>
<property>
<name>jobTracker</name>
<value>yarnRM</value>
</property>
<property>
<name>nameNode</name>
<value>hdfs://nameservice1</value>
</property>
<property>
<name>oozie.use.system.libpath</name>
<value>true</value>
</property>
<property>
<name>hiveMetastoreUris</name>
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
</property>
<property>
<name>hiveJdbcUrl</name>
<value>jdbc:hive2://iis-cdh5-test-m3.ocean.icm.edu.pl:10000</value>
</property>
<property>
<name>hiveDbName</name>
<value>openaire</value>
</property>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>

View File

@ -0,0 +1,97 @@
<workflow-app name="StatsActionSets" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>outputPath</name>
<description>the path where to store the actionset</description>
</property>
<property>
<name>statsDB</name>
<description>the name of the stats db to be used</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
<property>
<name>oozieActionShareLibForSpark2</name>
<description>oozie action sharelib for spark 2.*</description>
</property>
<property>
<name>spark2ExtraListeners</name>
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
<description>spark 2.* extra listeners classname</description>
</property>
<property>
<name>spark2SqlQueryExecutionListeners</name>
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
<description>spark 2.* sql query execution listeners classname</description>
</property>
<property>
<name>spark2YarnHistoryServerAddress</name>
<description>spark 2.* yarn history server address</description>
</property>
<property>
<name>spark2EventLogDir</name>
<description>spark 2.* event log dir location</description>
</property>
</parameters>
<global>
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<configuration>
<property>
<name>mapreduce.job.queuename</name>
<value>${queueName}</value>
</property>
<property>
<name>oozie.launcher.mapred.job.queue.name</name>
<value>${oozieLauncherQueueName}</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>${oozieActionShareLibForSpark2}</value>
</property>
</configuration>
</global>
<start to="atomicactionsStats"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="atomicactionsStats">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Produces the atomic action with the stats green_oa for results</name>
<class>eu.dnetlib.dhp.actionmanager.stats_actionsets.StatsAtomicActionsJob</class>
<jar>dhp-stats-actionsets-${projectVersion}.jar</jar>
<spark-opts>
--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}
</spark-opts>
<arg>--hive_metastore_uris</arg><arg>${hiveMetastoreUris}</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
<arg>--statsDB</arg><arg>${statsDB}</arg>
<arg>--workingPath</arg><arg>${workingDir}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,12 @@
# Set root logger level to DEBUG and its only appender to A1.
log4j.rootLogger=INFO, A1
# A1 is set to be a ConsoleAppender.
log4j.appender.A1=org.apache.log4j.ConsoleAppender
# A1 uses PatternLayout.
log4j.appender.A1.layout=org.apache.log4j.PatternLayout
log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
log4j.logger.org.apache.spark=FATAL
log4j.logger.org.spark_project=FATAL

View File

@ -6,7 +6,6 @@ then
ln -sfn ${PYTHON_EGG_CACHE}${link_folder} ${link_folder}
fi
#export HADOOP_USER_NAME="dimitris.pierrakos"
export HADOOP_USER_NAME=$6
export PROD_USAGE_STATS_DB="openaire_prod_usage_stats"
function copydb() {

View File

@ -0,0 +1,42 @@
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
function createPDFsAggregated() {
db=$1
impala-shell --user $HADOOP_USER_NAME -i impala-cluster-dn1.openaire.eu -d ${db} -q "drop table if exists indi_is_result_accessible";
impala-shell --user $HADOOP_USER_NAME -i impala-cluster-dn1.openaire.eu -d ${db} -q "create table indi_is_result_accessible stored as parquet as
select distinct p.id, coalesce(is_result_accessible, 0) as is_result_accessible from result p
left outer join
(select id, 1 as is_result_accessible from (select pl.* from result r
join pdfaggregation_i.publication p on r.id=p.id
join pdfaggregation_i.payload pl on pl.id=p.id
union all
select pl.* from result r
join pdfaggregation_i.publication p on r.id=p.dedupid
join pdfaggregation_i.payload pl on pl.id=p.id) foo) tmp on p.id=tmp.id";
}
STATS_DB=$1
MONITOR_DB=$2
HADOOP_USER_NAME=$3
createPDFsAggregated $STATS_DB
createPDFsAggregated $MONITOR_DB
createPDFsAggregated $MONITOR_DB'_funded'
createPDFsAggregated $MONITOR_DB'_institutions'
createPDFsAggregated $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}
do
tmp=`echo "$i" | sed 's/'-'/'_'/g' | sed 's/'::'/'_'/g'`
createPDFsAggregated ${MONITOR_DB}'_'${tmp}
done

View File

@ -49,5 +49,7 @@ select * from openaire_prod_usage_stats.views_stats;
-- Creation date of the database
------------------------------------------------------------------------------------------------
------------------------------------------------------------------------------------------------
DROP TABLE IF EXISTS ${stats_db_name}.creation_date purge;
create table ${stats_db_name}.creation_date STORED AS PARQUET as
select date_format(current_date(), 'dd-MM-yyyy') as date;

View File

@ -20,6 +20,7 @@ WHERE project_tmp.id IN (SELECT pr.id
${stats_db_name}.result r
WHERE pr.result = r.id
AND r.type = 'publication');
DROP TABLE IF EXISTS ${stats_db_name}.stored purge;
CREATE TABLE ${stats_db_name}.project stored as parquet as
SELECT p.id,

View File

@ -1,22 +1,32 @@
------------------------------------------------------------------------------------------------------
-- Creating parquet tables from the updated temporary tables and removing unnecessary temporary tables
------------------------------------------------------------------------------------------------------
DROP TABLE IF EXISTS ${stats_db_name}.datasource purge;
CREATE TABLE ${stats_db_name}.datasource stored AS parquet AS
SELECT *
FROM ${stats_db_name}.datasource_tmp;
DROP TABLE IF EXISTS ${stats_db_name}.publication purge;
CREATE TABLE ${stats_db_name}.publication stored AS parquet AS
SELECT *
FROM ${stats_db_name}.publication_tmp;
DROP TABLE IF EXISTS ${stats_db_name}.dataset purge;
CREATE TABLE ${stats_db_name}.dataset stored AS parquet AS
SELECT *
FROM ${stats_db_name}.dataset_tmp;
DROP TABLE IF EXISTS ${stats_db_name}.software purge;
CREATE TABLE ${stats_db_name}.software stored AS parquet AS
SELECT *
FROM ${stats_db_name}.software_tmp;
DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct purge;
CREATE TABLE ${stats_db_name}.otherresearchproduct stored AS parquet AS
SELECT *
FROM ${stats_db_name}.otherresearchproduct_tmp;

View File

@ -5,6 +5,8 @@
-- Sources related tables/views
------------------------------------------------------
------------------------------------------------------
DROP TABLE IF EXISTS ${stats_db_name}.publication_sources purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_sources STORED AS PARQUET as
SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource
FROM (
@ -16,6 +18,8 @@ LEFT OUTER JOIN
from ${openaire_db_name}.datasource d
WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on p.datasource = d.id;
DROP TABLE IF EXISTS ${stats_db_name}.dataset_sources purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_sources STORED AS PARQUET as
SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource
FROM (
@ -27,6 +31,8 @@ LEFT OUTER JOIN
from ${openaire_db_name}.datasource d
WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on p.datasource = d.id;
DROP TABLE IF EXISTS ${stats_db_name}.software_sources purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_sources STORED AS PARQUET as
SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource
FROM (
@ -38,6 +44,8 @@ LEFT OUTER JOIN
from ${openaire_db_name}.datasource d
WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on p.datasource = d.id;
DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_sources purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_sources STORED AS PARQUET as
SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource
FROM (
@ -48,7 +56,7 @@ LEFT OUTER JOIN
SELECT substr(d.id, 4) id
from ${openaire_db_name}.datasource d
WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on p.datasource = d.id;
CREATE VIEW IF NOT EXISTS ${stats_db_name}.result_sources AS
SELECT * FROM ${stats_db_name}.publication_sources
UNION ALL
@ -58,6 +66,7 @@ SELECT * FROM ${stats_db_name}.software_sources
UNION ALL
SELECT * FROM ${stats_db_name}.otherresearchproduct_sources;
DROP TABLE IF EXISTS ${stats_db_name}.result_orcid purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_orcid STORED AS PARQUET as
select distinct res.id, regexp_replace(res.orcid, 'http://orcid.org/' ,'') as orcid
@ -69,6 +78,8 @@ from (
LATERAL VIEW explode(auth.pid.qualifier.classid) apt as author_pid_type
WHERE res.datainfo.deletedbyinference = FALSE and res.datainfo.invisible = FALSE and author_pid_type = 'orcid') as res;
DROP TABLE IF EXISTS ${stats_db_name}.result_result purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_result stored as parquet as
select substr(rel.source, 4) as source, substr(rel.target, 4) as target, relclass, subreltype
from ${openaire_db_name}.relation rel
@ -82,6 +93,8 @@ where reltype='resultResult'
and r2.resulttype.classname != 'other'
and rel.datainfo.deletedbyinference=false and rel.datainfo.invisible = FALSE;
DROP TABLE IF EXISTS ${stats_db_name}.result_citations_oc purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_citations_oc stored as parquet as
select substr(target, 4) as id, count(distinct substr(source, 4)) as citations
from ${openaire_db_name}.relation rel
@ -97,6 +110,8 @@ where relClass='Cites' and rel.datainfo.provenanceaction.classid = 'sysimport:cr
and rel.datainfo.deletedbyinference=false and rel.datainfo.invisible = FALSE
group by substr(target, 4);
DROP TABLE IF EXISTS ${stats_db_name}.result_references_oc purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_references_oc stored as parquet as
select substr(source, 4) as id, count(distinct substr(target, 4)) as references
from ${openaire_db_name}.relation rel

View File

@ -5,21 +5,29 @@
-- Licences related tables/views
------------------------------------------------------
------------------------------------------------------
DROP TABLE IF EXISTS ${stats_db_name}.publication_licenses purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_licenses STORED AS PARQUET AS
SELECT substr(p.id, 4) as id, licenses.value as type
SELECT substr(p.id, 4) as id, licenses.value as type
from ${openaire_db_name}.publication p LATERAL VIEW explode(p.instance.license) instances as licenses
where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false and p.datainfo.invisible = FALSE;
DROP TABLE IF EXISTS ${stats_db_name}.dataset_licenses purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_licenses STORED AS PARQUET AS
SELECT substr(p.id, 4) as id, licenses.value as type
from ${openaire_db_name}.dataset p LATERAL VIEW explode(p.instance.license) instances as licenses
where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false and p.datainfo.invisible = FALSE;
DROP TABLE IF EXISTS ${stats_db_name}.software_licenses purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_licenses STORED AS PARQUET AS
SELECT substr(p.id, 4) as id, licenses.value as type
from ${openaire_db_name}.software p LATERAL VIEW explode(p.instance.license) instances as licenses
where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false and p.datainfo.invisible = FALSE;
DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_licenses purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_licenses STORED AS PARQUET AS
SELECT substr(p.id, 4) as id, licenses.value as type
from ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.instance.license) instances as licenses
@ -34,10 +42,14 @@ SELECT * FROM ${stats_db_name}.software_licenses
UNION ALL
SELECT * FROM ${stats_db_name}.otherresearchproduct_licenses;
DROP TABLE IF EXISTS ${stats_db_name}.organization_pids purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.organization_pids STORED AS PARQUET AS
select substr(o.id, 4) as id, ppid.qualifier.classname as type, ppid.value as pid
from ${openaire_db_name}.organization o lateral view explode(o.pid) pids as ppid;
DROP TABLE IF EXISTS ${stats_db_name}.organization_sources purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.organization_sources STORED AS PARQUET as
SELECT o.id, case when d.id is null then 'other' else o.datasource end as datasource
FROM (
@ -48,6 +60,8 @@ FROM (
from ${openaire_db_name}.datasource d
WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on o.datasource = d.id;
DROP TABLE IF EXISTS ${stats_db_name}.result_accessroute purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_accessroute STORED AS PARQUET as
select distinct substr(id,4) as id, accessroute from ${openaire_db_name}.result
lateral view explode (instance.accessright.openaccessroute) openaccessroute as accessroute;

View File

@ -6,21 +6,29 @@
------------------------------------------------------
------------------------------------------------------
DROP TABLE IF EXISTS ${stats_db_name}.publication_refereed purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_refereed STORED AS PARQUET as
select substr(r.id, 4) as id, inst.refereed.classname as refereed
from ${openaire_db_name}.publication r lateral view explode(r.instance) instances as inst
where r.datainfo.deletedbyinference=false and r.datainfo.invisible = FALSE;
DROP TABLE IF EXISTS ${stats_db_name}.dataset_refereed purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_refereed STORED AS PARQUET as
select substr(r.id, 4) as id, inst.refereed.classname as refereed
from ${openaire_db_name}.dataset r lateral view explode(r.instance) instances as inst
where r.datainfo.deletedbyinference=false and r.datainfo.invisible = FALSE;
DROP TABLE IF EXISTS ${stats_db_name}.software_refereed purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_refereed STORED AS PARQUET as
select substr(r.id, 4) as id, inst.refereed.classname as refereed
from ${openaire_db_name}.software r lateral view explode(r.instance) instances as inst
where r.datainfo.deletedbyinference=false and r.datainfo.invisible = FALSE;
DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_refereed purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_refereed STORED AS PARQUET as
select substr(r.id, 4) as id, inst.refereed.classname as refereed
from ${openaire_db_name}.otherresearchproduct r lateral view explode(r.instance) instances as inst
@ -35,12 +43,16 @@ select * from ${stats_db_name}.software_refereed
union all
select * from ${stats_db_name}.otherresearchproduct_refereed;
DROP TABLE IF EXISTS ${stats_db_name}.indi_impact_measures purge;
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] 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';
DROP TABLE IF EXISTS ${stats_db_name}.result_apc_affiliations purge;
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,

View File

@ -1,6 +1,8 @@
-------------------------------------------
--- Extra tables, mostly used by indicators
DROP TABLE IF EXISTS ${stats_db_name}.result_projectcount purge;
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
@ -8,6 +10,8 @@ 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;
DROP TABLE IF EXISTS ${stats_db_name}.result_fundercount purge;
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
@ -15,6 +19,8 @@ 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;
DROP TABLE IF EXISTS ${stats_db_name}.project_resultcount purge;
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
@ -37,6 +43,8 @@ create or replace view ${stats_db_name}.totalresearchersft as select * from stat
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;
DROP TABLE IF EXISTS ${stats_db_name}.result_instance purge;
create table if not exists ${stats_db_name}.result_instance stored as parquet as
select distinct r.*
from (
@ -45,6 +53,8 @@ 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;
DROP TABLE IF EXISTS ${stats_db_name}.result_apc purge;
create table if not exists ${stats_db_name}.result_apc STORED AS PARQUET as
select r.id, r.amount, r.currency
from (

View File

@ -1,6 +1,18 @@
-- Sprint 1 ----
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 ${stats_db_name}.publication p
-- left outer join (
-- select p.id, 1 as green_oa
-- 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;
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 ${stats_db_name}.publication p
@ -11,7 +23,7 @@ from ${stats_db_name}.publication p
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
or ri.accessright = 'Embargo' or ri.accessright = 'Open Source') and datasource.name!='Other') tmp
on p.id= tmp.id;
drop table if exists ${stats_db_name}.indi_pub_grey_lit purge;
@ -104,7 +116,7 @@ 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 ${stats_db_name}.tmp purge;
drop table if exists ${stats_db_name}.tmp purge;
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
@ -118,7 +130,7 @@ 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 ${stats_db_name}.tmp purge;
drop table if exists ${stats_db_name}.tmp purge;
create TEMPORARY TABLE ${stats_db_name}.tmp AS
select o.id organization, o.name, ro.project as project from ${stats_db_name}.organization o
@ -133,7 +145,7 @@ from ${stats_db_name}.tmp as o1
where o1.organization<>o2.organization and o1.name<>o2.name
group by o1.name,o2.name, o1.organization, o2.organization;
drop table ${stats_db_name}.tmp purge;
drop table if exists ${stats_db_name}.tmp purge;
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
@ -149,7 +161,7 @@ from ${stats_db_name}.tmp as o1
where o1.organization<>o2.organization and o1.country<>o2.country
group by o1.organization, o2.country, o1.name;
drop table ${stats_db_name}.tmp purge;
drop table if exists ${stats_db_name}.tmp purge;
drop table if exists ${stats_db_name}.indi_funder_country_collab purge;
@ -178,20 +190,29 @@ from ${stats_db_name}.tmp as o1
where o1.country<>o2.country
group by o1.country, o2.country;
drop table ${stats_db_name}.tmp purge;
drop table if exists ${stats_db_name}.tmp purge;
---- Sprint 4 ----
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 ${stats_db_name}.publication_datasources pd
-- left outer join (
-- 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;
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 ${stats_db_name}.publication_datasources pd
left outer join (
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;
left outer join (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;
drop table if exists ${stats_db_name}.indi_pub_in_transformative purge;
@ -312,28 +333,55 @@ drop table if exists ${stats_db_name}.indi_pub_gold_oa purge;
-- JOIN gold_oa on issn.issn = gold_oa.issn) tmp
-- on pd.id=tmp.id;
--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;
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 issn from (
SELECT issn_l as issn from stats_ext.issn_gold_oa_dataset_v5
UNION ALL
SELECT issn as issn from stats_ext.issn_gold_oa_dataset_v5
UNION ALL
select issn from stats_ext.alljournals where journal_is_in_doaj=true or journal_is_oa=true
UNION ALL
select issn_l as issn from stats_ext.alljournals where journal_is_in_doaj=true or journal_is_oa=true) foo),
dd as (
select distinct * from (
select id, issn_printed as issn from ${stats_db_name}.datasource d where d.id like '%doajarticles%'
UNION ALL
select id, issn_online as issn from ${stats_db_name}.datasource d where d.id like '%doajarticles%'
UNION ALL
select id, issn_printed as issn from ${stats_db_name}.datasource d join gold_oa on gold_oa.issn=d.issn_printed
UNION ALL
select id, issn_online as issn from ${stats_db_name}.datasource d join gold_oa on gold_oa.issn=d.issn_online) foo
)
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;
select pd.id, 1 as is_gold
FROM ${stats_db_name}.publication_datasources pd
join dd on dd.id=pd.datasource
left outer join ${stats_db_name}.result_accessroute ra on ra.id = pd.id where ra.accessroute = 'gold') tmp on tmp.id=pd.id;
drop table if exists ${stats_db_name}.indi_pub_hybrid_oa_with_cc purge;
@ -421,15 +469,26 @@ drop table if exists ${stats_db_name}.indi_pub_hybrid purge;
-- 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
--select distinct 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 ${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
select distinct pd.id,coalesce(is_hybrid,0) is_hybrid from ${stats_db_name}.publication pd
left outer join (select pd.id, 1 as is_hybrid from ${stats_db_name}.publication pd
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
join ${stats_db_name}.datasource d on d.id=ri.hostedby
where indi_gold.is_gold=0 and ((d.type like '%Journal%' and ri.accessright!='Closed Access' and ri.accessright!='Restricted' and ri.license is not null) or
ra.accessroute='hybrid'))tmp
on pd.id=tmp.id;
drop table if exists ${stats_db_name}.indi_org_fairness purge;
@ -492,7 +551,7 @@ CREATE TEMPORARY TABLE ${stats_db_name}.allresults as select year, ro.organizati
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
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 result_fair.year=allresults.year;
@ -813,15 +872,17 @@ drop table if exists ${stats_db_name}.indi_pub_bronze_oa purge;
--and (d.type='Journal' or d.type='Journal Aggregator/Publisher')
--and ri.accessright='Open Access') tmp on tmp.id=p.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
create table ${stats_db_name}.indi_pub_bronze_oa stored as parquet as
select distinct pd.id,coalesce(is_bronze_oa,0) is_bronze_oa from ${stats_db_name}.publication pd
left outer join (select pd.id, 1 as is_bronze_oa from ${stats_db_name}.publication pd
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}.indi_pub_hybrid indi_hybrid on indi_hybrid.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
join ${stats_db_name}.datasource d on d.id=ri.hostedby
where indi_gold.is_gold=0 and indi_hybrid.is_hybrid=0
and ((d.type like '%Journal%' and ri.accessright!='Closed Access'
and ri.accessright!='Restricted' and ri.license is null) or ra.accessroute='bronze')) tmp
on pd.id=tmp.id;
CREATE TEMPORARY TABLE ${stats_db_name}.project_year_result_year as
@ -1116,3 +1177,24 @@ select allresults.ri_initiative, result_findable.no_result_findable/allresults.n
from allresults
join result_findable on result_findable.ri_initiative=allresults.ri_initiative;
create table if not exists ${stats_db_name}.indi_pub_publicly_funded stored as parquet as
with org_names_pids as
(select org.id,name, pid from ${stats_db_name}.organization org
join ${stats_db_name}.organization_pids op on org.id=op.id),
publicly_funded_orgs as
(select distinct name from
(select pf.name from stats_ext.insitutions_for_publicly_funded pf
join ${stats_db_name}.fundref f on f.name=pf.name where f.type='government'
union all
select pf.name from stats_ext.insitutions_for_publicly_funded pf
join ${stats_db_name}.project p on p.funder=pf.name
union all
select pf.name from stats_ext.insitutions_for_publicly_funded pf
join org_names_pids op on (op.name=pf.name or op.pid=pf.ror)
and pf.publicly_funded='yes') foo)
select distinct p.id, coalesce(publicly_funded, 0) as publicly_funded
from ${stats_db_name}.publication p
left outer join (
select distinct ro.id, 1 as publicly_funded from result_organization ro
join ${stats_db_name}.organization o on o.id=ro.organization
join publicly_funded_orgs pfo on o.name=pfo.name) tmp on p.id=tmp.id;

View File

@ -3,6 +3,8 @@
----------------------------------------------------
-- Peer reviewed:
drop table if exists ${stats_db_name}.result_peerreviewed purge;
create table IF NOT EXISTS ${stats_db_name}.result_peerreviewed STORED AS PARQUET as
select r.id as id, case when doi.doi_from_crossref=1 and grey.grey_lit=0 then true else false end as peer_reviewed
from ${stats_db_name}.result r
@ -10,12 +12,16 @@ left outer join ${stats_db_name}.indi_pub_doi_from_crossref doi on doi.id=r.id
left outer join ${stats_db_name}.indi_pub_grey_lit grey on grey.id=r.id;
-- Green OA:
drop table if exists ${stats_db_name}.result_greenoa purge;
create table IF NOT EXISTS ${stats_db_name}.result_greenoa STORED AS PARQUET as
select r.id, case when green.green_oa=1 then true else false end as green
from ${stats_db_name}.result r
left outer join ${stats_db_name}.indi_pub_green_oa green on green.id=r.id;
-- GOLD OA:
drop table if exists ${stats_db_name}.result_gold purge;
create table IF NOT EXISTS ${stats_db_name}.result_gold STORED AS PARQUET as
select r.id, case when gold.is_gold=1 then true else false end as gold
from ${stats_db_name}.result r

View File

@ -1,6 +1,7 @@
-- replace the creation of the result view to include the boolean fields from the previous tables (green, gold,
-- peer reviewed)
drop table if exists ${stats_db_name}.result_tmp;
CREATE TABLE ${stats_db_name}.result_tmp (
id STRING,
title STRING,

View File

@ -5,6 +5,7 @@
--------------------------------------------------------------
-- Publication temporary table
DROP TABLE IF EXISTS ${stats_db_name}.publication_tmp purge;
CREATE TABLE ${stats_db_name}.publication_tmp
(
id STRING,
@ -40,12 +41,16 @@ SELECT substr(p.id, 4) as id,
from ${openaire_db_name}.publication p
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.publication_classifications purge;
CREATE TABLE ${stats_db_name}.publication_classifications STORED AS PARQUET AS
SELECT substr(p.id, 4) as id, instancetype.classname as type
from ${openaire_db_name}.publication p
LATERAL VIEW explode(p.instance.instancetype) instances as instancetype
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.publication_concepts purge;
CREATE TABLE ${stats_db_name}.publication_concepts STORED AS PARQUET AS
SELECT substr(p.id, 4) as id, case
when contexts.context.id RLIKE '^[^::]+::[^::]+::.+$' then contexts.context.id
@ -55,6 +60,8 @@ from ${openaire_db_name}.publication p
LATERAL VIEW explode(p.context) contexts as context
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.publication_datasources purge;
CREATE TABLE ${stats_db_name}.publication_datasources STORED AS PARQUET as
SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource
FROM (
@ -66,29 +73,39 @@ FROM (
from ${openaire_db_name}.datasource d
WHERE d.datainfo.deletedbyinference = false and d.datainfo.invisible=false) d on p.datasource = d.id;
DROP TABLE IF EXISTS ${stats_db_name}.publication_languages purge;
CREATE TABLE ${stats_db_name}.publication_languages STORED AS PARQUET AS
select substr(p.id, 4) as id, p.language.classname as language
FROM ${openaire_db_name}.publication p
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.publication_oids purge;
CREATE TABLE ${stats_db_name}.publication_oids STORED AS PARQUET AS
SELECT substr(p.id, 4) AS id, oids.ids AS oid
FROM ${openaire_db_name}.publication p
LATERAL VIEW explode(p.originalid) oids AS ids
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.publication_pids purge;
CREATE TABLE ${stats_db_name}.publication_pids STORED AS PARQUET AS
SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value as pid
FROM ${openaire_db_name}.publication p
LATERAL VIEW explode(p.pid) pids AS ppid
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.publication_topics purge;
CREATE TABLE ${stats_db_name}.publication_topics STORED AS PARQUET as
select substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS TYPE, subjects.subject.value AS topic
FROM ${openaire_db_name}.publication p
LATERAL VIEW explode(p.subject) subjects AS subject
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.publication_citations purge;
CREATE TABLE ${stats_db_name}.publication_citations STORED AS PARQUET AS
SELECT substr(p.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites
FROM ${openaire_db_name}.publication p

View File

@ -1,79 +1,3 @@
--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 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
-- ) )) 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;
@ -91,85 +15,63 @@ 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 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 table TARGET.indi_pub_publicly_funded stored as parquet as select * from SOURCE.indi_pub_publicly_funded orig where exists (select 1 from TARGET.result r where r.id=orig.id);
create table TARGET.result_instance stored as parquet as select * from SOURCE.result_instance 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;
-- datasources
create view if not exists TARGET.datasource as select * from SOURCE.datasource;
@ -178,7 +80,6 @@ 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;
-- organizations
create view if not exists TARGET.organization as select * from SOURCE.organization;
@ -196,28 +97,27 @@ 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;
-- 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;
@ -226,32 +126,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;
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;
@ -262,17 +162,17 @@ 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.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;

View File

@ -78,10 +78,11 @@ create table TARGET.result stored as parquet as
'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;
'openorgs____::8839b55dae0c84d56fd533f52d5d483a', -- Leibniz Institute of Ecological Urban and Regional Development
'openorgs____::526468206bca24c1c90da6a312295cf4', -- Cyprus University of Technology
'openorgs____::b5ca9d4340e26454e367e2908ef3872f' -- Alma Mater Studiorum University of Bologna
--ANALYZE TABLE TARGET.result COMPUTE STATISTICS;
) )) foo;
create view if not exists TARGET.category as select * from SOURCE.category;
create view if not exists TARGET.concept as select * from SOURCE.concept;
@ -97,86 +98,63 @@ 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 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 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;
-- datasources
create view if not exists TARGET.datasource as select * from SOURCE.datasource;
@ -185,7 +163,6 @@ 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;
-- organizations
create view if not exists TARGET.organization as select * from SOURCE.organization;
@ -203,28 +180,26 @@ 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;
-- 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;
@ -233,32 +208,29 @@ 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;
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;
@ -269,19 +241,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;
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 table TARGET.result_instance stored as parquet as select * from SOURCE.result_instance orig where exists (select 1 from TARGET.result r where r.id=orig.id);
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 table TARGET.indi_pub_publicly_funded stored as parquet as select * from SOURCE.indi_pub_publicly_funded orig where exists (select 1 from TARGET.result r where r.id=orig.id);
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 view TARGET.indi_is_funder_plan_s as select * from SOURCE.indi_is_funder_plan_s;
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;

View File

@ -11,5 +11,4 @@ create table TARGET.result stored as parquet as
join SOURCE.context cont on cont.id=cat.context
-- join SOURCE.result
where rc.id=r.id and conc.category like CONTEXT)
) foo;
--ANALYZE TABLE TARGET.result COMPUTE STATISTICS;
) foo;

View File

@ -11,5 +11,4 @@ create table TARGET.result stored as parquet as
join SOURCE.context cont on cont.id=cat.context
-- join SOURCE.result
where rc.id=r.id and conc.category not in (CONTEXTS))
) foo;
--ANALYZE TABLE TARGET.result COMPUTE STATISTICS;
) foo;

View File

@ -4,6 +4,4 @@ create database if not exists TARGET;
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)
) foo;
--ANALYZE TABLE TARGET.result COMPUTE STATISTICS;
) foo;

View File

@ -58,7 +58,7 @@ create table TARGET.result stored as parquet as
'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;
'openorgs____::8839b55dae0c84d56fd533f52d5d483a', -- Leibniz Institute of Ecological Urban and Regional Development
'openorgs____::526468206bca24c1c90da6a312295cf4', -- Cyprus University of Technology
'openorgs____::b5ca9d4340e26454e367e2908ef3872f' -- Alma Mater Studiorum University of Bologna
))) foo;

View File

@ -8,7 +8,6 @@ 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;
create table ${observatory_db_name}.result_affiliated_country stored as parquet as
select
@ -39,7 +38,6 @@ 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;
create table ${observatory_db_name}.result_affiliated_year stored as parquet as
select
@ -70,7 +68,6 @@ 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;
create table ${observatory_db_name}.result_affiliated_year_country stored as parquet as
select
@ -101,7 +98,6 @@ 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;
create table ${observatory_db_name}.result_affiliated_datasource stored as parquet as
select
@ -134,8 +130,6 @@ 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;
create table ${observatory_db_name}.result_affiliated_datasource_country stored as parquet as
select
count(distinct r.id) as total,
@ -167,8 +161,6 @@ 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;
create table ${observatory_db_name}.result_affiliated_organization stored as parquet as
select
count(distinct r.id) as total,
@ -198,8 +190,6 @@ 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;
create table ${observatory_db_name}.result_affiliated_organization_country stored as parquet as
select
count(distinct r.id) as total,
@ -229,8 +219,6 @@ 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;
create table ${observatory_db_name}.result_affiliated_funder stored as parquet as
select
count(distinct r.id) as total,
@ -262,8 +250,6 @@ 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;
create table ${observatory_db_name}.result_affiliated_funder_country stored as parquet as
select
count(distinct r.id) as total,
@ -295,8 +281,6 @@ 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;
create table ${observatory_db_name}.result_deposited_country stored as parquet as
select
count(distinct r.id) as total,
@ -328,8 +312,6 @@ 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;
create table ${observatory_db_name}.result_deposited_year stored as parquet as
select
count(distinct r.id) as total,
@ -361,7 +343,6 @@ 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;
create table ${observatory_db_name}.result_deposited_year_country stored as parquet as
select
@ -394,8 +375,6 @@ 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;
create table ${observatory_db_name}.result_deposited_datasource stored as parquet as
select
count(distinct r.id) as total,
@ -427,8 +406,6 @@ 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;
create table ${observatory_db_name}.result_deposited_datasource_country stored as parquet as
select
count(distinct r.id) as total,
@ -460,8 +437,6 @@ 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;
create table ${observatory_db_name}.result_deposited_organization stored as parquet as
select
count(distinct r.id) as total,
@ -493,8 +468,6 @@ 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;
create table ${observatory_db_name}.result_deposited_organization_country stored as parquet as
select
count(distinct r.id) as total,
@ -526,8 +499,6 @@ 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;
create table ${observatory_db_name}.result_deposited_funder stored as parquet as
select
count(distinct r.id) as total,
@ -561,8 +532,6 @@ 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;
create table ${observatory_db_name}.result_deposited_funder_country stored as parquet as
select
count(distinct r.id) as total,
@ -595,5 +564,3 @@ from ${stats_db_name}.result r
group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid 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;

View File

@ -5,6 +5,8 @@
------------------------------------------------------
-- Dataset temporary table supporting updates
DROP TABLE IF EXISTS ${stats_db_name}.dataset_tmp purge;
CREATE TABLE ${stats_db_name}.dataset_tmp
(
id STRING,
@ -40,6 +42,8 @@ SELECT substr(d.id, 4) AS id,
FROM ${openaire_db_name}.dataset d
WHERE d.datainfo.deletedbyinference = FALSE and d.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.dataset_citations purge;
CREATE TABLE ${stats_db_name}.dataset_citations STORED AS PARQUET AS
SELECT substr(d.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites
FROM ${openaire_db_name}.dataset d
@ -47,12 +51,16 @@ FROM ${openaire_db_name}.dataset d
WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != ""
and d.datainfo.deletedbyinference = false and d.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.dataset_classifications purge;
CREATE TABLE ${stats_db_name}.dataset_classifications STORED AS PARQUET AS
SELECT substr(p.id, 4) AS id, instancetype.classname AS type
FROM ${openaire_db_name}.dataset p
LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.dataset_concepts purge;
CREATE TABLE ${stats_db_name}.dataset_concepts STORED AS PARQUET AS
SELECT substr(p.id, 4) as id, case
when contexts.context.id RLIKE '^[^::]+::[^::]+::.+$' then contexts.context.id
@ -62,6 +70,8 @@ from ${openaire_db_name}.dataset p
LATERAL VIEW explode(p.context) contexts as context
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.dataset_datasources purge;
CREATE TABLE ${stats_db_name}.dataset_datasources STORED AS PARQUET AS
SELECT p.id, case when d.id IS NULL THEN 'other' ELSE p.datasource END AS datasource
FROM (
@ -74,23 +84,31 @@ FROM (
FROM ${openaire_db_name}.datasource d
WHERE d.datainfo.deletedbyinference = false and d.datainfo.invisible=false) d ON p.datasource = d.id;
DROP TABLE IF EXISTS ${stats_db_name}.dataset_languages purge;
CREATE TABLE ${stats_db_name}.dataset_languages STORED AS PARQUET AS
SELECT substr(p.id, 4) AS id, p.language.classname AS language
FROM ${openaire_db_name}.dataset p
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.dataset_oids purge;
CREATE TABLE ${stats_db_name}.dataset_oids STORED AS PARQUET AS
SELECT substr(p.id, 4) AS id, oids.ids AS oid
FROM ${openaire_db_name}.dataset p
LATERAL VIEW explode(p.originalid) oids AS ids
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.dataset_pids purge;
CREATE TABLE ${stats_db_name}.dataset_pids STORED AS PARQUET AS
SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid
FROM ${openaire_db_name}.dataset p
LATERAL VIEW explode(p.pid) pids AS ppid
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.dataset_topics purge;
CREATE TABLE ${stats_db_name}.dataset_topics STORED AS PARQUET AS
SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic
FROM ${openaire_db_name}.dataset p

View File

@ -5,6 +5,7 @@
--------------------------------------------------------
-- Software temporary table supporting updates
DROP TABLE IF EXISTS ${stats_db_name}.software_tmp purge;
CREATE TABLE ${stats_db_name}.software_tmp
(
id STRING,
@ -40,6 +41,8 @@ SELECT substr(s.id, 4) as id,
from ${openaire_db_name}.software s
where s.datainfo.deletedbyinference = false and s.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.software_citations purge;
CREATE TABLE ${stats_db_name}.software_citations STORED AS PARQUET AS
SELECT substr(s.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites
FROM ${openaire_db_name}.software s
@ -47,6 +50,8 @@ FROM ${openaire_db_name}.software s
where xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != ""
and s.datainfo.deletedbyinference = false and s.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.software_classifications purge;
CREATE TABLE ${stats_db_name}.software_classifications STORED AS PARQUET AS
SELECT substr(p.id, 4) AS id, instancetype.classname AS type
FROM ${openaire_db_name}.software p
@ -62,6 +67,8 @@ FROM ${openaire_db_name}.software p
LATERAL VIEW explode(p.context) contexts AS context
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.software_datasources purge;
CREATE TABLE ${stats_db_name}.software_datasources STORED AS PARQUET AS
SELECT p.id, CASE WHEN d.id IS NULL THEN 'other' ELSE p.datasource end as datasource
FROM (
@ -74,23 +81,31 @@ FROM (
FROM ${openaire_db_name}.datasource d
WHERE d.datainfo.deletedbyinference = false and d.datainfo.invisible=false) d ON p.datasource = d.id;
DROP TABLE IF EXISTS ${stats_db_name}.software_languages purge;
CREATE TABLE ${stats_db_name}.software_languages STORED AS PARQUET AS
select substr(p.id, 4) AS id, p.language.classname AS language
FROM ${openaire_db_name}.software p
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.software_oids purge;
CREATE TABLE ${stats_db_name}.software_oids STORED AS PARQUET AS
SELECT substr(p.id, 4) AS id, oids.ids AS oid
FROM ${openaire_db_name}.software p
LATERAL VIEW explode(p.originalid) oids AS ids
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.software_pids purge;
CREATE TABLE ${stats_db_name}.software_pids STORED AS PARQUET AS
SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid
FROM ${openaire_db_name}.software p
LATERAL VIEW explode(p.pid) pids AS ppid
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.software_topics purge;
CREATE TABLE ${stats_db_name}.software_topics STORED AS PARQUET AS
SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic
FROM ${openaire_db_name}.software p

View File

@ -5,6 +5,8 @@
--------------------------------------------------------------------------------
-- Otherresearchproduct temporary table supporting updates
DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_tmp purge;
CREATE TABLE ${stats_db_name}.otherresearchproduct_tmp
(
id STRING,
@ -40,6 +42,8 @@ FROM ${openaire_db_name}.otherresearchproduct o
WHERE o.datainfo.deletedbyinference = FALSE and o.datainfo.invisible=false;
-- Otherresearchproduct_citations
DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_citations purge;
CREATE TABLE ${stats_db_name}.otherresearchproduct_citations STORED AS PARQUET AS
SELECT substr(o.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites
FROM ${openaire_db_name}.otherresearchproduct o LATERAL VIEW explode(o.extrainfo) citations AS citation
@ -51,6 +55,8 @@ SELECT substr(p.id, 4) AS id, instancetype.classname AS type
FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_concepts purge;
CREATE TABLE ${stats_db_name}.otherresearchproduct_concepts STORED AS PARQUET AS
SELECT substr(p.id, 4) as id, case
when contexts.context.id RLIKE '^[^::]+::[^::]+::.+$' then contexts.context.id
@ -59,6 +65,8 @@ SELECT substr(p.id, 4) as id, case
FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.context) contexts AS context
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_datasources purge;
CREATE TABLE ${stats_db_name}.otherresearchproduct_datasources STORED AS PARQUET AS
SELECT p.id, CASE WHEN d.id IS NULL THEN 'other' ELSE p.datasource END AS datasource
FROM (SELECT substr(p.id, 4) AS id, substr(instances.instance.hostedby.key, 4) AS datasource
@ -68,21 +76,29 @@ FROM (SELECT substr(p.id, 4) AS id, substr(instances.instance.hostedby.key, 4) A
from ${openaire_db_name}.datasource d
WHERE d.datainfo.deletedbyinference = false and d.datainfo.invisible=false) d on p.datasource = d.id;
DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_languages purge;
CREATE TABLE ${stats_db_name}.otherresearchproduct_languages STORED AS PARQUET AS
SELECT substr(p.id, 4) AS id, p.language.classname AS language
FROM ${openaire_db_name}.otherresearchproduct p
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_oids purge;
CREATE TABLE ${stats_db_name}.otherresearchproduct_oids STORED AS PARQUET AS
SELECT substr(p.id, 4) AS id, oids.ids AS oid
FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.originalid) oids AS ids
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_pids purge;
CREATE TABLE ${stats_db_name}.otherresearchproduct_pids STORED AS PARQUET AS
SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid
FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.pid) pids AS ppid
where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_topics purge;
CREATE TABLE ${stats_db_name}.otherresearchproduct_topics STORED AS PARQUET AS
SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic
FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.subject) subjects AS subject

View File

@ -3,29 +3,39 @@
-- Project table/view and Project related tables/views
------------------------------------------------------
------------------------------------------------------
DROP TABLE IF EXISTS ${stats_db_name}.project_oids purge;
CREATE TABLE ${stats_db_name}.project_oids STORED AS PARQUET AS
SELECT substr(p.id, 4) AS id, oids.ids AS oid
FROM ${openaire_db_name}.project p LATERAL VIEW explode(p.originalid) oids AS ids
where p.datainfo.deletedbyinference=false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.project_organizations purge;
CREATE TABLE ${stats_db_name}.project_organizations STORED AS PARQUET AS
SELECT substr(r.source, 4) AS id, substr(r.target, 4) AS organization
from ${openaire_db_name}.relation r
WHERE r.reltype = 'projectOrganization' and r.source like '40|%'
and r.datainfo.deletedbyinference = false and r.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.project_results purge;
CREATE TABLE ${stats_db_name}.project_results STORED AS PARQUET AS
SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS result, r.datainfo.provenanceaction.classname as provenance
FROM ${openaire_db_name}.relation r
WHERE r.reltype = 'resultProject' and r.target like '40|%'
and r.datainfo.deletedbyinference = false and r.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.project_classification purge;
create table ${stats_db_name}.project_classification STORED AS PARQUET as
select substr(p.id, 4) as id, class.h2020programme.code, class.level1, class.level2, class.level3
from ${openaire_db_name}.project p
lateral view explode(p.h2020classification) classifs as class
where p.datainfo.deletedbyinference=false and p.datainfo.invisible=false and class.h2020programme is not null;
DROP TABLE IF EXISTS ${stats_db_name}.project_tmp purge;
CREATE TABLE ${stats_db_name}.project_tmp
(
id STRING,
@ -80,12 +90,16 @@ SELECT substr(p.id, 4) AS id,
FROM ${openaire_db_name}.project p
WHERE p.datainfo.deletedbyinference = false and p.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.funder purge;
create table ${stats_db_name}.funder STORED AS PARQUET as
select distinct xpath_string(fund, '//funder/id') as id,
xpath_string(fund, '//funder/name') as name,
xpath_string(fund, '//funder/shortname') as shortname
from ${openaire_db_name}.project p lateral view explode(p.fundingtree.value) fundingtree as fund;
DROP TABLE IF EXISTS ${stats_db_name}.project_organization_contribution purge;
CREATE TABLE ${stats_db_name}.project_organization_contribution STORED AS PARQUET AS
SELECT distinct substr(r.source, 4) AS project, substr(r.target, 4) AS organization,
properties[0].value contribution, properties[1].value currency

View File

@ -123,6 +123,8 @@ UNION ALL
SELECT *
FROM ${stats_db_name}.otherresearchproduct_topics;
DROP TABLE IF EXISTS ${stats_db_name}.result_fos purge;
create table ${stats_db_name}.result_fos stored as parquet as
with
lvl1 as (select id, topic from ${stats_db_name}.result_topics where topic like '__ %' and type='Fields of Science and Technology classification'),
@ -133,6 +135,8 @@ from lvl1
join lvl2 on lvl1.id=lvl2.id and substr(lvl2.topic, 1, 2)=substr(lvl1.topic, 1, 2)
join lvl3 on lvl3.id=lvl1.id and substr(lvl3.topic, 1, 4)=substr(lvl2.topic, 1, 4);
DROP TABLE IF EXISTS ${stats_db_name}.result_organization purge;
CREATE TABLE ${stats_db_name}.result_organization STORED AS PARQUET AS
SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS organization
FROM ${openaire_db_name}.relation r
@ -140,6 +144,8 @@ WHERE r.reltype = 'resultOrganization'
and r.target like '50|%'
and r.datainfo.deletedbyinference = false and r.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.result_projects purge;
CREATE TABLE ${stats_db_name}.result_projects STORED AS PARQUET AS
select pr.result AS id, pr.id AS project, datediff(p.enddate, p.startdate) AS daysfromend, pr.provenance as provenance
FROM ${stats_db_name}.result r

View File

@ -5,6 +5,8 @@
-- Datasource table/view and Datasource related tables/views
------------------------------------------------------------
------------------------------------------------------------
DROP TABLE IF EXISTS ${stats_db_name}.datasource_tmp purge;
CREATE TABLE ${stats_db_name}.datasource_tmp
(
`id` string,
@ -48,6 +50,7 @@ WHERE d1.datainfo.deletedbyinference = FALSE and d1.datainfo.invisible=false;
-- Updating temporary table with everything that is not based on results -> This is done with the following "dual" table.
-- Creating a temporary dual table that will be removed after the following insert
CREATE TABLE ${stats_db_name}.dual ( dummy CHAR(1));
INSERT INTO ${stats_db_name}.dual VALUES ('X');
@ -74,16 +77,22 @@ DROP TABLE ${stats_db_name}.dual;
UPDATE ${stats_db_name}.datasource_tmp SET name='Other' WHERE name = 'Unknown Repository';
UPDATE ${stats_db_name}.datasource_tmp SET yearofvalidation=null WHERE yearofvalidation = '-1';
DROP TABLE IF EXISTS ${stats_db_name}.datasource_languages purge;
CREATE TABLE ${stats_db_name}.datasource_languages STORED AS PARQUET AS
SELECT substr(d.id, 4) AS id, langs.languages AS language
FROM ${openaire_db_name}.datasource d LATERAL VIEW explode(d.odlanguages.value) langs AS languages
where d.datainfo.deletedbyinference=false and d.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.datasource_oids purge;
CREATE TABLE ${stats_db_name}.datasource_oids STORED AS PARQUET AS
SELECT substr(d.id, 4) AS id, oids.ids AS oid
FROM ${openaire_db_name}.datasource d LATERAL VIEW explode(d.originalid) oids AS ids
where d.datainfo.deletedbyinference=false and d.datainfo.invisible=false;
DROP TABLE IF EXISTS ${stats_db_name}.datasource_organizations purge;
CREATE TABLE ${stats_db_name}.datasource_organizations STORED AS PARQUET AS
SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS organization
FROM ${openaire_db_name}.relation r
@ -91,6 +100,8 @@ WHERE r.reltype = 'datasourceOrganization' and r.datainfo.deletedbyinference = f
-- datasource sources:
-- where the datasource info have been collected from.
DROP TABLE IF EXISTS ${stats_db_name}.datasource_sources purge;
create table if not exists ${stats_db_name}.datasource_sources STORED AS PARQUET AS
select substr(d.id, 4) as id, substr(cf.key, 4) as datasource
from ${openaire_db_name}.datasource d lateral view explode(d.collectedfrom) cfrom as cf

View File

@ -3,6 +3,8 @@
-- Organization table/view and Organization related tables/views
----------------------------------------------------------------
----------------------------------------------------------------
DROP TABLE IF EXISTS ${stats_db_name}.organization purge;
CREATE TABLE IF NOT EXISTS ${stats_db_name}.organization STORED AS PARQUET AS
SELECT substr(o.id, 4) as id,
o.legalname.value as name,

View File

@ -114,6 +114,7 @@
<case to="step21-createObservatoryDB">${wf:conf('resumeFrom') eq 'step21-createObservatoryDB'}</case>
<case to="step21-createObservatoryDB-post">${wf:conf('resumeFrom') eq 'step21-createObservatoryDB-post'}</case>
<case to="step22-copyDataToImpalaCluster">${wf:conf('resumeFrom') eq 'step22-copyDataToImpalaCluster'}</case>
<case to="step22a-createPDFsAggregated">${wf:conf('resumeFrom') eq 'step22a-createPDFsAggregated'}</case>
<case to="step23-finalizeImpalaCluster">${wf:conf('resumeFrom') eq 'step23-finalizeImpalaCluster'}</case>
<case to="Step24-updateCache">${wf:conf('resumeFrom') eq 'Step24-updateCache'}</case>
<default to="Step1"/>
@ -448,6 +449,22 @@
<argument>${hadoop_user_name}</argument>
<file>copyDataToImpalaCluster.sh</file>
</shell>
<ok to="step22a-createPDFsAggregated"/>
<error to="Kill"/>
</action>
<action name="step22a-createPDFsAggregated">
<shell xmlns="uri:oozie:shell-action:0.1">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<exec>createPDFsAggregated.sh</exec>
<!-- <env-var>HADOOP_USER_NAME=${wf:user()}</env-var>-->
<!-- <argument>${external_stats_db_name}</argument>-->
<argument>${stats_db_name}</argument>
<argument>${monitor_db_name}</argument>
<argument>${hadoop_user_name}</argument>
<file>createPDFsAggregated.sh</file>
</shell>
<ok to="step23-finalizeImpalaCluster"/>
<error to="Kill"/>
</action>