Merge remote-tracking branch 'origin/beta' into beta

This commit is contained in:
Sandro La Bruzzo 2021-09-27 16:57:23 +02:00
commit 477cb10715
17 changed files with 1544 additions and 919 deletions

View File

@ -0,0 +1,151 @@
package eu.dnetlib.dhp.oa.dedup;
import java.io.IOException;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.PairFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.dom4j.DocumentException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.xml.sax.SAXException;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.oa.dedup.model.Block;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import eu.dnetlib.pace.config.DedupConfig;
import eu.dnetlib.pace.model.MapDocument;
import eu.dnetlib.pace.util.MapDocumentUtil;
import scala.Tuple2;
import scala.Tuple3;
public class SparkWhitelistSimRels extends AbstractSparkAction {
private static final Logger log = LoggerFactory.getLogger(SparkCreateSimRels.class);
private static final String WHITELIST_SEPARATOR = "####";
public SparkWhitelistSimRels(ArgumentApplicationParser parser, SparkSession spark) {
super(parser, spark);
}
public static void main(String[] args) throws Exception {
ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
SparkCreateSimRels.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/dedup/whitelistSimRels_parameters.json")));
parser.parseArgument(args);
SparkConf conf = new SparkConf();
new SparkWhitelistSimRels(parser, getSparkSession(conf))
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
}
@Override
public void run(ISLookUpService isLookUpService)
throws DocumentException, IOException, ISLookUpException, SAXException {
// read oozie parameters
final String graphBasePath = parser.get("graphBasePath");
final String isLookUpUrl = parser.get("isLookUpUrl");
final String actionSetId = parser.get("actionSetId");
final String workingPath = parser.get("workingPath");
final int numPartitions = Optional
.ofNullable(parser.get("numPartitions"))
.map(Integer::valueOf)
.orElse(NUM_PARTITIONS);
final String whiteListPath = parser.get("whiteListPath");
log.info("numPartitions: '{}'", numPartitions);
log.info("graphBasePath: '{}'", graphBasePath);
log.info("isLookUpUrl: '{}'", isLookUpUrl);
log.info("actionSetId: '{}'", actionSetId);
log.info("workingPath: '{}'", workingPath);
log.info("whiteListPath: '{}'", whiteListPath);
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
//file format: source####target
Dataset<Tuple2<String, String>> whiteListRels = spark.createDataset(sc
.textFile(whiteListPath)
//check if the line is in the correct format: id1####id2
.filter(s -> s.contains(WHITELIST_SEPARATOR) && s.split(WHITELIST_SEPARATOR).length == 2)
.map(s -> new Tuple2<>(s.split(WHITELIST_SEPARATOR)[0], s.split(WHITELIST_SEPARATOR)[1]))
.rdd(),
Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
// for each dedup configuration
for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) {
final String entity = dedupConf.getWf().getEntityType();
final String subEntity = dedupConf.getWf().getSubEntityValue();
log.info("Adding whitelist simrels for: '{}'", subEntity);
final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity);
Dataset<Tuple2<String, String>> entities = spark.createDataset(sc
.textFile(DedupUtility.createEntityPath(graphBasePath, subEntity))
.repartition(numPartitions)
.mapToPair(
(PairFunction<String, String, String>) s -> {
MapDocument d = MapDocumentUtil.asMapDocumentWithJPath(dedupConf, s);
return new Tuple2<>(d.getIdentifier(), "present");
})
.rdd(),
Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
Dataset<Tuple2<String, String>> whiteListRels1 = whiteListRels
.joinWith(entities, whiteListRels.col("_1").equalTo(entities.col("_1")), "inner")
.map((MapFunction<Tuple2<Tuple2<String, String>, Tuple2<String, String>>, Tuple2<String, String>>) Tuple2::_1, Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
Dataset<Tuple2<String, String>> whiteListRels2 = whiteListRels1
.joinWith(entities, whiteListRels1.col("_2").equalTo(entities.col("_1")), "inner")
.map((MapFunction<Tuple2<Tuple2<String, String>, Tuple2<String, String>>, Tuple2<String, String>>) Tuple2::_1, Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
Dataset<Relation> whiteListSimRels = whiteListRels2
.map((MapFunction<Tuple2<String, String>, Relation>)
r -> createSimRel(r._1(), r._2(), entity),
Encoders.bean(Relation.class)
);
saveParquet(whiteListSimRels, outputPath, SaveMode.Append);
}
}
private Relation createSimRel(String source, String target, String entity) {
final Relation r = new Relation();
r.setSource(source);
r.setTarget(target);
r.setSubRelType("dedupSimilarity");
r.setRelClass("isSimilarTo");
r.setDataInfo(new DataInfo());
switch (entity) {
case "result":
r.setRelType("resultResult");
break;
case "organization":
r.setRelType("organizationOrganization");
break;
default:
throw new IllegalArgumentException("unmanaged entity type: " + entity);
}
return r;
}
}

View File

@ -20,6 +20,10 @@
<name>workingPath</name> <name>workingPath</name>
<description>path for the working directory</description> <description>path for the working directory</description>
</property> </property>
<property>
<name>whiteListPath</name>
<description>path for the whitelist of similarity relations</description>
</property>
<property> <property>
<name>dedupGraphPath</name> <name>dedupGraphPath</name>
<description>path for the output graph</description> <description>path for the output graph</description>
@ -130,6 +134,34 @@
<arg>--workingPath</arg><arg>${workingPath}</arg> <arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--numPartitions</arg><arg>8000</arg> <arg>--numPartitions</arg><arg>8000</arg>
</spark> </spark>
<ok to="WhitelistSimRels"/>
<error to="Kill"/>
</action>
<action name="WhitelistSimRels">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Add Whitelist Similarity Relations</name>
<class>eu.dnetlib.dhp.oa.dedup.SparkWhitelistSimRels</class>
<jar>dhp-dedup-openaire-${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.shuffle.partitions=3840
</spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--whiteListPath</arg><arg>${whiteListPath}</arg>
<arg>--numPartitions</arg><arg>8000</arg>
</spark>
<ok to="CreateMergeRel"/> <ok to="CreateMergeRel"/>
<error to="Kill"/> <error to="Kill"/>
</action> </action>

View File

@ -0,0 +1,38 @@
[
{
"paramName": "la",
"paramLongName": "isLookUpUrl",
"paramDescription": "address for the LookUp",
"paramRequired": true
},
{
"paramName": "asi",
"paramLongName": "actionSetId",
"paramDescription": "action set identifier (name of the orchestrator)",
"paramRequired": true
},
{
"paramName": "i",
"paramLongName": "graphBasePath",
"paramDescription": "the base path of the raw graph",
"paramRequired": true
},
{
"paramName": "w",
"paramLongName": "workingPath",
"paramDescription": "path of the working directory",
"paramRequired": true
},
{
"paramName": "np",
"paramLongName": "numPartitions",
"paramDescription": "number of partitions for the similarity relations intermediate phases",
"paramRequired": false
},
{
"paramName": "wl",
"paramLongName": "whiteListPath",
"paramDescription": "whitelist file path for the addition of custom simrels",
"paramRequired": true
}
]

View File

@ -0,0 +1,2 @@
50|r37b0ad08687::f645b9729d1e1025a72c57883f0f2cac####50|r37b0ad08687::4c55b436743b5c49fa32cd582fd9e1aa
50|datacite____::a90f49f9fde5393c00633bea6e4e374a####50|datacite____::5f55cdee77303ba8a2bf9996c32a330c

View File

@ -16,7 +16,7 @@ curl -L ${CONTEXT_API}/contexts/?type=ri,community -H "accept: application/json"
cat contexts.csv | cut -d , -f1 | xargs -I {} curl -L ${CONTEXT_API}/context/{}/?all=true | /usr/local/sbin/jq -r '.[]|"\(.id|split(":")[0]),\(.id),\(.label)"' > categories.csv cat contexts.csv | cut -d , -f1 | xargs -I {} curl -L ${CONTEXT_API}/context/{}/?all=true | /usr/local/sbin/jq -r '.[]|"\(.id|split(":")[0]),\(.id),\(.label)"' > categories.csv
cat categories.csv | cut -d , -f2 | sed 's/:/%3A/g'| xargs -I {} curl -L ${CONTEXT_API}/context/category/{}/?all=true | /usr/local/sbin/jq -r '.[]|"\(.id|split("::")[0])::\(.id|split("::")[1]),\(.id),\(.label)"' > concepts.csv cat categories.csv | cut -d , -f2 | sed 's/:/%3A/g'| xargs -I {} curl -L ${CONTEXT_API}/context/category/{}/?all=true | /usr/local/sbin/jq -r '.[]|"\(.id|split("::")[0])::\(.id|split("::")[1]),\(.id),\(.label)"' > concepts.csv
cat contexts.csv | sed 's/^\(.*\),\(.*\)/\1,\1::other,\2/' >> categories.csv cat contexts.csv | sed 's/^\(.*\),\(.*\)/\1,\1::other,\2/' >> categories.csv
cat categories.csv | grep -v ::other | sed 's/^.*,\(.*\),\(.*\)/\1,\1::other,\2/' >> concepts.csv cat categories.csv | sed 's/^.*,\(.*\),\(.*\)/\1,\1::other,\2/' >> concepts.csv
echo "uploading context data to hdfs" echo "uploading context data to hdfs"
hdfs dfs -mkdir ${TMP} hdfs dfs -mkdir ${TMP}

View File

@ -15,5 +15,5 @@ hdfs dfs -copyToLocal $SCRIPT_PATH
echo "Creating indicators" echo "Creating indicators"
impala-shell -q "invalidate metadata" impala-shell -q "invalidate metadata"
impala-shell -d ${TARGET} -q "show tables" --delimited | sed "s/^\(.*\)/compute stats ${TARGET}.\1;/" | impala-shell -c -f - impala-shell -d ${TARGET} -q "show tables" --delimited | sed "s/^\(.*\)/compute stats ${TARGET}.\1;/" | impala-shell -c -f -
cat step16_7-createIndicatorsTables.sql | impala-shell -d $TARGET -f - cat step16-createIndicatorsTables.sql | impala-shell -d $TARGET -f -
echo "Indicators created" echo "Indicators created"

View File

@ -9,16 +9,9 @@ fi
export SOURCE=$1 export SOURCE=$1
export TARGET=$2 export TARGET=$2
export SHADOW=$3 export SHADOW=$3
export SCRIPT_PATH=$4
echo "Getting file from " $4 impala-shell -q "invalidate metadata;"
hdfs dfs -copyToLocal $4 impala-shell -d ${TARGET} -q "show tables" --delimited | sed "s/\(.*\)/compute stats ${TARGET}.\1;/" | impala-shell -f -
echo "Creating observatory database"
impala-shell -q "drop database if exists ${TARGET} cascade"
impala-shell -q "create database if not exists ${TARGET}"
impala-shell -d ${SOURCE} -q "show tables" --delimited | sed "s/\(.*\)/create view ${TARGET}.\1 as select * from ${SOURCE}.\1;/" | impala-shell -f -
cat step21-createObservatoryDB.sql | sed s/SOURCE/$1/g | sed s/TARGET/$2/g1 | impala-shell -f -
echo "Impala shell finished" echo "Impala shell finished"
echo "Updating shadow observatory database" echo "Updating shadow observatory database"

View File

@ -0,0 +1,16 @@
export PYTHON_EGG_CACHE=/home/$(whoami)/.python-eggs
export link_folder=/tmp/impala-shell-python-egg-cache-$(whoami)
if ! [ -L $link_folder ]
then
rm -Rf "$link_folder"
ln -sfn ${PYTHON_EGG_CACHE}${link_folder} ${link_folder}
fi
export SOURCE=$1
export TARGET=$2
export SHADOW=$3
echo "Creating observatory database"
impala-shell -q "drop database if exists ${TARGET} cascade"
impala-shell -q "create database if not exists ${TARGET}"
impala-shell -d ${SOURCE} -q "show tables" --delimited | sed "s/\(.*\)/create view ${TARGET}.\1 as select * from ${SOURCE}.\1;/" | impala-shell -f -

View File

@ -23,6 +23,11 @@ CREATE OR REPLACE VIEW ${stats_db_name}.rndexpediture AS
SELECT * SELECT *
FROM ${external_stats_db_name}.rndexpediture; FROM ${external_stats_db_name}.rndexpediture;
CREATE OR REPLACE VIEW ${stats_db_name}.licenses_normalized AS
SELECT *
FROM ${external_stats_db_name}.licenses_normalized;
------------------------------------------------------------------------------------------------ ------------------------------------------------------------------------------------------------
------------------------------------------------------------------------------------------------ ------------------------------------------------------------------------------------------------
-- Creation date of the database -- Creation date of the database

View File

@ -1,62 +0,0 @@
----------------------------------------------------
-- Shortcuts for various definitions in stats db ---
----------------------------------------------------
-- Peer reviewed:
-- Results that have been collected from Crossref
create table ${stats_db_name}.result_peerreviewed as
with peer_reviewed as (
select distinct r.id as id
from ${stats_db_name}.result r
join ${stats_db_name}.result_sources rs on rs.id=r.id
join ${stats_db_name}.datasource d on d.id=rs.datasource
where d.name='Crossref')
select distinct peer_reviewed.id as id, true as peer_reviewed
from peer_reviewed
union all
select distinct r.id as id, false as peer_reviewed
from ${stats_db_name}.result r
left outer join peer_reviewed pr on pr.id=r.id
where pr.id is null;
-- Green OA:
-- OA results that are hosted by an Institutional repository and have NOT been harvested from a DOAJ journal.
create table ${stats_db_name}.result_greenoa as
with result_green as (
select distinct r.id as id
from ${stats_db_name}.result r
join ${stats_db_name}.result_datasources rd on rd.id=r.id
join ${stats_db_name}.datasource d on d.id=rd.datasource
left outer join (
select rd.id from ${stats_db_name}.result_datasources rd
join ${stats_db_name}.datasource d on rd.datasource=d.id
join ${stats_db_name}.datasource_sources sds on sds.id=d.id
join ${stats_db_name}.datasource sd on sd.id=sds.datasource
where sd.name='DOAJ-ARTICLES'
) as doaj on doaj.id=r.id
where r.bestlicence in ('Open Access', 'Open Source') and d.type='Institutional Repository' and doaj.id is null)
select distinct result_green.id, true as green
from result_green
union all
select distinct r.id as id, false as green
from ${stats_db_name}.result r
left outer join result_green rg on rg.id=r.id
where rg.id is null;
-- GOLD OA:
-- OA results that have been harvested from a DOAJ journal.
create table ${stats_db_name}.result_gold as
with result_gold as (
select distinct r.id as id
from ${stats_db_name}.result r
join ${stats_db_name}.result_datasources rd on rd.id=r.id
join ${stats_db_name}.datasource d on d.id=rd.datasource
join ${stats_db_name}.datasource_sources sds on sds.id=d.id
join ${stats_db_name}.datasource sd on sd.id=sds.datasource
where r.type='publication' and r.bestlicence='Open Access' and sd.name='DOAJ-Articles')
select distinct result_gold.id, true as gold
from result_gold
union all
select distinct r.id, false as gold
from ${stats_db_name}.result r
where r.id not in (select id from result_gold);

View File

@ -0,0 +1,22 @@
----------------------------------------------------
-- Shortcuts for various definitions in stats db ---
----------------------------------------------------
-- Peer reviewed:
create table ${stats_db_name}.result_peerreviewed 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
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:
create table ${stats_db_name}.result_greenoa 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:
create table ${stats_db_name}.result_gold as
select r.id, case when gold.gold_oa=1 then true else false end as gold
from ${stats_db_name}.result r
left outer join ${stats_db_name}.indi_pub_gold_oa gold on gold.id=r.id;

View File

@ -104,25 +104,42 @@ create table TARGET.project_results as select id as result, project as id from T
compute stats TARGET.project_results; compute stats TARGET.project_results;
-- indicators -- indicators
create table TARGET.indi_pub_green_oa as select * from SOURCE.indi_pub_green_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id); create view TARGET.indi_dataset_avg_year_content_oa as select * from SOURCE.indi_dataset_avg_year_content_oa orig;
compute stats TARGET.indi_pub_green_oa; create view TARGET.indi_dataset_avg_year_context_oa as select * from SOURCE.indi_dataset_avg_year_context_oa orig;
create table TARGET.indi_pub_grey_lit as select * from SOURCE.indi_pub_grey_lit orig where exists (select 1 from TARGET.result r where r.id=orig.id);
compute stats TARGET.indi_pub_grey_lit;
create table TARGET.indi_pub_doi_from_crossref as select * from SOURCE.indi_pub_doi_from_crossref orig where exists (select 1 from TARGET.result r where r.id=orig.id);
compute stats TARGET.indi_pub_doi_from_crossref;
create table TARGET.indi_pub_gold_oa as select * from SOURCE.indi_pub_gold_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id);
compute stats TARGET.indi_pub_gold_oa;
create view TARGET.indi_dataset_avg_year_country_oa as select * from SOURCE.indi_dataset_avg_year_country_oa orig; create view TARGET.indi_dataset_avg_year_country_oa as select * from SOURCE.indi_dataset_avg_year_country_oa orig;
create view TARGET.indi_other_avg_year_content_oa as select * from SOURCE.indi_other_avg_year_content_oa orig;
create view TARGET.indi_other_avg_year_context_oa as select * from SOURCE.indi_other_avg_year_context_oa orig;
create view TARGET.indi_other_avg_year_country_oa as select * from SOURCE.indi_other_avg_year_country_oa orig;
create view TARGET.indi_project_datasets_count as select * from SOURCE.indi_project_datasets_count orig; create view TARGET.indi_project_datasets_count as select * from SOURCE.indi_project_datasets_count orig;
create view TARGET.indi_project_otherresearch_count as select * from SOURCE.indi_project_otherresearch_count orig; create view TARGET.indi_project_otherresearch_count as select * from SOURCE.indi_project_otherresearch_count orig;
create view TARGET.indi_project_pubs_count as select * from SOURCE.indi_project_pubs_count orig; create view TARGET.indi_project_pubs_count as select * from SOURCE.indi_project_pubs_count orig;
create view TARGET.indi_project_software_count as select * from SOURCE.indi_project_software_count orig; create view TARGET.indi_project_software_count as select * from SOURCE.indi_project_software_count orig;
create view TARGET.indi_pub_avg_year_content_oa as select * from SOURCE.indi_pub_avg_year_content_oa orig;
create view TARGET.indi_pub_avg_year_context_oa as select * from SOURCE.indi_pub_avg_year_context_oa orig;
create view TARGET.indi_pub_avg_year_country_oa as select * from SOURCE.indi_pub_avg_year_country_oa orig; create view TARGET.indi_pub_avg_year_country_oa as select * from SOURCE.indi_pub_avg_year_country_oa orig;
create table TARGET.indi_pub_green_oa as select * from SOURCE.indi_pub_green_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id);
compute stats TARGET.indi_pub_green_oa;
create table TARGET.indi_pub_grey_lit as select * from SOURCE.indi_pub_grey_lit orig where exists (select 1 from TARGET.result r where r.id=orig.id);
compute stats TARGET.indi_pub_grey_lit;
create table TARGET.indi_pub_doi_from_crossref as select * from SOURCE.indi_pub_doi_from_crossref orig where exists (select 1 from TARGET.result r where r.id=orig.id);
compute stats TARGET.indi_pub_doi_from_crossref;
create table TARGET.indi_pub_gold_oa as select * from SOURCE.indi_pub_gold_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id);
compute stats TARGET.indi_pub_gold_oa;
create table TARGET.indi_pub_has_abstract as select * from SOURCE.indi_pub_has_abstract orig where exists (select 1 from TARGET.result r where r.id=orig.id);
compute stats TARGET.indi_pub_has_abstract;
create table TARGET.indi_pub_has_cc_licence as select * from SOURCE.indi_pub_has_cc_licence orig where exists (select 1 from TARGET.result r where r.id=orig.id);
compute stats TARGET.indi_pub_has_cc_licence;
create table TARGET.indi_pub_has_cc_licence_url as select * from SOURCE.indi_pub_has_cc_licence_url orig where exists (select 1 from TARGET.result r where r.id=orig.id);
compute stats TARGET.indi_pub_has_cc_licence_url;
create view TARGET.indi_software_avg_year_content_oa as select * from SOURCE.indi_software_avg_year_content_oa orig;
create view TARGET.indi_software_avg_year_context_oa as select * from SOURCE.indi_software_avg_year_context_oa orig;
create view TARGET.indi_software_avg_year_country_oa as select * from SOURCE.indi_software_avg_year_country_oa orig;
--denorm --denorm
alter table TARGET.result rename to TARGET.res_tmp; alter table TARGET.result rename to TARGET.res_tmp;

View File

@ -1,259 +1,561 @@
create table TARGET.result_affiliated_country stored as parquet as create table ${observatory_db_name}.result_cc_licence stored as parquet as
select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, select r.id, coalesce(rln.count, 0) > 0 as cc_licence
case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, from ${stats_db_name}.result r
r.peer_reviewed, r.type, c.code as ccode, c.name as cname left outer join (
from SOURCE.result r select rl.id, sum(case when lower(rln.normalized) like 'cc-%' then 1 else 0 end) as count
join SOURCE.result_organization ro on ro.id=r.id from ${stats_db_name}.result_licenses rl
join SOURCE.organization o on o.id=ro.organization left outer join ${stats_db_name}.licenses_normalized rln on rl.type=rln.license
join SOURCE.country c on c.code=o.country and c.continent_name='Europe' group by rl.id
left outer join SOURCE.result_licenses rl on rl.id=r.id ) rln on rln.id=r.id;
left outer join SOURCE.result_pids pids on pids.id=r.id
group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, c.code, c.name;
create table TARGET.result_affiliated_year stored as parquet as create table ${observatory_db_name}.result_affiliated_country stored as parquet as
select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, select
case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, r.peer_reviewed, r.type, r.year count(distinct r.id) as total,
from SOURCE.result r r.green,
join SOURCE.result_organization ro on ro.id=r.id r.gold,
join SOURCE.organization o on o.id=ro.organization case when rl.type is not null then true else false end as licence,
join SOURCE.country c on c.code=o.country and c.continent_name='Europe' case when pids.pid is not null then true else false end as pid,
left outer join SOURCE.result_licenses rl on rl.id=r.id case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa,
left outer join SOURCE.result_pids pids on pids.id=r.id r.peer_reviewed,
group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, r.year; rln.cc_licence,
r.abstract as abstract,
r.authors > 1 as multiple_authors,
rpc.count > 1 as multiple_projects,
rfc.count > 1 as multiple_funders,
r.type,
c.code as ccode, c.name as cname
from ${stats_db_name}.result r
join ${stats_db_name}.result_organization ro on ro.id=r.id
join ${stats_db_name}.organization o on o.id=ro.organization
join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe'
left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id
left outer join ${stats_db_name}.result_pids pids on pids.id=r.id
left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id
left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id
left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id
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, c.code, c.name;
create table TARGET.result_affiliated_year_country stored as parquet as create table ${observatory_db_name}.result_affiliated_year stored as parquet as
select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, select
case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, count(distinct r.id) as total,
r.peer_reviewed, r.type, r.year, c.code as ccode, c.name as cname r.green,
from SOURCE.result r r.gold,
join SOURCE.result_organization ro on ro.id=r.id case when rl.type is not null then true else false end as licence,
join SOURCE.organization o on o.id=ro.organization case when pids.pid is not null then true else false end as pid,
join SOURCE.country c on c.code=o.country and c.continent_name='Europe' case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa,
left outer join SOURCE.result_licenses rl on rl.id=r.id r.peer_reviewed,
left outer join SOURCE.result_pids pids on pids.id=r.id rln.cc_licence,
group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, r.year, c.code, c.name; r.abstract as abstract,
r.authors > 1 as multiple_authors,
rpc.count > 1 as multiple_projects,
rfc.count > 1 as multiple_funders,
r.type,
r.year
from ${stats_db_name}.result r
join ${stats_db_name}.result_organization ro on ro.id=r.id
join ${stats_db_name}.organization o on o.id=ro.organization
join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe'
left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id
left outer join ${stats_db_name}.result_pids pids on pids.id=r.id
left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id
left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id
left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id
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, r.year;
create table TARGET.result_affiliated_datasource stored as parquet as create table ${observatory_db_name}.result_affiliated_year_country stored as parquet as
select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, select
case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, r.peer_reviewed, r.type, d.name as dname count(distinct r.id) as total,
from SOURCE.result r r.green,
join SOURCE.result_organization ro on ro.id=r.id r.gold,
join SOURCE.organization o on o.id=ro.organization case when rl.type is not null then true else false end as licence,
join SOURCE.country c on c.code=o.country and c.continent_name='Europe' case when pids.pid is not null then true else false end as pid,
left outer join SOURCE.result_datasources rd on rd.id=r.id case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa,
left outer join SOURCE.datasource d on d.id=rd.datasource r.peer_reviewed,
left outer join SOURCE.result_licenses rl on rl.id=r.id rln.cc_licence,
left outer join SOURCE.result_pids pids on pids.id=r.id r.abstract as abstract,
group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, d.name; r.authors > 1 as multiple_authors,
rpc.count > 1 as multiple_projects,
rfc.count > 1 as multiple_funders,
r.type,
r.year, c.code as ccode, c.name as cname
from ${stats_db_name}.result r
join ${stats_db_name}.result_organization ro on ro.id=r.id
join ${stats_db_name}.organization o on o.id=ro.organization
join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe'
left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id
left outer join ${stats_db_name}.result_pids pids on pids.id=r.id
left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id
left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id
left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id
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, r.year, c.code, c.name;
create table TARGET.result_affiliated_datasource_country stored as parquet as create table ${observatory_db_name}.result_affiliated_datasource stored as parquet as
select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, select
case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, count(distinct r.id) as total,
r.peer_reviewed, r.type, d.name as dname, c.code as ccode, c.name as cname r.green,
from SOURCE.result r r.gold,
join SOURCE.result_organization ro on ro.id=r.id case when rl.type is not null then true else false end as licence,
join SOURCE.organization o on o.id=ro.organization case when pids.pid is not null then true else false end as pid,
join SOURCE.country c on c.code=o.country and c.continent_name='Europe' case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa,
left outer join SOURCE.result_datasources rd on rd.id=r.id r.peer_reviewed,
left outer join SOURCE.datasource d on d.id=rd.datasource rln.cc_licence,
left outer join SOURCE.result_licenses rl on rl.id=r.id r.abstract as abstract,
left outer join SOURCE.result_pids pids on pids.id=r.id r.authors > 1 as multiple_authors,
group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, d.name, c.code, c.name; rpc.count > 1 as multiple_projects,
rfc.count > 1 as multiple_funders,
r.type,
d.name as dname
from ${stats_db_name}.result r
join ${stats_db_name}.result_organization ro on ro.id=r.id
join ${stats_db_name}.organization o on o.id=ro.organization
join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe'
left outer join ${stats_db_name}.result_datasources rd on rd.id=r.id
left outer join ${stats_db_name}.datasource d on d.id=rd.datasource
left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id
left outer join ${stats_db_name}.result_pids pids on pids.id=r.id
left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id
left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id
left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id
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, d.name;
create table TARGET.result_affiliated_organization stored as parquet as create table ${observatory_db_name}.result_affiliated_datasource_country stored as parquet as
select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, select
case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, count(distinct r.id) as total,
r.peer_reviewed, r.type, o.name as oname r.green,
from SOURCE.result r r.gold,
join SOURCE.result_organization ro on ro.id=r.id case when rl.type is not null then true else false end as licence,
join SOURCE.organization o on o.id=ro.organization case when pids.pid is not null then true else false end as pid,
join SOURCE.country c on c.code=o.country and c.continent_name='Europe' case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa,
left outer join SOURCE.result_licenses rl on rl.id=r.id r.peer_reviewed,
left outer join SOURCE.result_pids pids on pids.id=r.id rln.cc_licence,
group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, o.name; r.abstract as abstract,
r.authors > 1 as multiple_authors,
rpc.count > 1 as multiple_projects,
rfc.count > 1 as multiple_funders,
r.type,
d.name as dname, c.code as ccode, c.name as cname
from ${stats_db_name}.result r
join ${stats_db_name}.result_organization ro on ro.id=r.id
join ${stats_db_name}.organization o on o.id=ro.organization
join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe'
left outer join ${stats_db_name}.result_datasources rd on rd.id=r.id
left outer join ${stats_db_name}.datasource d on d.id=rd.datasource
left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id
left outer join ${stats_db_name}.result_pids pids on pids.id=r.id
left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id
left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id
left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id
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, d.name, c.code, c.name;
create table TARGET.result_affiliated_organization_country stored as parquet as create table ${observatory_db_name}.result_affiliated_organization stored as parquet as
select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, select
case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, count(distinct r.id) as total,
r.peer_reviewed, r.type, o.name as oname, c.code as ccode, c.name as cname r.green,
from SOURCE.result r r.gold,
join SOURCE.result_organization ro on ro.id=r.id case when rl.type is not null then true else false end as licence,
join SOURCE.organization o on o.id=ro.organization case when pids.pid is not null then true else false end as pid,
join SOURCE.country c on c.code=o.country and c.continent_name='Europe' case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa,
left outer join SOURCE.result_licenses rl on rl.id=r.id r.peer_reviewed,
left outer join SOURCE.result_pids pids on pids.id=r.id rln.cc_licence,
group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, o.name, c.code, c.name; r.abstract as abstract,
r.authors > 1 as multiple_authors,
rpc.count > 1 as multiple_projects,
rfc.count > 1 as multiple_funders,
r.type,
o.name as oname
from ${stats_db_name}.result r
join ${stats_db_name}.result_organization ro on ro.id=r.id
join ${stats_db_name}.organization o on o.id=ro.organization
join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe'
left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id
left outer join ${stats_db_name}.result_pids pids on pids.id=r.id
left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id
left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id
left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id
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, o.name;
create table TARGET.result_affiliated_funder stored as parquet as create table ${observatory_db_name}.result_affiliated_organization_country stored as parquet as
select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, select
case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, r.peer_reviewed, r.type, p.funder as pfunder count(distinct r.id) as total,
from SOURCE.result r r.green,
join SOURCE.result_organization ro on ro.id=r.id r.gold,
join SOURCE.organization o on o.id=ro.organization case when rl.type is not null then true else false end as licence,
join SOURCE.country c on c.code=o.country and c.continent_name='Europe' case when pids.pid is not null then true else false end as pid,
join SOURCE.result_projects rp on rp.id=r.id case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa,
join SOURCE.project p on p.id=rp.project r.peer_reviewed,
left outer join SOURCE.result_licenses rl on rl.id=r.id rln.cc_licence,
left outer join SOURCE.result_pids pids on pids.id=r.id r.abstract as abstract,
group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, p.funder; r.authors > 1 as multiple_authors,
rpc.count > 1 as multiple_projects,
rfc.count > 1 as multiple_funders,
r.type,
o.name as oname, c.code as ccode, c.name as cname
from ${stats_db_name}.result r
join ${stats_db_name}.result_organization ro on ro.id=r.id
join ${stats_db_name}.organization o on o.id=ro.organization
join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe'
left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id
left outer join ${stats_db_name}.result_pids pids on pids.id=r.id
left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id
left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id
left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id
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, o.name, c.code, c.name;
create table TARGET.result_affiliated_funder_country stored as parquet as create table ${observatory_db_name}.result_affiliated_funder stored as parquet as
select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, select
case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, count(distinct r.id) as total,
r.peer_reviewed, r.type, p.funder as pfunder, c.code as ccode, c.name as cname r.green,
from SOURCE.result r r.gold,
join SOURCE.result_organization ro on ro.id=r.id case when rl.type is not null then true else false end as licence,
join SOURCE.organization o on o.id=ro.organization case when pids.pid is not null then true else false end as pid,
join SOURCE.country c on c.code=o.country and c.continent_name='Europe' case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa,
join SOURCE.result_projects rp on rp.id=r.id r.peer_reviewed,
join SOURCE.project p on p.id=rp.project rln.cc_licence,
left outer join SOURCE.result_licenses rl on rl.id=r.id r.abstract as abstract,
left outer join SOURCE.result_pids pids on pids.id=r.id r.authors > 1 as multiple_authors,
group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, p.funder, c.code, c.name; rpc.count > 1 as multiple_projects,
rfc.count > 1 as multiple_funders,
r.type,
p.funder as pfunder
from ${stats_db_name}.result r
join ${stats_db_name}.result_organization ro on ro.id=r.id
join ${stats_db_name}.organization o on o.id=ro.organization
join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe'
join ${stats_db_name}.result_projects rp on rp.id=r.id
join ${stats_db_name}.project p on p.id=rp.project
left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id
left outer join ${stats_db_name}.result_pids pids on pids.id=r.id
left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id
left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id
left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id
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;
create table TARGET.result_deposited_country stored as parquet as create table ${observatory_db_name}.result_affiliated_funder_country stored as parquet as
select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, select
case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, count(distinct r.id) as total,
r.peer_reviewed, r.type, c.code as ccode, c.name as cname r.green,
from SOURCE.result r r.gold,
join SOURCE.result_datasources rd on rd.id=r.id case when rl.type is not null then true else false end as licence,
join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') case when pids.pid is not null then true else false end as pid,
join SOURCE.datasource_organizations dor on dor.id=d.id case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa,
join SOURCE.organization o on o.id=dor.organization r.peer_reviewed,
join SOURCE.country c on c.code=o.country and c.continent_name='Europe' rln.cc_licence,
left outer join SOURCE.result_licenses rl on rl.id=r.id r.abstract as abstract,
left outer join SOURCE.result_pids pids on pids.id=r.id r.authors > 1 as multiple_authors,
group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, c.code, c.name; rpc.count > 1 as multiple_projects,
rfc.count > 1 as multiple_funders,
r.type,
p.funder as pfunder, c.code as ccode, c.name as cname
from ${stats_db_name}.result r
join ${stats_db_name}.result_organization ro on ro.id=r.id
join ${stats_db_name}.organization o on o.id=ro.organization
join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe'
join ${stats_db_name}.result_projects rp on rp.id=r.id
join ${stats_db_name}.project p on p.id=rp.project
left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id
left outer join ${stats_db_name}.result_pids pids on pids.id=r.id
left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id
left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id
left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id
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;
create table TARGET.result_deposited_year stored as parquet as create table ${observatory_db_name}.result_deposited_country stored as parquet as
select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, select
case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, r.peer_reviewed, r.type, r.year count(distinct r.id) as total,
from SOURCE.result r r.green,
join SOURCE.result_datasources rd on rd.id=r.id r.gold,
join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') case when rl.type is not null then true else false end as licence,
join SOURCE.datasource_organizations dor on dor.id=d.id case when pids.pid is not null then true else false end as pid,
join SOURCE.organization o on o.id=dor.organization case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa,
join SOURCE.country c on c.code=o.country and c.continent_name='Europe' r.peer_reviewed,
left outer join SOURCE.result_licenses rl on rl.id=r.id rln.cc_licence,
left outer join SOURCE.result_pids pids on pids.id=r.id r.abstract as abstract,
group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, r.year; r.authors > 1 as multiple_authors,
rpc.count > 1 as multiple_projects,
rfc.count > 1 as multiple_funders,
r.type,
c.code as ccode, c.name as cname
from ${stats_db_name}.result r
join ${stats_db_name}.result_datasources rd on rd.id=r.id
join ${stats_db_name}.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository')
join ${stats_db_name}.datasource_organizations dor on dor.id=d.id
join ${stats_db_name}.organization o on o.id=dor.organization
join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe'
left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id
left outer join ${stats_db_name}.result_pids pids on pids.id=r.id
left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id
left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id
left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id
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, c.code, c.name;
create table TARGET.result_deposited_year_country stored as parquet as create table ${observatory_db_name}.result_deposited_year stored as parquet as
select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, select
case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, count(distinct r.id) as total,
r.peer_reviewed, r.type, r.year, c.code as ccode, c.name as cname r.green,
from SOURCE.result r r.gold,
join SOURCE.result_datasources rd on rd.id=r.id case when rl.type is not null then true else false end as licence,
join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') case when pids.pid is not null then true else false end as pid,
join SOURCE.datasource_organizations dor on dor.id=d.id case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa,
join SOURCE.organization o on o.id=dor.organization r.peer_reviewed,
join SOURCE.country c on c.code=o.country and c.continent_name='Europe' rln.cc_licence,
left outer join SOURCE.result_licenses rl on rl.id=r.id r.abstract as abstract,
left outer join SOURCE.result_pids pids on pids.id=r.id r.authors > 1 as multiple_authors,
group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, r.year, c.code, c.name; rpc.count > 1 as multiple_projects,
rfc.count > 1 as multiple_funders,
r.type,
r.year
from ${stats_db_name}.result r
join ${stats_db_name}.result_datasources rd on rd.id=r.id
join ${stats_db_name}.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository')
join ${stats_db_name}.datasource_organizations dor on dor.id=d.id
join ${stats_db_name}.organization o on o.id=dor.organization
join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe'
left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id
left outer join ${stats_db_name}.result_pids pids on pids.id=r.id
left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id
left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id
left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id
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, r.year;
create table TARGET.result_deposited_datasource stored as parquet as create table ${observatory_db_name}.result_deposited_year_country stored as parquet as
select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, select
case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, count(distinct r.id) as total,
r.peer_reviewed, r.type, d.name as dname r.green,
from SOURCE.result r r.gold,
join SOURCE.result_datasources rd on rd.id=r.id case when rl.type is not null then true else false end as licence,
join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') case when pids.pid is not null then true else false end as pid,
join SOURCE.datasource_organizations dor on dor.id=d.id case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa,
join SOURCE.organization o on o.id=dor.organization r.peer_reviewed,
join SOURCE.country c on c.code=o.country and c.continent_name='Europe' rln.cc_licence,
left outer join SOURCE.result_licenses rl on rl.id=r.id r.abstract as abstract,
left outer join SOURCE.result_pids pids on pids.id=r.id r.authors > 1 as multiple_authors,
group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, d.name; rpc.count > 1 as multiple_projects,
rfc.count > 1 as multiple_funders,
r.type,
r.year, c.code as ccode, c.name as cname
from ${stats_db_name}.result r
join ${stats_db_name}.result_datasources rd on rd.id=r.id
join ${stats_db_name}.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository')
join ${stats_db_name}.datasource_organizations dor on dor.id=d.id
join ${stats_db_name}.organization o on o.id=dor.organization
join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe'
left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id
left outer join ${stats_db_name}.result_pids pids on pids.id=r.id
left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id
left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id
left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id
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, r.year, c.code, c.name;
create table TARGET.result_deposited_datasource_country stored as parquet as create table ${observatory_db_name}.result_deposited_datasource stored as parquet as
select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, select
case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, count(distinct r.id) as total,
r.peer_reviewed, r.type, d.name as dname, c.code as ccode, c.name as cname r.green,
from SOURCE.result r r.gold,
join SOURCE.result_datasources rd on rd.id=r.id case when rl.type is not null then true else false end as licence,
join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') case when pids.pid is not null then true else false end as pid,
join SOURCE.datasource_organizations dor on dor.id=d.id case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa,
join SOURCE.organization o on o.id=dor.organization r.peer_reviewed,
join SOURCE.country c on c.code=o.country and c.continent_name='Europe' rln.cc_licence,
left outer join SOURCE.result_licenses rl on rl.id=r.id r.abstract as abstract,
left outer join SOURCE.result_pids pids on pids.id=r.id r.authors > 1 as multiple_authors,
group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, d.name, c.code, c.name; rpc.count > 1 as multiple_projects,
rfc.count > 1 as multiple_funders,
r.type,
d.name as dname
from ${stats_db_name}.result r
join ${stats_db_name}.result_datasources rd on rd.id=r.id
join ${stats_db_name}.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository')
join ${stats_db_name}.datasource_organizations dor on dor.id=d.id
join ${stats_db_name}.organization o on o.id=dor.organization
join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe'
left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id
left outer join ${stats_db_name}.result_pids pids on pids.id=r.id
left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id
left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id
left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id
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, d.name;
create table TARGET.result_deposited_organization stored as parquet as create table ${observatory_db_name}.result_deposited_datasource_country stored as parquet as
select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, select
case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, r.peer_reviewed, r.type, o.name as oname count(distinct r.id) as total,
from SOURCE.result r r.green,
join SOURCE.result_datasources rd on rd.id=r.id r.gold,
join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') case when rl.type is not null then true else false end as licence,
join SOURCE.datasource_organizations dor on dor.id=d.id case when pids.pid is not null then true else false end as pid,
join SOURCE.organization o on o.id=dor.organization case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa,
join SOURCE.country c on c.code=o.country and c.continent_name='Europe' r.peer_reviewed,
left outer join SOURCE.result_licenses rl on rl.id=r.id rln.cc_licence,
left outer join SOURCE.result_pids pids on pids.id=r.id r.abstract as abstract,
group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, o.name; r.authors > 1 as multiple_authors,
rpc.count > 1 as multiple_projects,
rfc.count > 1 as multiple_funders,
r.type,
d.name as dname, c.code as ccode, c.name as cname
from ${stats_db_name}.result r
join ${stats_db_name}.result_datasources rd on rd.id=r.id
join ${stats_db_name}.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository')
join ${stats_db_name}.datasource_organizations dor on dor.id=d.id
join ${stats_db_name}.organization o on o.id=dor.organization
join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe'
left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id
left outer join ${stats_db_name}.result_pids pids on pids.id=r.id
left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id
left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id
left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id
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, d.name, c.code, c.name;
create table TARGET.result_deposited_organization_country stored as parquet as create table ${observatory_db_name}.result_deposited_organization stored as parquet as
select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, select
case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, count(distinct r.id) as total,
r.peer_reviewed, r.type, o.name as oname, c.code as ccode, c.name as cname r.green,
from SOURCE.result r r.gold,
join SOURCE.result_datasources rd on rd.id=r.id case when rl.type is not null then true else false end as licence,
join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') case when pids.pid is not null then true else false end as pid,
join SOURCE.datasource_organizations dor on dor.id=d.id case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa,
join SOURCE.organization o on o.id=dor.organization r.peer_reviewed,
join SOURCE.country c on c.code=o.country and c.continent_name='Europe' rln.cc_licence,
left outer join SOURCE.result_licenses rl on rl.id=r.id r.abstract as abstract,
left outer join SOURCE.result_pids pids on pids.id=r.id r.authors > 1 as multiple_authors,
group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, o.name, c.code, c.name; rpc.count > 1 as multiple_projects,
rfc.count > 1 as multiple_funders,
r.type,
o.name as oname
from ${stats_db_name}.result r
join ${stats_db_name}.result_datasources rd on rd.id=r.id
join ${stats_db_name}.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository')
join ${stats_db_name}.datasource_organizations dor on dor.id=d.id
join ${stats_db_name}.organization o on o.id=dor.organization
join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe'
left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id
left outer join ${stats_db_name}.result_pids pids on pids.id=r.id
left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id
left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id
left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id
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, o.name;
create table TARGET.result_deposited_funder stored as parquet as create table ${observatory_db_name}.result_deposited_organization_country stored as parquet as
select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, select
case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, count(distinct r.id) as total,
r.peer_reviewed, r.type, p.funder as pfunder r.green,
from SOURCE.result r r.gold,
join SOURCE.result_datasources rd on rd.id=r.id case when rl.type is not null then true else false end as licence,
join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') case when pids.pid is not null then true else false end as pid,
join SOURCE.datasource_organizations dor on dor.id=d.id case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa,
join SOURCE.organization o on o.id=dor.organization r.peer_reviewed,
join SOURCE.country c on c.code=o.country and c.continent_name='Europe' rln.cc_licence,
join SOURCE.result_projects rp on rp.id=r.id r.abstract as abstract,
join SOURCE.project p on p.id=rp.project r.authors > 1 as multiple_authors,
left outer join SOURCE.result_licenses rl on rl.id=r.id rpc.count > 1 as multiple_projects,
left outer join SOURCE.result_pids pids on pids.id=r.id rfc.count > 1 as multiple_funders,
group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, p.funder; r.type,
o.name as oname, c.code as ccode, c.name as cname
from ${stats_db_name}.result r
join ${stats_db_name}.result_datasources rd on rd.id=r.id
join ${stats_db_name}.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository')
join ${stats_db_name}.datasource_organizations dor on dor.id=d.id
join ${stats_db_name}.organization o on o.id=dor.organization
join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe'
left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id
left outer join ${stats_db_name}.result_pids pids on pids.id=r.id
left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id
left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id
left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id
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, o.name, c.code, c.name;
create table TARGET.result_deposited_funder_country stored as parquet as create table ${observatory_db_name}.result_deposited_funder stored as parquet as
select count(distinct r.id) as total, r.green, r.gold, case when rl.type is not null then true else false end as licence, select
case when pids.pid is not null then true else false end as pid, case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa, count(distinct r.id) as total,
r.peer_reviewed, r.type, p.funder as pfunder, c.code as ccode, c.name as cname r.green,
from SOURCE.result r r.gold,
join SOURCE.result_datasources rd on rd.id=r.id case when rl.type is not null then true else false end as licence,
join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository') case when pids.pid is not null then true else false end as pid,
join SOURCE.datasource_organizations dor on dor.id=d.id case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa,
join SOURCE.organization o on o.id=dor.organization r.peer_reviewed,
join SOURCE.country c on c.code=o.country and c.continent_name='Europe' rln.cc_licence,
join SOURCE.result_projects rp on rp.id=r.id r.abstract as abstract,
join SOURCE.project p on p.id=rp.project r.authors > 1 as multiple_authors,
left outer join SOURCE.result_licenses rl on rl.id=r.id rpc.count > 1 as multiple_projects,
left outer join SOURCE.result_pids pids on pids.id=r.id rfc.count > 1 as multiple_funders,
group by r.green, r.gold, licence, pid, oa, r.peer_reviewed, r.type, p.funder, c.code, c.name; r.type,
p.funder as pfunder
from ${stats_db_name}.result r
join ${stats_db_name}.result_datasources rd on rd.id=r.id
join ${stats_db_name}.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository')
join ${stats_db_name}.datasource_organizations dor on dor.id=d.id
join ${stats_db_name}.organization o on o.id=dor.organization
join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe'
join ${stats_db_name}.result_projects rp on rp.id=r.id
join ${stats_db_name}.project p on p.id=rp.project
left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id
left outer join ${stats_db_name}.result_pids pids on pids.id=r.id
left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id
left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id
left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id
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;
compute stats TARGET.result_affiliated_country; create table ${observatory_db_name}.result_deposited_funder_country stored as parquet as
compute stats TARGET.result_affiliated_year; select
compute stats TARGET.result_affiliated_year_country; count(distinct r.id) as total,
compute stats TARGET.result_affiliated_datasource; r.green,
compute stats TARGET.result_affiliated_datasource_country; r.gold,
compute stats TARGET.result_affiliated_organization; case when rl.type is not null then true else false end as licence,
compute stats TARGET.result_affiliated_organization_country; case when pids.pid is not null then true else false end as pid,
compute stats TARGET.result_affiliated_funder; case when r.access_mode in ('Open Access', 'Open Source') then true else false end as oa,
compute stats TARGET.result_affiliated_funder_country; r.peer_reviewed,
compute stats TARGET.result_deposited_country; rln.cc_licence,
compute stats TARGET.result_deposited_year; r.abstract as abstract,
compute stats TARGET.result_deposited_year_country; r.authors > 1 as multiple_authors,
compute stats TARGET.result_deposited_datasource; rpc.count > 1 as multiple_projects,
compute stats TARGET.result_deposited_datasource_country; rfc.count > 1 as multiple_funders,
compute stats TARGET.result_deposited_organization; r.type,
compute stats TARGET.result_deposited_organization_country; p.funder as pfunder, c.code as ccode, c.name as cname
compute stats TARGET.result_deposited_funder; from ${stats_db_name}.result r
compute stats TARGET.result_deposited_funder_country; join ${stats_db_name}.result_datasources rd on rd.id=r.id
join ${stats_db_name}.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository')
join ${stats_db_name}.datasource_organizations dor on dor.id=d.id
join ${stats_db_name}.organization o on o.id=dor.organization
join ${stats_db_name}.country c on c.code=o.country and c.continent_name='Europe'
join ${stats_db_name}.result_projects rp on rp.id=r.id
join ${stats_db_name}.project p on p.id=rp.project
left outer join ${stats_db_name}.result_licenses rl on rl.id=r.id
left outer join ${stats_db_name}.result_pids pids on pids.id=r.id
left outer join ${observatory_db_name}.result_cc_licence rln on rln.id=r.id
left outer join ${stats_db_name}.result_projectcount rpc on rpc.id=r.id
left outer join ${stats_db_name}.result_fundercount rfc on rfc.id=r.id
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;

View File

@ -239,14 +239,51 @@
<param>stats_db_name=${stats_db_name}</param> <param>stats_db_name=${stats_db_name}</param>
<param>openaire_db_name=${openaire_db_name}</param> <param>openaire_db_name=${openaire_db_name}</param>
</hive2> </hive2>
<ok to="Step16"/> <ok to="Step15_5"/>
<error to="Kill"/> <error to="Kill"/>
</action> </action>
<action name="Step16"> <action name="Step15_5">
<hive2 xmlns="uri:oozie:hive2-action:0.1"> <hive2 xmlns="uri:oozie:hive2-action:0.1">
<jdbc-url>${hive_jdbc_url}</jdbc-url> <jdbc-url>${hive_jdbc_url}</jdbc-url>
<script>scripts/step16.sql</script> <script>scripts/step15_5.sql</script>
<param>stats_db_name=${stats_db_name}</param>
<param>openaire_db_name=${openaire_db_name}</param>
</hive2>
<ok to="Contexts"/>
<error to="Kill"/>
</action>
<action name="Contexts">
<shell xmlns="uri:oozie:shell-action:0.1">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<exec>contexts.sh</exec>
<argument>${context_api_url}</argument>
<argument>${stats_db_name}</argument>
<file>contexts.sh</file>
</shell>
<ok to="Step16-createIndicatorsTables"/>
<error to="Kill"/>
</action>
<action name="Step16-createIndicatorsTables">
<shell xmlns="uri:oozie:shell-action:0.1">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<exec>indicators.sh</exec>
<argument>${stats_db_name}</argument>
<argument>${wf:appPath()}/scripts/step16-createIndicatorsTables.sql</argument>
<file>indicators.sh</file>
</shell>
<ok to="Step16_1-definitions"/>
<error to="Kill"/>
</action>
<action name="Step16_1-definitions">
<hive2 xmlns="uri:oozie:hive2-action:0.1">
<jdbc-url>${hive_jdbc_url}</jdbc-url>
<script>scripts/step16_1-definitions.sql</script>
<param>stats_db_name=${stats_db_name}</param> <param>stats_db_name=${stats_db_name}</param>
<param>openaire_db_name=${openaire_db_name}</param> <param>openaire_db_name=${openaire_db_name}</param>
</hive2> </hive2>
@ -261,48 +298,11 @@
<param>stats_db_name=${stats_db_name}</param> <param>stats_db_name=${stats_db_name}</param>
<param>openaire_db_name=${openaire_db_name}</param> <param>openaire_db_name=${openaire_db_name}</param>
</hive2> </hive2>
<ok to="Step16_6"/> <ok to="Step19-finalize"/>
<error to="Kill"/>
</action>
<action name="Step16_6">
<hive2 xmlns="uri:oozie:hive2-action:0.1">
<jdbc-url>${hive_jdbc_url}</jdbc-url>
<script>scripts/step16_6.sql</script>
<param>stats_db_name=${stats_db_name}</param>
<param>openaire_db_name=${openaire_db_name}</param>
</hive2>
<ok to="Step16_7-createIndicatorsTables"/>
<error to="Kill"/>
</action>
<action name="Step16_7-createIndicatorsTables">
<shell xmlns="uri:oozie:shell-action:0.1">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<exec>indicators.sh</exec>
<argument>${stats_db_name}</argument>
<argument>${wf:appPath()}/scripts/step16_7-createIndicatorsTables.sql</argument>
<file>indicators.sh</file>
</shell>
<ok to="Step17"/>
<error to="Kill"/>
</action>
<action name="Step17">
<shell xmlns="uri:oozie:shell-action:0.1">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<exec>contexts.sh</exec>
<argument>${context_api_url}</argument>
<argument>${stats_db_name}</argument>
<file>contexts.sh</file>
</shell>
<ok to="Step19"/>
<error to="Kill"/> <error to="Kill"/>
</action> </action>
<action name="Step19"> <action name="Step19-finalize">
<shell xmlns="uri:oozie:shell-action:0.1"> <shell xmlns="uri:oozie:shell-action:0.1">
<job-tracker>${jobTracker}</job-tracker> <job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node> <name-node>${nameNode}</name-node>
@ -326,20 +326,44 @@
<argument>${wf:appPath()}/scripts/step20-createMonitorDB.sql</argument> <argument>${wf:appPath()}/scripts/step20-createMonitorDB.sql</argument>
<file>monitor.sh</file> <file>monitor.sh</file>
</shell> </shell>
<ok to="step21-createObservatoryDB-pre"/>
<error to="Kill"/>
</action>
<action name="step21-createObservatoryDB-pre">
<shell xmlns="uri:oozie:shell-action:0.1">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<exec>observatory-pre.sh</exec>
<argument>${stats_db_name}</argument>
<argument>${observatory_db_name}</argument>
<argument>${observatory_db_shadow_name}</argument>
<file>observatory-pre.sh</file>
</shell>
<ok to="step21-createObservatoryDB"/> <ok to="step21-createObservatoryDB"/>
<error to="Kill"/> <error to="Kill"/>
</action> </action>
<action name="step21-createObservatoryDB"> <action name="step21-createObservatoryDB">
<hive2 xmlns="uri:oozie:hive2-action:0.1">
<jdbc-url>${hive_jdbc_url}</jdbc-url>
<script>scripts/step21-createObservatoryDB.sql</script>
<param>stats_db_name=${stats_db_name}</param>
<param>observatory_db_name=${observatory_db_name}</param>
</hive2>
<ok to="step21-createObservatoryDB-post"/>
<error to="Kill"/>
</action>
<action name="step21-createObservatoryDB-post">
<shell xmlns="uri:oozie:shell-action:0.1"> <shell xmlns="uri:oozie:shell-action:0.1">
<job-tracker>${jobTracker}</job-tracker> <job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node> <name-node>${nameNode}</name-node>
<exec>observatory.sh</exec> <exec>observatory-post.sh</exec>
<argument>${stats_db_name}</argument> <argument>${stats_db_name}</argument>
<argument>${observatory_db_name}</argument> <argument>${observatory_db_name}</argument>
<argument>${observatory_db_shadow_name}</argument> <argument>${observatory_db_shadow_name}</argument>
<argument>${wf:appPath()}/scripts/step21-createObservatoryDB.sql</argument> <file>observatory-post.sh</file>
<file>observatory.sh</file>
</shell> </shell>
<ok to="Step22"/> <ok to="Step22"/>
<error to="Kill"/> <error to="Kill"/>