forked from D-Net/dnet-hadoop
Merge remote-tracking branch 'origin/beta' into beta
This commit is contained in:
commit
477cb10715
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -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>
|
||||||
|
|
|
@ -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
|
||||||
|
}
|
||||||
|
]
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,2 @@
|
||||||
|
50|r37b0ad08687::f645b9729d1e1025a72c57883f0f2cac####50|r37b0ad08687::4c55b436743b5c49fa32cd582fd9e1aa
|
||||||
|
50|datacite____::a90f49f9fde5393c00633bea6e4e374a####50|datacite____::5f55cdee77303ba8a2bf9996c32a330c
|
|
@ -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}
|
||||||
|
|
|
@ -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"
|
|
@ -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"
|
|
@ -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 -
|
|
@ -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
|
||||||
|
|
|
@ -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);
|
|
|
@ -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;
|
|
@ -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;
|
||||||
|
|
||||||
|
|
|
@ -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;
|
|
@ -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"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
|
||||||
<action name="Step16_6">
|
<action name="Step19-finalize">
|
||||||
<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"/>
|
|
||||||
</action>
|
|
||||||
|
|
||||||
<action name="Step19">
|
|
||||||
<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"/>
|
||||||
|
|
Loading…
Reference in New Issue