forked from D-Net/dnet-hadoop
Merge branch 'beta' of https://code-repo.d4science.org/D-Net/dnet-hadoop into beta
This commit is contained in:
commit
c48c43fa9e
|
@ -0,0 +1,53 @@
|
|||
package eu.dnetlib.dhp.actionmanager.datacite
|
||||
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.oaf.{Oaf, Result}
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.LocalFileSystem
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.sql.{Encoder, Encoders, SparkSession}
|
||||
import org.apache.spark.sql.functions.max
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
import java.text.SimpleDateFormat
|
||||
import java.util.{Date, Locale}
|
||||
import scala.io.Source
|
||||
|
||||
object SparkDownloadUpdateDatacite {
|
||||
val log: Logger = LoggerFactory.getLogger(getClass)
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
|
||||
val conf = new SparkConf
|
||||
val parser = new ArgumentApplicationParser(Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/actionmanager/datacite/generate_dataset_params.json")).mkString)
|
||||
parser.parseArgument(args)
|
||||
val master = parser.get("master")
|
||||
val sourcePath = parser.get("sourcePath")
|
||||
val workingPath = parser.get("workingPath")
|
||||
|
||||
val hdfsuri = parser.get("namenode")
|
||||
log.info(s"namenode is $hdfsuri")
|
||||
|
||||
|
||||
val spark: SparkSession = SparkSession.builder().config(conf)
|
||||
.appName(getClass.getSimpleName)
|
||||
.master(master)
|
||||
.getOrCreate()
|
||||
|
||||
implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo[Oaf]
|
||||
implicit val resEncoder: Encoder[Result] = Encoders.kryo[Result]
|
||||
|
||||
import spark.implicits._
|
||||
|
||||
|
||||
val maxDate:String = spark.read.load(workingPath).as[Oaf].filter(s => s.isInstanceOf[Result]).map(r => r.asInstanceOf[Result].getDateofcollection).select(max("value")).first().getString(0)
|
||||
val ISO8601FORMAT = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ", Locale.US)
|
||||
val string_to_date =ISO8601FORMAT.parse(maxDate)
|
||||
val ts = string_to_date.getTime
|
||||
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,154 @@
|
|||
|
||||
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.JavaSparkContext;
|
||||
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.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;
|
||||
|
||||
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;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,117 @@
|
|||
|
||||
package eu.dnetlib.dhp.oa.dedup;
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.http.client.methods.CloseableHttpResponse;
|
||||
import org.apache.http.client.methods.HttpGet;
|
||||
import org.apache.http.impl.client.CloseableHttpClient;
|
||||
import org.apache.http.impl.client.HttpClients;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||
|
||||
public class UpdateOpenorgsJob {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(UpdateOpenorgsJob.class);
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(
|
||||
SparkCreateSimRels.class
|
||||
.getResourceAsStream("/eu/dnetlib/dhp/oa/dedup/updateOpenorgsJob_parameters.json")));
|
||||
parser.parseArgument(args);
|
||||
|
||||
final String apiUrl = parser.get("apiUrl");
|
||||
final int delay = Integer.parseInt(parser.get("delay"));
|
||||
|
||||
log.info("apiUrl: '{}'", apiUrl);
|
||||
log.info("delay: '{}'", delay);
|
||||
|
||||
APIResponse res = httpCall(apiUrl);
|
||||
while (res != null && res.getStatus().equals(ImportStatus.RUNNING)) {
|
||||
TimeUnit.MINUTES.sleep(delay);
|
||||
res = httpCall(apiUrl + "/status");
|
||||
}
|
||||
|
||||
if (res == null) {
|
||||
log.error("Openorgs Update FAILED: No response");
|
||||
throw new RuntimeException("Openorgs Update FAILED: No response");
|
||||
}
|
||||
|
||||
if (res.getStatus() == null || !res.getStatus().equals(ImportStatus.SUCCESS)) {
|
||||
log.error("Openorgs Update FAILED: '{}' - '{}'", res.getStatus(), res.getMessage());
|
||||
throw new RuntimeException(res.getMessage());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static APIResponse httpCall(final String url) throws Exception {
|
||||
final HttpGet req = new HttpGet(url);
|
||||
|
||||
try (final CloseableHttpClient client = HttpClients.createDefault()) {
|
||||
try (final CloseableHttpResponse response = client.execute(req)) {
|
||||
final String s = IOUtils.toString(response.getEntity().getContent());
|
||||
return (new ObjectMapper()).readValue(s, APIResponse.class);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class APIResponse {
|
||||
private String id;
|
||||
private Long dateStart;
|
||||
private Long dateEnd;
|
||||
private ImportStatus status;
|
||||
private String message;
|
||||
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public void setId(String id) {
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
public Long getDateStart() {
|
||||
return dateStart;
|
||||
}
|
||||
|
||||
public void setDateStart(Long dateStart) {
|
||||
this.dateStart = dateStart;
|
||||
}
|
||||
|
||||
public Long getDateEnd() {
|
||||
return dateEnd;
|
||||
}
|
||||
|
||||
public void setDateEnd(Long dateEnd) {
|
||||
this.dateEnd = dateEnd;
|
||||
}
|
||||
|
||||
public ImportStatus getStatus() {
|
||||
return status;
|
||||
}
|
||||
|
||||
public void setStatus(ImportStatus status) {
|
||||
this.status = status;
|
||||
}
|
||||
|
||||
public String getMessage() {
|
||||
return message;
|
||||
}
|
||||
|
||||
public void setMessage(String message) {
|
||||
this.message = message;
|
||||
}
|
||||
}
|
||||
|
||||
enum ImportStatus {
|
||||
SUCCESS, FAILED, RUNNING, NOT_LAUNCHED, NOT_YET_STARTED
|
||||
}
|
|
@ -28,6 +28,11 @@
|
|||
<name>dbPwd</name>
|
||||
<description>password to access the OpenOrgs database</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>dbConnections</name>
|
||||
<value>10</value>
|
||||
<description>number of connections to the postgres db</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>workingPath</name>
|
||||
<description>path for the working directory</description>
|
||||
|
@ -223,7 +228,7 @@
|
|||
<arg>--dbTable</arg><arg>${dbTable}</arg>
|
||||
<arg>--dbUser</arg><arg>${dbUser}</arg>
|
||||
<arg>--dbPwd</arg><arg>${dbPwd}</arg>
|
||||
<arg>--numConnections</arg><arg>20</arg>
|
||||
<arg>--numConnections</arg><arg>${dbConnections}</arg>
|
||||
</spark>
|
||||
<ok to="PrepareNewOrgs"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -254,19 +259,24 @@
|
|||
<arg>--dbTable</arg><arg>${dbTable}</arg>
|
||||
<arg>--dbUser</arg><arg>${dbUser}</arg>
|
||||
<arg>--dbPwd</arg><arg>${dbPwd}</arg>
|
||||
<arg>--numConnections</arg><arg>20</arg>
|
||||
<arg>--numConnections</arg><arg>${dbConnections}</arg>
|
||||
</spark>
|
||||
<ok to="update_openorgs"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="update_openorgs">
|
||||
<shell xmlns="uri:oozie:shell-action:0.1">
|
||||
<job-tracker>${jobTracker}</job-tracker>
|
||||
<name-node>${nameNode}</name-node>
|
||||
<exec>/usr/bin/curl</exec>
|
||||
<argument>${apiUrl}</argument>
|
||||
</shell>
|
||||
<java>
|
||||
<configuration>
|
||||
<property>
|
||||
<name>oozie.launcher.mapreduce.user.classpath.first</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
</configuration>
|
||||
<main-class>eu.dnetlib.dhp.oa.dedup.UpdateOpenorgsJob</main-class>
|
||||
<arg>--apiUrl</arg><arg>${apiUrl}</arg>
|
||||
<arg>--delay</arg><arg>5</arg>
|
||||
</java>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
|
|
@ -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,14 @@
|
|||
[
|
||||
{
|
||||
"paramName": "api",
|
||||
"paramLongName": "apiUrl",
|
||||
"paramDescription": "the url of the API",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "d",
|
||||
"paramLongName": "delay",
|
||||
"paramDescription": "delay for the HTTP call in minutes",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
|
@ -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));
|
||||
|
||||
|
@ -202,6 +215,84 @@ 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(
|
||||
|
@ -297,7 +388,7 @@ public class SparkDedupTest implements Serializable {
|
|||
}
|
||||
|
||||
@Test
|
||||
@Order(3)
|
||||
@Order(4)
|
||||
void createMergeRelsTest() throws Exception {
|
||||
|
||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
|
@ -353,7 +444,7 @@ public class SparkDedupTest implements Serializable {
|
|||
}
|
||||
|
||||
@Test
|
||||
@Order(4)
|
||||
@Order(5)
|
||||
void createDedupRecordTest() throws Exception {
|
||||
|
||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
|
@ -394,13 +485,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(
|
||||
|
@ -479,7 +570,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 +607,7 @@ public class SparkDedupTest implements Serializable {
|
|||
}
|
||||
|
||||
@Test
|
||||
@Order(6)
|
||||
@Order(7)
|
||||
void propagateRelationTest() throws Exception {
|
||||
|
||||
ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
|
@ -566,7 +657,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
|
|
@ -13,10 +13,30 @@ import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
|||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import org.json4s.DefaultFormats
|
||||
import org.json4s.JsonAST.{JField, JObject, JString,JArray}
|
||||
import org.json4s.jackson.JsonMethods.parse
|
||||
|
||||
object SparkGenerateDoiBoost {
|
||||
|
||||
|
||||
def extractIdGRID(input:String):List[(String,String)] = {
|
||||
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
|
||||
lazy val json: org.json4s.JValue = parse(input)
|
||||
|
||||
val id:String = (json \ "id").extract[String]
|
||||
|
||||
val grids:List[String] = for {
|
||||
|
||||
JObject(pid) <- json \ "pid"
|
||||
JField("qualifier", JObject(qualifier)) <- pid
|
||||
JField("classid", JString(classid)) <-qualifier
|
||||
JField("value", JString(vl)) <- pid
|
||||
if classid == "GRID"
|
||||
} yield vl
|
||||
grids.map(g => (id, s"unresolved::grid::${g.toLowerCase}"))(collection.breakOut)
|
||||
}
|
||||
|
||||
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
|
@ -36,6 +56,7 @@ object SparkGenerateDoiBoost {
|
|||
|
||||
val hostedByMapPath = parser.get("hostedByMapPath")
|
||||
val workingDirPath = parser.get("workingPath")
|
||||
val openaireOrganizationPath = parser.get("openaireOrganizationPath")
|
||||
|
||||
val crossrefAggregator = new Aggregator[(String, Publication), Publication, Publication] with Serializable {
|
||||
override def zero: Publication = new Publication
|
||||
|
@ -156,7 +177,7 @@ object SparkGenerateDoiBoost {
|
|||
magPubs.joinWith(a,magPubs("_1").equalTo(a("PaperId"))).flatMap(item => {
|
||||
val pub:Publication = item._1._2
|
||||
val affiliation = item._2
|
||||
val affId:String = if (affiliation.GridId.isDefined) DoiBoostMappingUtil.generateGridAffiliationId(affiliation.GridId.get) else DoiBoostMappingUtil.generateMAGAffiliationId(affiliation.AffiliationId.toString)
|
||||
val affId:String = if (affiliation.GridId.isDefined) s"unresolved::grid::${affiliation.GridId.get.toLowerCase}" else DoiBoostMappingUtil.generateMAGAffiliationId(affiliation.AffiliationId.toString)
|
||||
val r:Relation = new Relation
|
||||
r.setSource(pub.getId)
|
||||
r.setTarget(affId)
|
||||
|
@ -174,9 +195,35 @@ object SparkGenerateDoiBoost {
|
|||
r1.setDataInfo(pub.getDataInfo)
|
||||
r1.setCollectedfrom(List(DoiBoostMappingUtil.createMAGCollectedFrom()).asJava)
|
||||
List(r, r1)
|
||||
})(mapEncoderRel).write.mode(SaveMode.Overwrite).save(s"$workingDirPath/doiBoostPublicationAffiliation")
|
||||
})(mapEncoderRel).write.mode(SaveMode.Overwrite).save(s"$workingDirPath/doiBoostPublicationAffiliation_unresolved")
|
||||
|
||||
|
||||
|
||||
|
||||
val unresolvedRels:Dataset[(String, Relation)] = spark.read.load(s"$workingDirPath/doiBoostPublicationAffiliation_unresolved").as[Relation].map(r => {
|
||||
|
||||
if (r.getSource.startsWith("unresolved"))
|
||||
(r.getSource, r)
|
||||
else if (r.getTarget.startsWith("unresolved"))
|
||||
(r.getTarget,r)
|
||||
else
|
||||
("resolved", r)
|
||||
})
|
||||
|
||||
val openaireOrganization:Dataset[(String,String)] = spark.read.text(openaireOrganizationPath).as[String].flatMap(s => extractIdGRID(s)).groupByKey(_._2).reduceGroups((x,y) => if (x != null) x else y ).map(_._2)
|
||||
|
||||
unresolvedRels.joinWith(openaireOrganization,unresolvedRels("_1").equalTo(openaireOrganization("_2")))
|
||||
.map { x =>
|
||||
val currentRels = x._1._2
|
||||
val currentOrgs = x._2
|
||||
if (currentOrgs!= null)
|
||||
if(currentRels.getSource.startsWith("unresolved"))
|
||||
currentRels.setSource(currentOrgs._1)
|
||||
else
|
||||
currentRels.setTarget(currentOrgs._1)
|
||||
currentRels
|
||||
}.write.save(s"$workingDirPath/doiBoostPublicationAffiliation")
|
||||
|
||||
magPubs.joinWith(a,magPubs("_1").equalTo(a("PaperId"))).map( item => {
|
||||
val affiliation = item._2
|
||||
if (affiliation.GridId.isEmpty) {
|
||||
|
|
|
@ -1,7 +1,8 @@
|
|||
[
|
||||
{"paramName": "m", "paramLongName":"master", "paramDescription": "the master name", "paramRequired": true},
|
||||
{"paramName": "hb", "paramLongName":"hostedByMapPath", "paramDescription": "the hosted By Map Path", "paramRequired": true},
|
||||
{"paramName": "oo", "paramLongName":"openaireOrganizationPath", "paramDescription": "the openaire Organization Path", "paramRequired": true},
|
||||
{"paramName": "ap", "paramLongName":"affiliationPath", "paramDescription": "the Affliation Path", "paramRequired": true},
|
||||
{"paramName": "pa", "paramLongName":"paperAffiliationPath", "paramDescription": "the paperAffiliation Path", "paramRequired": true},
|
||||
{"paramName": "w", "paramLongName":"workingPath", "paramDescription": "the Working Path", "paramRequired": true}
|
||||
{"paramName": "w", "paramLongName":"workingPath", "paramDescription": "the Working Path", "paramRequired": true}
|
||||
]
|
||||
|
|
|
@ -27,6 +27,12 @@
|
|||
<name>hostedByMapPath</name>
|
||||
<description>the hostedByMap Path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>openaireOrganizationPath</name>
|
||||
<description>the OpenAire Organizations Path</description>
|
||||
</property>
|
||||
|
||||
|
||||
<property>
|
||||
<name>outputPath</name>
|
||||
<description>the Path of the sequence file action set</description>
|
||||
|
@ -214,6 +220,7 @@
|
|||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--hostedByMapPath</arg><arg>${hostedByMapPath}</arg>
|
||||
<arg>--openaireOrganizationPath</arg><arg>${openaireOrganizationPath}</arg>
|
||||
<arg>--affiliationPath</arg><arg>${inputPathMAG}/dataset/Affiliations</arg>
|
||||
<arg>--paperAffiliationPath</arg><arg>${inputPathMAG}/dataset/PaperAuthorAffiliations</arg>
|
||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||
|
|
|
@ -6,18 +6,130 @@ import eu.dnetlib.dhp.schema.oaf.Result
|
|||
import eu.dnetlib.dhp.sx.graph.bio.pubmed.{PMArticle, PMAuthor, PMJournal, PMParser, PubMedToOaf}
|
||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.{FSDataOutputStream, FileSystem, Path}
|
||||
import org.apache.http.client.config.RequestConfig
|
||||
import org.apache.http.client.methods.HttpGet
|
||||
import org.apache.http.impl.client.HttpClientBuilder
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.expressions.Aggregator
|
||||
import org.apache.spark.sql._
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
import java.io.InputStream
|
||||
import scala.io.Source
|
||||
import scala.xml.pull.XMLEventReader
|
||||
|
||||
object SparkCreateBaselineDataFrame {
|
||||
|
||||
|
||||
def requestBaseLineUpdatePage(maxFile:String):List[(String,String)] = {
|
||||
val data =requestPage("https://ftp.ncbi.nlm.nih.gov/pubmed/updatefiles/")
|
||||
|
||||
val result =data.lines.filter(l => l.startsWith("<a href=")).map{l =>
|
||||
val end = l.lastIndexOf("\">")
|
||||
val start = l.indexOf("<a href=\"")
|
||||
|
||||
if (start>= 0 && end >start)
|
||||
l.substring(start+9, (end-start))
|
||||
else
|
||||
""
|
||||
}.filter(s =>s.endsWith(".gz") ).filter(s => s > maxFile).map(s => (s,s"https://ftp.ncbi.nlm.nih.gov/pubmed/updatefiles/$s")).toList
|
||||
|
||||
result
|
||||
}
|
||||
|
||||
|
||||
def downloadBaselinePart(url:String):InputStream = {
|
||||
val r = new HttpGet(url)
|
||||
val timeout = 60; // seconds
|
||||
val config = RequestConfig.custom()
|
||||
.setConnectTimeout(timeout * 1000)
|
||||
.setConnectionRequestTimeout(timeout * 1000)
|
||||
.setSocketTimeout(timeout * 1000).build()
|
||||
val client = HttpClientBuilder.create().setDefaultRequestConfig(config).build()
|
||||
val response = client.execute(r)
|
||||
println(s"get response with status${response.getStatusLine.getStatusCode}")
|
||||
response.getEntity.getContent
|
||||
|
||||
}
|
||||
|
||||
def requestPage(url:String):String = {
|
||||
val r = new HttpGet(url)
|
||||
val timeout = 60; // seconds
|
||||
val config = RequestConfig.custom()
|
||||
.setConnectTimeout(timeout * 1000)
|
||||
.setConnectionRequestTimeout(timeout * 1000)
|
||||
.setSocketTimeout(timeout * 1000).build()
|
||||
val client = HttpClientBuilder.create().setDefaultRequestConfig(config).build()
|
||||
try {
|
||||
var tries = 4
|
||||
while (tries > 0) {
|
||||
println(s"requesting ${r.getURI}")
|
||||
try {
|
||||
val response = client.execute(r)
|
||||
println(s"get response with status${response.getStatusLine.getStatusCode}")
|
||||
if (response.getStatusLine.getStatusCode > 400) {
|
||||
tries -= 1
|
||||
}
|
||||
else
|
||||
return IOUtils.toString(response.getEntity.getContent)
|
||||
} catch {
|
||||
case e: Throwable =>
|
||||
println(s"Error on requesting ${r.getURI}")
|
||||
e.printStackTrace()
|
||||
tries -= 1
|
||||
}
|
||||
}
|
||||
""
|
||||
} finally {
|
||||
if (client != null)
|
||||
client.close()
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
def downloadBaseLineUpdate(baselinePath:String, hdfsServerUri:String ):Unit = {
|
||||
|
||||
|
||||
val conf = new Configuration
|
||||
conf.set("fs.defaultFS", hdfsServerUri)
|
||||
val fs = FileSystem.get(conf)
|
||||
val p = new Path(baselinePath)
|
||||
val files = fs.listFiles(p,false)
|
||||
var max_file = ""
|
||||
while (files.hasNext) {
|
||||
val c = files.next()
|
||||
val data = c.getPath.toString
|
||||
val fileName = data.substring(data.lastIndexOf("/")+1)
|
||||
|
||||
if (fileName> max_file)
|
||||
max_file = fileName
|
||||
}
|
||||
|
||||
val files_to_download = requestBaseLineUpdatePage(max_file)
|
||||
|
||||
files_to_download.foreach { u =>
|
||||
val hdfsWritePath: Path = new Path(s"$baselinePath/${u._1}")
|
||||
val fsDataOutputStream: FSDataOutputStream = fs.create(hdfsWritePath, true)
|
||||
val i = downloadBaselinePart(u._2)
|
||||
val buffer = Array.fill[Byte](1024)(0)
|
||||
while(i.read(buffer)>0) {
|
||||
fsDataOutputStream.write(buffer)
|
||||
}
|
||||
i.close()
|
||||
println(s"Downloaded ${u._2} into $baselinePath/${u._1}")
|
||||
fsDataOutputStream.close()
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
val pmArticleAggregator: Aggregator[(String, PMArticle), PMArticle, PMArticle] = new Aggregator[(String, PMArticle), PMArticle, PMArticle] with Serializable {
|
||||
override def zero: PMArticle = new PMArticle
|
||||
|
||||
|
@ -51,6 +163,10 @@ object SparkCreateBaselineDataFrame {
|
|||
val targetPath = parser.get("targetPath")
|
||||
log.info("targetPath: {}", targetPath)
|
||||
|
||||
val hdfsServerUri = parser.get("hdfsServerUri")
|
||||
log.info("hdfsServerUri: {}", targetPath)
|
||||
|
||||
|
||||
val isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl)
|
||||
val vocabularies = VocabularyGroup.loadVocsFromIS(isLookupService)
|
||||
val spark: SparkSession =
|
||||
|
@ -61,16 +177,15 @@ object SparkCreateBaselineDataFrame {
|
|||
.master(parser.get("master")).getOrCreate()
|
||||
import spark.implicits._
|
||||
|
||||
|
||||
val sc = spark.sparkContext
|
||||
|
||||
|
||||
|
||||
implicit val PMEncoder: Encoder[PMArticle] = Encoders.kryo(classOf[PMArticle])
|
||||
implicit val PMJEncoder: Encoder[PMJournal] = Encoders.kryo(classOf[PMJournal])
|
||||
implicit val PMAEncoder: Encoder[PMAuthor] = Encoders.kryo(classOf[PMAuthor])
|
||||
implicit val resultEncoder: Encoder[Result] = Encoders.kryo(classOf[Result])
|
||||
|
||||
downloadBaseLineUpdate(s"$workingPath/baseline", hdfsServerUri)
|
||||
|
||||
val k: RDD[(String, String)] = sc.wholeTextFiles(s"$workingPath/baseline",2000)
|
||||
val ds:Dataset[PMArticle] = spark.createDataset(k.filter(i => i._1.endsWith(".gz")).flatMap(i =>{
|
||||
val xml = new XMLEventReader(Source.fromBytes(i._2.getBytes()))
|
||||
|
@ -87,7 +202,5 @@ object SparkCreateBaselineDataFrame {
|
|||
.map(a => PubMedToOaf.convert(a, vocabularies)).as[Result]
|
||||
.filter(p => p!= null)
|
||||
.write.mode(SaveMode.Overwrite).save(targetPath)
|
||||
|
||||
//s"$workingPath/oaf/baseline_oaf"
|
||||
}
|
||||
}
|
||||
|
|
|
@ -5,17 +5,15 @@ import eu.dnetlib.dhp.sx.graph.bio.BioDBToOAF.EBILinkItem
|
|||
import eu.dnetlib.dhp.sx.graph.bio.pubmed.{PMArticle, PMAuthor, PMJournal}
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.http.client.config.RequestConfig
|
||||
import org.apache.http.client.methods.{HttpGet, HttpUriRequest}
|
||||
import org.apache.http.client.methods.HttpGet
|
||||
import org.apache.http.impl.client.HttpClientBuilder
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.sql.expressions.Aggregator
|
||||
import org.apache.spark.sql.functions.max
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||
import org.apache.spark.sql._
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
||||
object SparkDownloadEBILinks {
|
||||
|
||||
|
||||
def createEBILinks(pmid:Long):EBILinkItem = {
|
||||
|
||||
val res = requestLinks(pmid)
|
||||
|
@ -24,39 +22,42 @@ object SparkDownloadEBILinks {
|
|||
null
|
||||
}
|
||||
|
||||
def requestPage(url:String):String = {
|
||||
val r = new HttpGet(url)
|
||||
val timeout = 60; // seconds
|
||||
val config = RequestConfig.custom()
|
||||
.setConnectTimeout(timeout * 1000)
|
||||
.setConnectionRequestTimeout(timeout * 1000)
|
||||
.setSocketTimeout(timeout * 1000).build()
|
||||
val client = HttpClientBuilder.create().setDefaultRequestConfig(config).build()
|
||||
try {
|
||||
var tries = 4
|
||||
while (tries > 0) {
|
||||
println(s"requesting ${r.getURI}")
|
||||
try {
|
||||
val response = client.execute(r)
|
||||
println(s"get response with status${response.getStatusLine.getStatusCode}")
|
||||
if (response.getStatusLine.getStatusCode > 400) {
|
||||
tries -= 1
|
||||
}
|
||||
else
|
||||
return IOUtils.toString(response.getEntity.getContent)
|
||||
} catch {
|
||||
case e: Throwable =>
|
||||
println(s"Error on requesting ${r.getURI}")
|
||||
e.printStackTrace()
|
||||
tries -= 1
|
||||
}
|
||||
}
|
||||
""
|
||||
} finally {
|
||||
if (client != null)
|
||||
client.close()
|
||||
}
|
||||
}
|
||||
|
||||
def requestLinks(PMID:Long):String = {
|
||||
val r = new HttpGet(s"https://www.ebi.ac.uk/europepmc/webservices/rest/MED/$PMID/datalinks?format=json")
|
||||
val timeout = 60; // seconds
|
||||
val config = RequestConfig.custom()
|
||||
.setConnectTimeout(timeout * 1000)
|
||||
.setConnectionRequestTimeout(timeout * 1000)
|
||||
.setSocketTimeout(timeout * 1000).build()
|
||||
val client = HttpClientBuilder.create().setDefaultRequestConfig(config).build()
|
||||
try {
|
||||
var tries = 4
|
||||
while (tries > 0) {
|
||||
println(s"requesting ${r.getURI}")
|
||||
try {
|
||||
val response = client.execute(r)
|
||||
println(s"get response with status${response.getStatusLine.getStatusCode}")
|
||||
if (response.getStatusLine.getStatusCode > 400) {
|
||||
tries -= 1
|
||||
}
|
||||
else
|
||||
return IOUtils.toString(response.getEntity.getContent)
|
||||
} catch {
|
||||
case e: Throwable =>
|
||||
println(s"Error on requesting ${r.getURI}")
|
||||
e.printStackTrace()
|
||||
tries -= 1
|
||||
}
|
||||
}
|
||||
""
|
||||
} finally {
|
||||
if (client != null)
|
||||
client.close()
|
||||
}
|
||||
requestPage(s"https://www.ebi.ac.uk/europepmc/webservices/rest/MED/$PMID/datalinks?format=json")
|
||||
|
||||
}
|
||||
def main(args: Array[String]): Unit = {
|
||||
|
|
|
@ -32,14 +32,9 @@ object SparkEBILinksToOaf {
|
|||
import spark.implicits._
|
||||
implicit val PMEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf])
|
||||
|
||||
val ebi_rdd:Dataset[EBILinkItem] = spark.createDataset(spark.sparkContext.textFile(sourcePath).map(s => BioDBToOAF.extractEBILinksFromDump(s))).as[EBILinkItem]
|
||||
|
||||
ebi_rdd.write.mode(SaveMode.Overwrite).save(s"${sourcePath}_dataset")
|
||||
|
||||
val ebLinks:Dataset[EBILinkItem] = spark.read.load(s"${sourcePath}_dataset").as[EBILinkItem].filter(l => l.links!= null)
|
||||
|
||||
ebLinks.flatMap(j =>BioDBToOAF.parse_ebi_links(j.links))
|
||||
.repartition(4000)
|
||||
.filter(p => BioDBToOAF.EBITargetLinksFilter(p))
|
||||
.flatMap(p => BioDBToOAF.convertEBILinksToOaf(p))
|
||||
.write.mode(SaveMode.Overwrite).save(targetPath)
|
||||
|
|
|
@ -1,10 +1,10 @@
|
|||
DROP VIEW IF EXISTS ${hiveDbName}.result;
|
||||
|
||||
CREATE VIEW IF NOT EXISTS ${hiveDbName}.result as
|
||||
select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance from ${hiveDbName}.publication p
|
||||
select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance, measures from ${hiveDbName}.publication p
|
||||
union all
|
||||
select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance from ${hiveDbName}.dataset d
|
||||
select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance, measures from ${hiveDbName}.dataset d
|
||||
union all
|
||||
select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance from ${hiveDbName}.software s
|
||||
select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance, measures from ${hiveDbName}.software s
|
||||
union all
|
||||
select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance from ${hiveDbName}.otherresearchproduct o;
|
||||
select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance, measures from ${hiveDbName}.otherresearchproduct o;
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
[
|
||||
{"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true},
|
||||
{"paramName":"i", "paramLongName":"isLookupUrl","paramDescription": "isLookupUrl", "paramRequired": true},
|
||||
{"paramName":"w", "paramLongName":"workingPath","paramDescription": "the path of the sequencial file to read", "paramRequired": true},
|
||||
{"paramName":"t", "paramLongName":"targetPath","paramDescription": "the oaf path ", "paramRequired": true}
|
||||
{"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true},
|
||||
{"paramName":"i", "paramLongName":"isLookupUrl", "paramDescription": "isLookupUrl", "paramRequired": true},
|
||||
{"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true},
|
||||
{"paramName":"t", "paramLongName":"targetPath", "paramDescription": "the oaf path ", "paramRequired": true},
|
||||
{"paramName":"h", "paramLongName":"hdfsServerUri", "paramDescription": "the working path ", "paramRequired": true}
|
||||
]
|
|
@ -1,5 +1,5 @@
|
|||
[
|
||||
{"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true},
|
||||
{"paramName":"s", "paramLongName":"sourcePath","paramDescription": "the source Path", "paramRequired": true},
|
||||
{"paramName":"w", "paramLongName":"workingPath","paramDescription": "the working path ", "paramRequired": true}
|
||||
{"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true},
|
||||
{"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the source Path", "paramRequired": true},
|
||||
{"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the working path ", "paramRequired": true}
|
||||
]
|
|
@ -25,7 +25,6 @@
|
|||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
|
||||
<action name="GenerateBaselineDataset">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
|
@ -43,6 +42,7 @@
|
|||
</spark-opts>
|
||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--master</arg><arg>yarn</arg>
|
||||
<arg>--hdfsServerUri</arg><arg>${nameNode}</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
|
@ -74,7 +74,7 @@
|
|||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Create Baselnie DataSet</name>
|
||||
<name>Create Baseline DataSet</name>
|
||||
|
||||
<class>eu.dnetlib.dhp.sx.ebi.SparkAddLinkUpdates</class>
|
||||
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
|
||||
|
|
|
@ -1,59 +1,67 @@
|
|||
<workflow-app name="Create EBI Dataset" xmlns="uri:oozie:workflow:0.5">
|
||||
<parameters>
|
||||
<property>
|
||||
<name>sourcePath</name>
|
||||
<description>the Working Path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>workingPath</name>
|
||||
<description>the Working Path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkDriverMemory</name>
|
||||
<description>memory for driver process</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorMemory</name>
|
||||
<description>memory for individual executor</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorCores</name>
|
||||
<description>number of cores used by single executor</description>
|
||||
</property>
|
||||
</parameters>
|
||||
<workflow-app name="Create EBI Dataset" xmlns="uri:oozie:workflow:0.5">
|
||||
<parameters>
|
||||
<property>
|
||||
<name>sourcePath</name>
|
||||
<description>the Working Path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>workingPath</name>
|
||||
<description>the Working Path</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkDriverMemory</name>
|
||||
<description>memory for driver process</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorMemory</name>
|
||||
<description>memory for individual executor</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>sparkExecutorCores</name>
|
||||
<description>number of cores used by single executor</description>
|
||||
</property>
|
||||
</parameters>
|
||||
|
||||
<start to="DownloadEBILinks"/>
|
||||
<start to="DownloadEBILinks"/>
|
||||
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
|
||||
<action name="DownloadEBILinks">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Incremental Download EBI Links</name>
|
||||
<class>eu.dnetlib.dhp.sx.graph.ebi.SparkDownloadEBILinks</class>
|
||||
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.shuffle.partitions=2000
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
|
||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--master</arg><arg>yarn</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<end name="End"/>
|
||||
</workflow-app>
|
||||
<action name="DownloadEBILinks">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Incremental Download EBI Links</name>
|
||||
<class>eu.dnetlib.dhp.sx.graph.ebi.SparkDownloadEBILinks</class>
|
||||
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.shuffle.partitions=2000
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
</spark-opts>
|
||||
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
|
||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--master</arg><arg>yarn</arg>
|
||||
</spark>
|
||||
<ok to="OverrideFolders"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
<action name="OverrideFolders">
|
||||
<fs>
|
||||
<delete path="${sourcePath}/ebi_links_dataset_old"/>
|
||||
<move source="${sourcePath}/ebi_links_dataset" target="${sourcePath}/ebi_links_dataset_old"/>
|
||||
<move source="${workingPath}/links_final" target="${sourcePath}/ebi_links_dataset"/>
|
||||
</fs>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
<end name="End"/>
|
||||
</workflow-app>
|
|
@ -7,6 +7,7 @@ import eu.dnetlib.dhp.schema.oaf.{Oaf, Relation, Result}
|
|||
import eu.dnetlib.dhp.sx.graph.bio.BioDBToOAF.ScholixResolved
|
||||
import eu.dnetlib.dhp.sx.graph.bio.BioDBToOAF
|
||||
import eu.dnetlib.dhp.sx.graph.bio.pubmed.PubMedToOaf.dataInfo
|
||||
import eu.dnetlib.dhp.sx.graph.ebi.SparkDownloadEBILinks
|
||||
import org.json4s.DefaultFormats
|
||||
import org.json4s.JsonAST.{JField, JObject, JString}
|
||||
import org.json4s.jackson.JsonMethods.parse
|
||||
|
@ -50,6 +51,8 @@ class BioScholixTest extends AbstractVocabularyTest{
|
|||
}
|
||||
|
||||
|
||||
|
||||
|
||||
@Test
|
||||
def testEBIData() = {
|
||||
val inputXML = Source.fromInputStream(getClass.getResourceAsStream("pubmed.xml")).mkString
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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 | grep -iv roar | 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
|
||||
|
|
|
@ -233,4 +233,50 @@ on p.id= tmp.id;
|
|||
|
||||
create table indi_pub_has_abstract stored as parquet as
|
||||
select distinct publication.id, coalesce(abstract, 1) has_abstract
|
||||
from publication;
|
||||
from publication;
|
||||
|
||||
create table indi_with_orcid stored as parquet as
|
||||
select distinct r.id, coalesce(has_orcid, 0) as has_orcid
|
||||
from result r
|
||||
left outer join (select id, 1 as has_orcid from result_orcid) tmp
|
||||
on r.id= tmp.id
|
||||
|
||||
create table indi_funded_result_with_fundref stored as parquet as
|
||||
select distinct r.id, coalesce(fundref, 0) as fundref
|
||||
from project_results r
|
||||
left outer join (select distinct id, 1 as fundref from project_results
|
||||
where provenance='Harvested') tmp
|
||||
on r.id= tmp.id
|
||||
|
||||
create table indi_result_org_country_collab stored as parquet as
|
||||
with tmp as
|
||||
(select o.id as id, o.country , ro.id as result,r.type from organization o
|
||||
join result_organization ro on o.id=ro.organization
|
||||
join result r on r.id=ro.id where o.country <> 'UNKNOWN')
|
||||
select o1.id org1,o2.country country2, o1.type, count(distinct o1.result) as collaborations
|
||||
from tmp as o1
|
||||
join tmp as o2 on o1.result=o2.result
|
||||
where o1.id<>o2.id and o1.country<>o2.country
|
||||
group by o1.id, o1.type,o2.country
|
||||
|
||||
create table indi_result_org_collab stored as parquet as
|
||||
with tmp as
|
||||
(select o.id, ro.id as result,r.type from organization o
|
||||
join result_organization ro on o.id=ro.organization
|
||||
join result r on r.id=ro.id)
|
||||
select o1.id org1,o2.id org2, o1.type, count(distinct o1.result) as collaborations
|
||||
from tmp as o1
|
||||
join tmp as o2 on o1.result=o2.result
|
||||
where o1.id<>o2.id
|
||||
group by o1.id, o2.id, o1.type
|
||||
|
||||
create table indi_result_org_country_collab stored as parquet as
|
||||
with tmp as
|
||||
(select o.id as id, o.country , ro.id as result,r.type from organization o
|
||||
join result_organization ro on o.id=ro.organization
|
||||
join result r on r.id=ro.id where o.country <> 'UNKNOWN')
|
||||
select o1.id org1,o2.country country2, o1.type, count(distinct o1.result) as collaborations
|
||||
from tmp as o1
|
||||
join tmp as o2 on o1.result=o2.result
|
||||
where o1.id<>o2.id and o1.country<>o2.country
|
||||
group by o1.id, o1.type,o2.country
|
|
@ -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"/>
|
||||
<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"/>
|
||||
<ok to="Step19-finalize"/>
|
||||
<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