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>
|
||||
<description>path for the working directory</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>whiteListPath</name>
|
||||
<description>path for the whitelist of similarity relations</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>dedupGraphPath</name>
|
||||
<description>path for the output graph</description>
|
||||
|
@ -130,6 +134,34 @@
|
|||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--numPartitions</arg><arg>8000</arg>
|
||||
</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"/>
|
||||
<error to="Kill"/>
|
||||
</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
|
||||
}
|
||||
]
|
|
@ -5,13 +5,16 @@ import static java.nio.file.Files.createTempDirectory;
|
|||
|
||||
import static org.apache.spark.sql.functions.count;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.Mockito.lenient;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.net.URISyntaxException;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
|
@ -55,6 +58,10 @@ public class SparkDedupTest implements Serializable {
|
|||
private static String testOutputBasePath;
|
||||
private static String testDedupGraphBasePath;
|
||||
private static final String testActionSetId = "test-orchestrator";
|
||||
private static String whitelistPath;
|
||||
private static List<String> whiteList;
|
||||
|
||||
private static String WHITELIST_SEPARATOR = "####";
|
||||
|
||||
@BeforeAll
|
||||
public static void cleanUp() throws IOException, URISyntaxException {
|
||||
|
@ -71,6 +78,12 @@ public class SparkDedupTest implements Serializable {
|
|||
.toAbsolutePath()
|
||||
.toString();
|
||||
|
||||
whitelistPath = Paths
|
||||
.get(SparkDedupTest.class.getResource("/eu/dnetlib/dhp/dedup/whitelist.simrels.txt").toURI())
|
||||
.toFile()
|
||||
.getAbsolutePath();
|
||||
whiteList = IOUtils.readLines(new FileReader(whitelistPath));
|
||||
|
||||
FileUtils.deleteDirectory(new File(testOutputBasePath));
|
||||
FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
|
||||
|
||||
|
@ -158,7 +171,7 @@ public class SparkDedupTest implements Serializable {
|
|||
|
||||
parser
|
||||
.parseArgument(
|
||||
new String[] {
|
||||
new String[]{
|
||||
"-i", testGraphBasePath,
|
||||
"-asi", testActionSetId,
|
||||
"-la", "lookupurl",
|
||||
|
@ -202,6 +215,78 @@ public class SparkDedupTest implements Serializable {
|
|||
|
||||
@Test
|
||||
@Order(2)
|
||||
void whitelistSimRelsTest() throws Exception {
|
||||
|
||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
SparkWhitelistSimRels.class
|
||||
.getResourceAsStream(
|
||||
"/eu/dnetlib/dhp/oa/dedup/whitelistSimRels_parameters.json")));
|
||||
|
||||
parser
|
||||
.parseArgument(
|
||||
new String[]{
|
||||
"-i", testGraphBasePath,
|
||||
"-asi", testActionSetId,
|
||||
"-la", "lookupurl",
|
||||
"-w", testOutputBasePath,
|
||||
"-np", "50",
|
||||
"-wl", whitelistPath
|
||||
});
|
||||
|
||||
new SparkWhitelistSimRels(parser, spark).run(isLookUpService);
|
||||
|
||||
long orgs_simrel = spark
|
||||
.read()
|
||||
.load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "organization"))
|
||||
.count();
|
||||
|
||||
long pubs_simrel = spark
|
||||
.read()
|
||||
.load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "publication"))
|
||||
.count();
|
||||
|
||||
long ds_simrel = spark
|
||||
.read()
|
||||
.load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "dataset"))
|
||||
.count();
|
||||
|
||||
long orp_simrel = spark
|
||||
.read()
|
||||
.load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "otherresearchproduct"))
|
||||
.count();
|
||||
|
||||
//entities simrels supposed to be equal to the number of previous step (no rels in whitelist)
|
||||
assertEquals(3082, orgs_simrel);
|
||||
assertEquals(7036, pubs_simrel);
|
||||
assertEquals(442, ds_simrel);
|
||||
assertEquals(6750, orp_simrel);
|
||||
|
||||
//entities simrels to be different from the number of previous step (new simrels in the whitelist)
|
||||
Dataset<Row> sw_simrel = spark
|
||||
.read()
|
||||
.load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "software"));
|
||||
|
||||
//check if the first relation in the whitelist exists
|
||||
assertTrue(sw_simrel
|
||||
.as(Encoders.bean(Relation.class))
|
||||
.toJavaRDD()
|
||||
.filter(rel ->
|
||||
rel.getSource().equalsIgnoreCase(whiteList.get(0).split(WHITELIST_SEPARATOR)[0]) && rel.getTarget().equalsIgnoreCase(whiteList.get(0).split(WHITELIST_SEPARATOR)[1])).count() > 0);
|
||||
//check if the second relation in the whitelist exists
|
||||
assertTrue(sw_simrel
|
||||
.as(Encoders.bean(Relation.class))
|
||||
.toJavaRDD()
|
||||
.filter(rel ->
|
||||
rel.getSource().equalsIgnoreCase(whiteList.get(1).split(WHITELIST_SEPARATOR)[0]) && rel.getTarget().equalsIgnoreCase(whiteList.get(1).split(WHITELIST_SEPARATOR)[1])).count() > 0);
|
||||
|
||||
assertEquals(338, sw_simrel.count());
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
@Order(3)
|
||||
void cutMergeRelsTest() throws Exception {
|
||||
|
||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
|
@ -213,7 +298,7 @@ public class SparkDedupTest implements Serializable {
|
|||
|
||||
parser
|
||||
.parseArgument(
|
||||
new String[] {
|
||||
new String[]{
|
||||
"-i",
|
||||
testGraphBasePath,
|
||||
"-asi",
|
||||
|
@ -297,7 +382,7 @@ public class SparkDedupTest implements Serializable {
|
|||
}
|
||||
|
||||
@Test
|
||||
@Order(3)
|
||||
@Order(4)
|
||||
void createMergeRelsTest() throws Exception {
|
||||
|
||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
|
@ -309,7 +394,7 @@ public class SparkDedupTest implements Serializable {
|
|||
|
||||
parser
|
||||
.parseArgument(
|
||||
new String[] {
|
||||
new String[]{
|
||||
"-i",
|
||||
testGraphBasePath,
|
||||
"-asi",
|
||||
|
@ -353,7 +438,7 @@ public class SparkDedupTest implements Serializable {
|
|||
}
|
||||
|
||||
@Test
|
||||
@Order(4)
|
||||
@Order(5)
|
||||
void createDedupRecordTest() throws Exception {
|
||||
|
||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
|
@ -364,7 +449,7 @@ public class SparkDedupTest implements Serializable {
|
|||
"/eu/dnetlib/dhp/oa/dedup/createDedupRecord_parameters.json")));
|
||||
parser
|
||||
.parseArgument(
|
||||
new String[] {
|
||||
new String[]{
|
||||
"-i",
|
||||
testGraphBasePath,
|
||||
"-asi",
|
||||
|
@ -394,13 +479,13 @@ public class SparkDedupTest implements Serializable {
|
|||
|
||||
assertEquals(85, orgs_deduprecord);
|
||||
assertEquals(65, pubs_deduprecord);
|
||||
assertEquals(51, sw_deduprecord);
|
||||
assertEquals(49, sw_deduprecord);
|
||||
assertEquals(97, ds_deduprecord);
|
||||
assertEquals(89, orp_deduprecord);
|
||||
}
|
||||
|
||||
@Test
|
||||
@Order(5)
|
||||
@Order(6)
|
||||
void updateEntityTest() throws Exception {
|
||||
|
||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
|
@ -411,7 +496,7 @@ public class SparkDedupTest implements Serializable {
|
|||
"/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json")));
|
||||
parser
|
||||
.parseArgument(
|
||||
new String[] {
|
||||
new String[]{
|
||||
"-i", testGraphBasePath, "-w", testOutputBasePath, "-o", testDedupGraphBasePath
|
||||
});
|
||||
|
||||
|
@ -479,7 +564,7 @@ public class SparkDedupTest implements Serializable {
|
|||
assertEquals(838, organizations);
|
||||
assertEquals(100, projects);
|
||||
assertEquals(100, datasource);
|
||||
assertEquals(200, softwares);
|
||||
assertEquals(198, softwares);
|
||||
assertEquals(389, dataset);
|
||||
assertEquals(517, otherresearchproduct);
|
||||
|
||||
|
@ -516,7 +601,7 @@ public class SparkDedupTest implements Serializable {
|
|||
}
|
||||
|
||||
@Test
|
||||
@Order(6)
|
||||
@Order(7)
|
||||
void propagateRelationTest() throws Exception {
|
||||
|
||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
|
@ -527,7 +612,7 @@ public class SparkDedupTest implements Serializable {
|
|||
"/eu/dnetlib/dhp/oa/dedup/propagateRelation_parameters.json")));
|
||||
parser
|
||||
.parseArgument(
|
||||
new String[] {
|
||||
new String[]{
|
||||
"-i", testGraphBasePath, "-w", testOutputBasePath, "-o", testDedupGraphBasePath
|
||||
});
|
||||
|
||||
|
@ -566,7 +651,7 @@ public class SparkDedupTest implements Serializable {
|
|||
}
|
||||
|
||||
@Test
|
||||
@Order(7)
|
||||
@Order(8)
|
||||
void testRelations() throws Exception {
|
||||
testUniqueness("/eu/dnetlib/dhp/dedup/test/relation_1.json", 12, 10);
|
||||
testUniqueness("/eu/dnetlib/dhp/dedup/test/relation_2.json", 10, 2);
|
||||
|
|
|
@ -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 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 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"
|
||||
hdfs dfs -mkdir ${TMP}
|
||||
|
|
|
@ -15,5 +15,5 @@ hdfs dfs -copyToLocal $SCRIPT_PATH
|
|||
echo "Creating indicators"
|
||||
impala-shell -q "invalidate metadata"
|
||||
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"
|
|
@ -9,16 +9,9 @@ fi
|
|||
export SOURCE=$1
|
||||
export TARGET=$2
|
||||
export SHADOW=$3
|
||||
export SCRIPT_PATH=$4
|
||||
|
||||
echo "Getting file from " $4
|
||||
hdfs dfs -copyToLocal $4
|
||||
|
||||
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 -
|
||||
impala-shell -q "invalidate metadata;"
|
||||
impala-shell -d ${TARGET} -q "show tables" --delimited | sed "s/\(.*\)/compute stats ${TARGET}.\1;/" | impala-shell -f -
|
||||
echo "Impala shell finished"
|
||||
|
||||
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 *
|
||||
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
|
||||
|
|
|
@ -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;
|
||||
|
||||
-- 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);
|
||||
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 view TARGET.indi_dataset_avg_year_content_oa as select * from SOURCE.indi_dataset_avg_year_content_oa orig;
|
||||
create view TARGET.indi_dataset_avg_year_context_oa as select * from SOURCE.indi_dataset_avg_year_context_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_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_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 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
|
||||
alter table TARGET.result rename to TARGET.res_tmp;
|
||||
|
||||
|
|
|
@ -1,259 +1,561 @@
|
|||
create table TARGET.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,
|
||||
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, c.code as ccode, c.name as cname
|
||||
from SOURCE.result r
|
||||
join SOURCE.result_organization ro on ro.id=r.id
|
||||
join SOURCE.organization o on o.id=ro.organization
|
||||
join SOURCE.country c on c.code=o.country and c.continent_name='Europe'
|
||||
left outer join SOURCE.result_licenses rl on rl.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 ${observatory_db_name}.result_cc_licence stored as parquet as
|
||||
select r.id, coalesce(rln.count, 0) > 0 as cc_licence
|
||||
from ${stats_db_name}.result r
|
||||
left outer join (
|
||||
select rl.id, sum(case when lower(rln.normalized) like 'cc-%' then 1 else 0 end) as count
|
||||
from ${stats_db_name}.result_licenses rl
|
||||
left outer join ${stats_db_name}.licenses_normalized rln on rl.type=rln.license
|
||||
group by rl.id
|
||||
) rln on rln.id=r.id;
|
||||
|
||||
create table TARGET.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,
|
||||
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
|
||||
from SOURCE.result r
|
||||
join SOURCE.result_organization ro on ro.id=r.id
|
||||
join SOURCE.organization o on o.id=ro.organization
|
||||
join SOURCE.country c on c.code=o.country and c.continent_name='Europe'
|
||||
left outer join SOURCE.result_licenses rl on rl.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, r.year;
|
||||
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,
|
||||
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,
|
||||
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
|
||||
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,
|
||||
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, c.code as ccode, c.name as cname
|
||||
from SOURCE.result r
|
||||
join SOURCE.result_organization ro on ro.id=r.id
|
||||
join SOURCE.organization o on o.id=ro.organization
|
||||
join SOURCE.country c on c.code=o.country and c.continent_name='Europe'
|
||||
left outer join SOURCE.result_licenses rl on rl.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, r.year, c.code, c.name;
|
||||
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,
|
||||
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,
|
||||
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,
|
||||
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
|
||||
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,
|
||||
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
|
||||
from SOURCE.result r
|
||||
join SOURCE.result_organization ro on ro.id=r.id
|
||||
join SOURCE.organization o on o.id=ro.organization
|
||||
join SOURCE.country c on c.code=o.country and c.continent_name='Europe'
|
||||
left outer join SOURCE.result_datasources rd on rd.id=r.id
|
||||
left outer join SOURCE.datasource d on d.id=rd.datasource
|
||||
left outer join SOURCE.result_licenses rl on rl.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, d.name;
|
||||
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,
|
||||
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,
|
||||
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,
|
||||
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
|
||||
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,
|
||||
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, c.code as ccode, c.name as cname
|
||||
from SOURCE.result r
|
||||
join SOURCE.result_organization ro on ro.id=r.id
|
||||
join SOURCE.organization o on o.id=ro.organization
|
||||
join SOURCE.country c on c.code=o.country and c.continent_name='Europe'
|
||||
left outer join SOURCE.result_datasources rd on rd.id=r.id
|
||||
left outer join SOURCE.datasource d on d.id=rd.datasource
|
||||
left outer join SOURCE.result_licenses rl on rl.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, d.name, c.code, c.name;
|
||||
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,
|
||||
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,
|
||||
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,
|
||||
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
|
||||
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,
|
||||
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
|
||||
from SOURCE.result r
|
||||
join SOURCE.result_organization ro on ro.id=r.id
|
||||
join SOURCE.organization o on o.id=ro.organization
|
||||
join SOURCE.country c on c.code=o.country and c.continent_name='Europe'
|
||||
left outer join SOURCE.result_licenses rl on rl.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, o.name;
|
||||
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,
|
||||
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,
|
||||
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,
|
||||
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
|
||||
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,
|
||||
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, c.code as ccode, c.name as cname
|
||||
from SOURCE.result r
|
||||
join SOURCE.result_organization ro on ro.id=r.id
|
||||
join SOURCE.organization o on o.id=ro.organization
|
||||
join SOURCE.country c on c.code=o.country and c.continent_name='Europe'
|
||||
left outer join SOURCE.result_licenses rl on rl.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, o.name, c.code, c.name;
|
||||
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,
|
||||
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,
|
||||
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,
|
||||
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
|
||||
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,
|
||||
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
|
||||
from SOURCE.result r
|
||||
join SOURCE.result_organization ro on ro.id=r.id
|
||||
join SOURCE.organization o on o.id=ro.organization
|
||||
join SOURCE.country c on c.code=o.country and c.continent_name='Europe'
|
||||
join SOURCE.result_projects rp on rp.id=r.id
|
||||
join SOURCE.project p on p.id=rp.project
|
||||
left outer join SOURCE.result_licenses rl on rl.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, p.funder;
|
||||
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,
|
||||
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,
|
||||
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,
|
||||
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
|
||||
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,
|
||||
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, c.code as ccode, c.name as cname
|
||||
from SOURCE.result r
|
||||
join SOURCE.result_organization ro on ro.id=r.id
|
||||
join SOURCE.organization o on o.id=ro.organization
|
||||
join SOURCE.country c on c.code=o.country and c.continent_name='Europe'
|
||||
join SOURCE.result_projects rp on rp.id=r.id
|
||||
join SOURCE.project p on p.id=rp.project
|
||||
left outer join SOURCE.result_licenses rl on rl.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, p.funder, c.code, c.name;
|
||||
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,
|
||||
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,
|
||||
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,
|
||||
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
|
||||
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,
|
||||
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, c.code as ccode, c.name as cname
|
||||
from SOURCE.result r
|
||||
join SOURCE.result_datasources rd on rd.id=r.id
|
||||
join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository')
|
||||
join SOURCE.datasource_organizations dor on dor.id=d.id
|
||||
join SOURCE.organization o on o.id=dor.organization
|
||||
join SOURCE.country c on c.code=o.country and c.continent_name='Europe'
|
||||
left outer join SOURCE.result_licenses rl on rl.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 ${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,
|
||||
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,
|
||||
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,
|
||||
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
|
||||
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,
|
||||
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
|
||||
from SOURCE.result r
|
||||
join SOURCE.result_datasources rd on rd.id=r.id
|
||||
join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository')
|
||||
join SOURCE.datasource_organizations dor on dor.id=d.id
|
||||
join SOURCE.organization o on o.id=dor.organization
|
||||
join SOURCE.country c on c.code=o.country and c.continent_name='Europe'
|
||||
left outer join SOURCE.result_licenses rl on rl.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, r.year;
|
||||
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,
|
||||
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,
|
||||
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_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
|
||||
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,
|
||||
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, c.code as ccode, c.name as cname
|
||||
from SOURCE.result r
|
||||
join SOURCE.result_datasources rd on rd.id=r.id
|
||||
join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository')
|
||||
join SOURCE.datasource_organizations dor on dor.id=d.id
|
||||
join SOURCE.organization o on o.id=dor.organization
|
||||
join SOURCE.country c on c.code=o.country and c.continent_name='Europe'
|
||||
left outer join SOURCE.result_licenses rl on rl.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, r.year, c.code, c.name;
|
||||
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,
|
||||
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,
|
||||
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,
|
||||
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
|
||||
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,
|
||||
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
|
||||
from SOURCE.result r
|
||||
join SOURCE.result_datasources rd on rd.id=r.id
|
||||
join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository')
|
||||
join SOURCE.datasource_organizations dor on dor.id=d.id
|
||||
join SOURCE.organization o on o.id=dor.organization
|
||||
join SOURCE.country c on c.code=o.country and c.continent_name='Europe'
|
||||
left outer join SOURCE.result_licenses rl on rl.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, d.name;
|
||||
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,
|
||||
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,
|
||||
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,
|
||||
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
|
||||
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,
|
||||
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, c.code as ccode, c.name as cname
|
||||
from SOURCE.result r
|
||||
join SOURCE.result_datasources rd on rd.id=r.id
|
||||
join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository')
|
||||
join SOURCE.datasource_organizations dor on dor.id=d.id
|
||||
join SOURCE.organization o on o.id=dor.organization
|
||||
join SOURCE.country c on c.code=o.country and c.continent_name='Europe'
|
||||
left outer join SOURCE.result_licenses rl on rl.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, d.name, c.code, c.name;
|
||||
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,
|
||||
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,
|
||||
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,
|
||||
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
|
||||
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,
|
||||
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
|
||||
from SOURCE.result r
|
||||
join SOURCE.result_datasources rd on rd.id=r.id
|
||||
join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository')
|
||||
join SOURCE.datasource_organizations dor on dor.id=d.id
|
||||
join SOURCE.organization o on o.id=dor.organization
|
||||
join SOURCE.country c on c.code=o.country and c.continent_name='Europe'
|
||||
left outer join SOURCE.result_licenses rl on rl.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, o.name;
|
||||
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,
|
||||
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,
|
||||
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,
|
||||
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
|
||||
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,
|
||||
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, c.code as ccode, c.name as cname
|
||||
from SOURCE.result r
|
||||
join SOURCE.result_datasources rd on rd.id=r.id
|
||||
join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository')
|
||||
join SOURCE.datasource_organizations dor on dor.id=d.id
|
||||
join SOURCE.organization o on o.id=dor.organization
|
||||
join SOURCE.country c on c.code=o.country and c.continent_name='Europe'
|
||||
left outer join SOURCE.result_licenses rl on rl.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, o.name, c.code, c.name;
|
||||
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,
|
||||
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,
|
||||
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,
|
||||
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
|
||||
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,
|
||||
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
|
||||
from SOURCE.result r
|
||||
join SOURCE.result_datasources rd on rd.id=r.id
|
||||
join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository')
|
||||
join SOURCE.datasource_organizations dor on dor.id=d.id
|
||||
join SOURCE.organization o on o.id=dor.organization
|
||||
join SOURCE.country c on c.code=o.country and c.continent_name='Europe'
|
||||
join SOURCE.result_projects rp on rp.id=r.id
|
||||
join SOURCE.project p on p.id=rp.project
|
||||
left outer join SOURCE.result_licenses rl on rl.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, p.funder;
|
||||
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,
|
||||
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,
|
||||
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,
|
||||
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
|
||||
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,
|
||||
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, c.code as ccode, c.name as cname
|
||||
from SOURCE.result r
|
||||
join SOURCE.result_datasources rd on rd.id=r.id
|
||||
join SOURCE.datasource d on d.id=rd.datasource and d.type in ('Institutional Repository','Data Repository', 'Repository', 'Publication Repository')
|
||||
join SOURCE.datasource_organizations dor on dor.id=d.id
|
||||
join SOURCE.organization o on o.id=dor.organization
|
||||
join SOURCE.country c on c.code=o.country and c.continent_name='Europe'
|
||||
join SOURCE.result_projects rp on rp.id=r.id
|
||||
join SOURCE.project p on p.id=rp.project
|
||||
left outer join SOURCE.result_licenses rl on rl.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, p.funder, c.code, c.name;
|
||||
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,
|
||||
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,
|
||||
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,
|
||||
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;
|
||||
compute stats TARGET.result_affiliated_year;
|
||||
compute stats TARGET.result_affiliated_year_country;
|
||||
compute stats TARGET.result_affiliated_datasource;
|
||||
compute stats TARGET.result_affiliated_datasource_country;
|
||||
compute stats TARGET.result_affiliated_organization;
|
||||
compute stats TARGET.result_affiliated_organization_country;
|
||||
compute stats TARGET.result_affiliated_funder;
|
||||
compute stats TARGET.result_affiliated_funder_country;
|
||||
compute stats TARGET.result_deposited_country;
|
||||
compute stats TARGET.result_deposited_year;
|
||||
compute stats TARGET.result_deposited_year_country;
|
||||
compute stats TARGET.result_deposited_datasource;
|
||||
compute stats TARGET.result_deposited_datasource_country;
|
||||
compute stats TARGET.result_deposited_organization;
|
||||
compute stats TARGET.result_deposited_organization_country;
|
||||
compute stats TARGET.result_deposited_funder;
|
||||
compute stats TARGET.result_deposited_funder_country;
|
||||
create table ${observatory_db_name}.result_deposited_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,
|
||||
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,
|
||||
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,
|
||||
p.funder as pfunder, 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'
|
||||
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>openaire_db_name=${openaire_db_name}</param>
|
||||
</hive2>
|
||||
<ok to="Step16"/>
|
||||
<ok to="Step15_5"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="Step16">
|
||||
<action name="Step15_5">
|
||||
<hive2 xmlns="uri:oozie:hive2-action:0.1">
|
||||
<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>openaire_db_name=${openaire_db_name}</param>
|
||||
</hive2>
|
||||
|
@ -261,48 +298,11 @@
|
|||
<param>stats_db_name=${stats_db_name}</param>
|
||||
<param>openaire_db_name=${openaire_db_name}</param>
|
||||
</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"/>
|
||||
</action>
|
||||
|
||||
<action name="Step19">
|
||||
<action name="Step19-finalize">
|
||||
<shell xmlns="uri:oozie:shell-action:0.1">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
|
@ -326,20 +326,44 @@
|
|||
<argument>${wf:appPath()}/scripts/step20-createMonitorDB.sql</argument>
|
||||
<file>monitor.sh</file>
|
||||
</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"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<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">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<exec>observatory.sh</exec>
|
||||
<exec>observatory-post.sh</exec>
|
||||
<argument>${stats_db_name}</argument>
|
||||
<argument>${observatory_db_name}</argument>
|
||||
<argument>${observatory_db_shadow_name}</argument>
|
||||
<argument>${wf:appPath()}/scripts/step21-createObservatoryDB.sql</argument>
|
||||
<file>observatory.sh</file>
|
||||
<file>observatory-post.sh</file>
|
||||
</shell>
|
||||
<ok to="Step22"/>
|
||||
<error to="Kill"/>
|
||||
|
|
Loading…
Reference in New Issue