forked from D-Net/dnet-hadoop
merging with branch beta - resolved conflicts
This commit is contained in:
commit
fec40bdd95
|
@ -1,14 +1,12 @@
|
||||||
package eu.dnetlib.dhp.sx.graph.bio
|
package eu.dnetllib.dhp.sx.bio
|
||||||
|
|
||||||
import eu.dnetlib.dhp.schema.common.ModelConstants
|
import eu.dnetlib.dhp.schema.common.ModelConstants
|
||||||
import eu.dnetlib.dhp.schema.oaf.utils.{GraphCleaningFunctions, OafMapperUtils}
|
import eu.dnetlib.dhp.schema.oaf.utils.{GraphCleaningFunctions, OafMapperUtils}
|
||||||
import eu.dnetlib.dhp.schema.oaf.{Author, DataInfo, Dataset, Instance, KeyValue, Oaf, Relation, StructuredProperty}
|
import eu.dnetlib.dhp.schema.oaf._
|
||||||
import org.json4s.DefaultFormats
|
import org.json4s.DefaultFormats
|
||||||
import org.json4s.JsonAST.{JField, JObject, JString}
|
import org.json4s.JsonAST.{JField, JObject, JString}
|
||||||
import org.json4s.jackson.JsonMethods.{compact, parse, render}
|
import org.json4s.jackson.JsonMethods.{compact, parse, render}
|
||||||
|
import collection.JavaConverters._
|
||||||
import scala.collection.JavaConverters._
|
|
||||||
|
|
||||||
object BioDBToOAF {
|
object BioDBToOAF {
|
||||||
|
|
||||||
case class EBILinkItem(id: Long, links: String) {}
|
case class EBILinkItem(id: Long, links: String) {}
|
||||||
|
@ -17,23 +15,23 @@ object BioDBToOAF {
|
||||||
|
|
||||||
case class UniprotDate(date: String, date_info: String) {}
|
case class UniprotDate(date: String, date_info: String) {}
|
||||||
|
|
||||||
case class ScholixResolved(pid:String, pidType:String, typology:String, tilte:List[String], datasource:List[String], date:List[String], authors:List[String]){}
|
case class ScholixResolved(pid: String, pidType: String, typology: String, tilte: List[String], datasource: List[String], date: List[String], authors: List[String]) {}
|
||||||
|
|
||||||
val DATA_INFO: DataInfo = OafMapperUtils.dataInfo(false, null, false, false, ModelConstants.PROVENANCE_ACTION_SET_QUALIFIER, "0.9")
|
val DATA_INFO: DataInfo = OafMapperUtils.dataInfo(false, null, false, false, ModelConstants.PROVENANCE_ACTION_SET_QUALIFIER, "0.9")
|
||||||
val SUBJ_CLASS = "Keywords"
|
val SUBJ_CLASS = "Keywords"
|
||||||
|
|
||||||
val DATE_RELATION_KEY = "RelationDate"
|
val DATE_RELATION_KEY = "RelationDate"
|
||||||
|
|
||||||
val resolvedURL:Map[String,String] = Map(
|
val resolvedURL: Map[String, String] = Map(
|
||||||
"genbank"-> "https://www.ncbi.nlm.nih.gov/nuccore/",
|
"genbank" -> "https://www.ncbi.nlm.nih.gov/nuccore/",
|
||||||
"ncbi-n" -> "https://www.ncbi.nlm.nih.gov/nuccore/",
|
"ncbi-n" -> "https://www.ncbi.nlm.nih.gov/nuccore/",
|
||||||
"ncbi-wgs" -> "https://www.ncbi.nlm.nih.gov/nuccore/",
|
"ncbi-wgs" -> "https://www.ncbi.nlm.nih.gov/nuccore/",
|
||||||
"ncbi-p" -> "https://www.ncbi.nlm.nih.gov/protein/",
|
"ncbi-p" -> "https://www.ncbi.nlm.nih.gov/protein/",
|
||||||
"ena" -> "https://www.ebi.ac.uk/ena/browser/view/",
|
"ena" -> "https://www.ebi.ac.uk/ena/browser/view/",
|
||||||
"clinicaltrials.gov"-> "https://clinicaltrials.gov/ct2/show/",
|
"clinicaltrials.gov" -> "https://clinicaltrials.gov/ct2/show/",
|
||||||
"onim"-> "https://omim.org/entry/",
|
"onim" -> "https://omim.org/entry/",
|
||||||
"refseq"-> "https://www.ncbi.nlm.nih.gov/nuccore/",
|
"refseq" -> "https://www.ncbi.nlm.nih.gov/nuccore/",
|
||||||
"geo"-> "https://www.ncbi.nlm.nih.gov/geo/query/acc.cgi?acc="
|
"geo" -> "https://www.ncbi.nlm.nih.gov/geo/query/acc.cgi?acc="
|
||||||
)
|
)
|
||||||
|
|
||||||
|
|
||||||
|
@ -45,7 +43,7 @@ object BioDBToOAF {
|
||||||
val ElsevierCollectedFrom: KeyValue = OafMapperUtils.keyValue("10|openaire____::8f87e10869299a5fe80b315695296b88", "Elsevier")
|
val ElsevierCollectedFrom: KeyValue = OafMapperUtils.keyValue("10|openaire____::8f87e10869299a5fe80b315695296b88", "Elsevier")
|
||||||
val springerNatureCollectedFrom: KeyValue = OafMapperUtils.keyValue("10|openaire____::6e380d9cf51138baec8480f5a0ce3a2e", "Springer Nature")
|
val springerNatureCollectedFrom: KeyValue = OafMapperUtils.keyValue("10|openaire____::6e380d9cf51138baec8480f5a0ce3a2e", "Springer Nature")
|
||||||
val EBICollectedFrom: KeyValue = OafMapperUtils.keyValue("10|opendoar____::83e60e09c222f206c725385f53d7e567c", "EMBL-EBIs Protein Data Bank in Europe (PDBe)")
|
val EBICollectedFrom: KeyValue = OafMapperUtils.keyValue("10|opendoar____::83e60e09c222f206c725385f53d7e567c", "EMBL-EBIs Protein Data Bank in Europe (PDBe)")
|
||||||
val pubmedCollectedFrom:KeyValue = OafMapperUtils.keyValue(ModelConstants.EUROPE_PUBMED_CENTRAL_ID, "Europe PubMed Central")
|
val pubmedCollectedFrom: KeyValue = OafMapperUtils.keyValue(ModelConstants.EUROPE_PUBMED_CENTRAL_ID, "Europe PubMed Central")
|
||||||
|
|
||||||
UNIPROTCollectedFrom.setDataInfo(DATA_INFO)
|
UNIPROTCollectedFrom.setDataInfo(DATA_INFO)
|
||||||
PDBCollectedFrom.setDataInfo(DATA_INFO)
|
PDBCollectedFrom.setDataInfo(DATA_INFO)
|
||||||
|
@ -58,9 +56,9 @@ object BioDBToOAF {
|
||||||
|
|
||||||
Map(
|
Map(
|
||||||
"uniprot" -> UNIPROTCollectedFrom,
|
"uniprot" -> UNIPROTCollectedFrom,
|
||||||
"pdb"-> PDBCollectedFrom,
|
"pdb" -> PDBCollectedFrom,
|
||||||
"elsevier" ->ElsevierCollectedFrom,
|
"elsevier" -> ElsevierCollectedFrom,
|
||||||
"ebi" ->EBICollectedFrom,
|
"ebi" -> EBICollectedFrom,
|
||||||
"Springer Nature" -> springerNatureCollectedFrom,
|
"Springer Nature" -> springerNatureCollectedFrom,
|
||||||
"NCBI Nucleotide" -> ncbiCollectedFrom,
|
"NCBI Nucleotide" -> ncbiCollectedFrom,
|
||||||
"European Nucleotide Archive" -> enaCollectedFrom,
|
"European Nucleotide Archive" -> enaCollectedFrom,
|
||||||
|
@ -68,7 +66,7 @@ object BioDBToOAF {
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
||||||
def crossrefLinksToOaf(input:String):Oaf = {
|
def crossrefLinksToOaf(input: String): Oaf = {
|
||||||
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
|
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
|
||||||
lazy val json = parse(input)
|
lazy val json = parse(input)
|
||||||
val source_pid = (json \ "Source" \ "Identifier" \ "ID").extract[String].toLowerCase
|
val source_pid = (json \ "Source" \ "Identifier" \ "ID").extract[String].toLowerCase
|
||||||
|
@ -77,16 +75,16 @@ object BioDBToOAF {
|
||||||
val target_pid = (json \ "Target" \ "Identifier" \ "ID").extract[String].toLowerCase
|
val target_pid = (json \ "Target" \ "Identifier" \ "ID").extract[String].toLowerCase
|
||||||
val target_pid_type = (json \ "Target" \ "Identifier" \ "IDScheme").extract[String].toLowerCase
|
val target_pid_type = (json \ "Target" \ "Identifier" \ "IDScheme").extract[String].toLowerCase
|
||||||
|
|
||||||
val relation_semantic= (json \ "RelationshipType" \ "Name").extract[String]
|
val relation_semantic = (json \ "RelationshipType" \ "Name").extract[String]
|
||||||
|
|
||||||
val date = GraphCleaningFunctions.cleanDate((json \ "LinkedPublicationDate").extract[String])
|
val date = GraphCleaningFunctions.cleanDate((json \ "LinkedPublicationDate").extract[String])
|
||||||
|
|
||||||
createRelation(target_pid, target_pid_type, generate_unresolved_id(source_pid, source_pid_type),collectedFromMap("elsevier"),"relationship", relation_semantic, date)
|
createRelation(target_pid, target_pid_type, generate_unresolved_id(source_pid, source_pid_type), collectedFromMap("elsevier"), "relationship", relation_semantic, date)
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
def scholixResolvedToOAF(input:ScholixResolved):Oaf = {
|
def scholixResolvedToOAF(input: ScholixResolved): Oaf = {
|
||||||
|
|
||||||
val d = new Dataset
|
val d = new Dataset
|
||||||
|
|
||||||
|
@ -127,18 +125,18 @@ object BioDBToOAF {
|
||||||
d.setInstance(List(i).asJava)
|
d.setInstance(List(i).asJava)
|
||||||
|
|
||||||
if (input.authors != null && input.authors.nonEmpty) {
|
if (input.authors != null && input.authors.nonEmpty) {
|
||||||
val authors = input.authors.map(a =>{
|
val authors = input.authors.map(a => {
|
||||||
val authorOAF = new Author
|
val authorOAF = new Author
|
||||||
authorOAF.setFullname(a)
|
authorOAF.setFullname(a)
|
||||||
authorOAF
|
authorOAF
|
||||||
})
|
})
|
||||||
d.setAuthor(authors.asJava)
|
d.setAuthor(authors.asJava)
|
||||||
}
|
}
|
||||||
if (input.date!= null && input.date.nonEmpty) {
|
if (input.date != null && input.date.nonEmpty) {
|
||||||
val dt = input.date.head
|
val dt = input.date.head
|
||||||
i.setDateofacceptance(OafMapperUtils.field(GraphCleaningFunctions.cleanDate(dt), DATA_INFO))
|
i.setDateofacceptance(OafMapperUtils.field(GraphCleaningFunctions.cleanDate(dt), DATA_INFO))
|
||||||
d.setDateofacceptance(OafMapperUtils.field(GraphCleaningFunctions.cleanDate(dt), DATA_INFO))
|
d.setDateofacceptance(OafMapperUtils.field(GraphCleaningFunctions.cleanDate(dt), DATA_INFO))
|
||||||
}
|
}
|
||||||
d
|
d
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -190,7 +188,7 @@ object BioDBToOAF {
|
||||||
OafMapperUtils.structuredProperty(s, SUBJ_CLASS, SUBJ_CLASS, ModelConstants.DNET_SUBJECT_TYPOLOGIES, ModelConstants.DNET_SUBJECT_TYPOLOGIES, null)
|
OafMapperUtils.structuredProperty(s, SUBJ_CLASS, SUBJ_CLASS, ModelConstants.DNET_SUBJECT_TYPOLOGIES, ModelConstants.DNET_SUBJECT_TYPOLOGIES, null)
|
||||||
).asJava)
|
).asJava)
|
||||||
}
|
}
|
||||||
var i_date:Option[UniprotDate] = None
|
var i_date: Option[UniprotDate] = None
|
||||||
|
|
||||||
if (dates.nonEmpty) {
|
if (dates.nonEmpty) {
|
||||||
i_date = dates.find(d => d.date_info.contains("entry version"))
|
i_date = dates.find(d => d.date_info.contains("entry version"))
|
||||||
|
@ -218,12 +216,12 @@ object BioDBToOAF {
|
||||||
|
|
||||||
|
|
||||||
if (references_pmid != null && references_pmid.nonEmpty) {
|
if (references_pmid != null && references_pmid.nonEmpty) {
|
||||||
val rel = createRelation(references_pmid.head, "pmid", d.getId, collectedFromMap("uniprot"), ModelConstants.RELATIONSHIP, ModelConstants.IS_RELATED_TO, if (i_date.isDefined) i_date.get.date else null)
|
val rel = createRelation(references_pmid.head, "pmid", d.getId, collectedFromMap("uniprot"), ModelConstants.RELATIONSHIP, ModelConstants.IS_RELATED_TO, if (i_date.isDefined) i_date.get.date else null)
|
||||||
rel.getCollectedfrom
|
rel.getCollectedfrom
|
||||||
List(d, rel)
|
List(d, rel)
|
||||||
}
|
}
|
||||||
else if (references_doi != null && references_doi.nonEmpty) {
|
else if (references_doi != null && references_doi.nonEmpty) {
|
||||||
val rel = createRelation(references_doi.head, "doi", d.getId, collectedFromMap("uniprot"), ModelConstants.RELATIONSHIP, ModelConstants.IS_RELATED_TO, if (i_date.isDefined) i_date.get.date else null)
|
val rel = createRelation(references_doi.head, "doi", d.getId, collectedFromMap("uniprot"), ModelConstants.RELATIONSHIP, ModelConstants.IS_RELATED_TO, if (i_date.isDefined) i_date.get.date else null)
|
||||||
List(d, rel)
|
List(d, rel)
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
|
@ -231,13 +229,12 @@ object BioDBToOAF {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
def generate_unresolved_id(pid: String, pidType: String): String = {
|
||||||
def generate_unresolved_id(pid:String, pidType:String) :String = {
|
|
||||||
s"unresolved::$pid::$pidType"
|
s"unresolved::$pid::$pidType"
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
def createRelation(pid: String, pidType: String, sourceId: String, collectedFrom: KeyValue, subRelType:String, relClass:String, date:String):Relation = {
|
def createRelation(pid: String, pidType: String, sourceId: String, collectedFrom: KeyValue, subRelType: String, relClass: String, date: String): Relation = {
|
||||||
|
|
||||||
val rel = new Relation
|
val rel = new Relation
|
||||||
rel.setCollectedfrom(List(collectedFromMap("pdb")).asJava)
|
rel.setCollectedfrom(List(collectedFromMap("pdb")).asJava)
|
||||||
|
@ -251,7 +248,7 @@ object BioDBToOAF {
|
||||||
rel.setTarget(s"unresolved::$pid::$pidType")
|
rel.setTarget(s"unresolved::$pid::$pidType")
|
||||||
|
|
||||||
|
|
||||||
val dateProps:KeyValue = OafMapperUtils.keyValue(DATE_RELATION_KEY, date)
|
val dateProps: KeyValue = OafMapperUtils.keyValue(DATE_RELATION_KEY, date)
|
||||||
|
|
||||||
rel.setProperties(List(dateProps).asJava)
|
rel.setProperties(List(dateProps).asJava)
|
||||||
|
|
||||||
|
@ -262,8 +259,8 @@ object BioDBToOAF {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
def createSupplementaryRelation(pid: String, pidType: String, sourceId: String, collectedFrom: KeyValue, date:String): Relation = {
|
def createSupplementaryRelation(pid: String, pidType: String, sourceId: String, collectedFrom: KeyValue, date: String): Relation = {
|
||||||
createRelation(pid,pidType,sourceId,collectedFrom, ModelConstants.SUPPLEMENT, ModelConstants.IS_SUPPLEMENT_TO, date)
|
createRelation(pid, pidType, sourceId, collectedFrom, ModelConstants.SUPPLEMENT, ModelConstants.IS_SUPPLEMENT_TO, date)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -338,7 +335,7 @@ object BioDBToOAF {
|
||||||
|
|
||||||
def EBITargetLinksFilter(input: EBILinks): Boolean = {
|
def EBITargetLinksFilter(input: EBILinks): Boolean = {
|
||||||
|
|
||||||
input.targetPidType.equalsIgnoreCase("ena") || input.targetPidType.equalsIgnoreCase("pdb") || input.targetPidType.equalsIgnoreCase("uniprot")
|
input.targetPidType.equalsIgnoreCase("ena") || input.targetPidType.equalsIgnoreCase("pdb") || input.targetPidType.equalsIgnoreCase("uniprot")
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,8 +1,8 @@
|
||||||
package eu.dnetlib.dhp.sx.graph.bio
|
package eu.dnetllib.dhp.sx.bio
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||||
import eu.dnetlib.dhp.schema.oaf.{Oaf, Result}
|
import eu.dnetlib.dhp.schema.oaf.Oaf
|
||||||
import BioDBToOAF.ScholixResolved
|
import eu.dnetllib.dhp.sx.bio.BioDBToOAF.ScholixResolved
|
||||||
import org.apache.commons.io.IOUtils
|
import org.apache.commons.io.IOUtils
|
||||||
import org.apache.spark.SparkConf
|
import org.apache.spark.SparkConf
|
||||||
import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}
|
import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}
|
||||||
|
@ -31,17 +31,16 @@ object SparkTransformBioDatabaseToOAF {
|
||||||
.master(parser.get("master")).getOrCreate()
|
.master(parser.get("master")).getOrCreate()
|
||||||
val sc = spark.sparkContext
|
val sc = spark.sparkContext
|
||||||
|
|
||||||
implicit val resultEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf])
|
implicit val resultEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf])
|
||||||
import spark.implicits._
|
import spark.implicits._
|
||||||
|
|
||||||
database.toUpperCase() match {
|
database.toUpperCase() match {
|
||||||
case "UNIPROT" =>
|
case "UNIPROT" =>
|
||||||
spark.createDataset(sc.textFile(dbPath).flatMap(i => BioDBToOAF.uniprotToOAF(i))).write.mode(SaveMode.Overwrite).save(targetPath)
|
spark.createDataset(sc.textFile(dbPath).flatMap(i => BioDBToOAF.uniprotToOAF(i))).write.mode(SaveMode.Overwrite).save(targetPath)
|
||||||
case "PDB"=>
|
case "PDB" =>
|
||||||
spark.createDataset(sc.textFile(dbPath).flatMap(i => BioDBToOAF.pdbTOOaf(i))).write.mode(SaveMode.Overwrite).save(targetPath)
|
spark.createDataset(sc.textFile(dbPath).flatMap(i => BioDBToOAF.pdbTOOaf(i))).write.mode(SaveMode.Overwrite).save(targetPath)
|
||||||
case "SCHOLIX" =>
|
case "SCHOLIX" =>
|
||||||
spark.read.load(dbPath).as[ScholixResolved].map(i => BioDBToOAF.scholixResolvedToOAF(i)).write.mode(SaveMode.Overwrite).save(targetPath)
|
spark.read.load(dbPath).as[ScholixResolved].map(i => BioDBToOAF.scholixResolvedToOAF(i)).write.mode(SaveMode.Overwrite).save(targetPath)
|
||||||
case "CROSSREF_LINKS"=>
|
case "CROSSREF_LINKS" =>
|
||||||
spark.createDataset(sc.textFile(dbPath).map(i => BioDBToOAF.crossrefLinksToOaf(i))).write.mode(SaveMode.Overwrite).save(targetPath)
|
spark.createDataset(sc.textFile(dbPath).map(i => BioDBToOAF.crossrefLinksToOaf(i))).write.mode(SaveMode.Overwrite).save(targetPath)
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -0,0 +1,202 @@
|
||||||
|
package eu.dnetllib.dhp.sx.bio.ebi
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||||
|
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup
|
||||||
|
import eu.dnetlib.dhp.schema.oaf.Result
|
||||||
|
import eu.dnetlib.dhp.utils.ISLookupClientFactory
|
||||||
|
import eu.dnetllib.dhp.sx.bio.pubmed.{PMArticle, PMAuthor, PMJournal, PMParser, PubMedToOaf}
|
||||||
|
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
|
||||||
|
|
||||||
|
override def reduce(b: PMArticle, a: (String, PMArticle)): PMArticle = {
|
||||||
|
if (b != null && b.getPmid != null) b else a._2
|
||||||
|
}
|
||||||
|
|
||||||
|
override def merge(b1: PMArticle, b2: PMArticle): PMArticle = {
|
||||||
|
if (b1 != null && b1.getPmid != null) b1 else b2
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
override def finish(reduction: PMArticle): PMArticle = reduction
|
||||||
|
|
||||||
|
override def bufferEncoder: Encoder[PMArticle] = Encoders.kryo[PMArticle]
|
||||||
|
|
||||||
|
override def outputEncoder: Encoder[PMArticle] = Encoders.kryo[PMArticle]
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
def main(args: Array[String]): Unit = {
|
||||||
|
val conf: SparkConf = new SparkConf()
|
||||||
|
val log: Logger = LoggerFactory.getLogger(getClass)
|
||||||
|
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkEBILinksToOaf.getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/bio/ebi/baseline_to_oaf_params.json")))
|
||||||
|
parser.parseArgument(args)
|
||||||
|
val isLookupUrl: String = parser.get("isLookupUrl")
|
||||||
|
log.info("isLookupUrl: {}", isLookupUrl)
|
||||||
|
val workingPath = parser.get("workingPath")
|
||||||
|
log.info("workingPath: {}", workingPath)
|
||||||
|
|
||||||
|
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 =
|
||||||
|
SparkSession
|
||||||
|
.builder()
|
||||||
|
.config(conf)
|
||||||
|
.appName(SparkEBILinksToOaf.getClass.getSimpleName)
|
||||||
|
.master(parser.get("master")).getOrCreate()
|
||||||
|
|
||||||
|
val sc = spark.sparkContext
|
||||||
|
import spark.implicits._
|
||||||
|
|
||||||
|
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_ftp", 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()))
|
||||||
|
new PMParser(xml)
|
||||||
|
|
||||||
|
}))
|
||||||
|
|
||||||
|
ds.map(p => (p.getPmid, p))(Encoders.tuple(Encoders.STRING, PMEncoder)).groupByKey(_._1)
|
||||||
|
.agg(pmArticleAggregator.toColumn)
|
||||||
|
.map(p => p._2).write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_dataset")
|
||||||
|
|
||||||
|
val exported_dataset = spark.read.load(s"$workingPath/baseline_dataset").as[PMArticle]
|
||||||
|
exported_dataset
|
||||||
|
.map(a => PubMedToOaf.convert(a, vocabularies)).as[Result]
|
||||||
|
.filter(p => p != null)
|
||||||
|
.write.mode(SaveMode.Overwrite).save(targetPath)
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,117 @@
|
||||||
|
package eu.dnetllib.dhp.sx.bio.ebi
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||||
|
import eu.dnetllib.dhp.sx.bio.BioDBToOAF.EBILinkItem
|
||||||
|
import eu.dnetllib.dhp.sx.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
|
||||||
|
import org.apache.http.impl.client.HttpClientBuilder
|
||||||
|
import org.apache.spark.SparkConf
|
||||||
|
import org.apache.spark.sql.functions.max
|
||||||
|
import org.apache.spark.sql._
|
||||||
|
import org.slf4j.{Logger, LoggerFactory}
|
||||||
|
|
||||||
|
object SparkDownloadEBILinks {
|
||||||
|
|
||||||
|
def createEBILinks(pmid: Long): EBILinkItem = {
|
||||||
|
|
||||||
|
val res = requestLinks(pmid)
|
||||||
|
if (res != null)
|
||||||
|
return EBILinkItem(pmid, res)
|
||||||
|
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 = {
|
||||||
|
requestPage(s"https://www.ebi.ac.uk/europepmc/webservices/rest/MED/$PMID/datalinks?format=json")
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
def main(args: Array[String]): Unit = {
|
||||||
|
|
||||||
|
val log: Logger = LoggerFactory.getLogger(getClass)
|
||||||
|
val MAX_ITEM_PER_PARTITION = 20000
|
||||||
|
val conf: SparkConf = new SparkConf()
|
||||||
|
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/ebi/ebi_download_update.json")))
|
||||||
|
parser.parseArgument(args)
|
||||||
|
val spark: SparkSession =
|
||||||
|
SparkSession
|
||||||
|
.builder()
|
||||||
|
.config(conf)
|
||||||
|
.appName(SparkEBILinksToOaf.getClass.getSimpleName)
|
||||||
|
.master(parser.get("master")).getOrCreate()
|
||||||
|
|
||||||
|
import spark.implicits._
|
||||||
|
|
||||||
|
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])
|
||||||
|
|
||||||
|
val sourcePath = parser.get("sourcePath")
|
||||||
|
log.info(s"sourcePath -> $sourcePath")
|
||||||
|
val workingPath = parser.get("workingPath")
|
||||||
|
log.info(s"workingPath -> $workingPath")
|
||||||
|
|
||||||
|
log.info("Getting max pubmedId where the links have been requested")
|
||||||
|
val links: Dataset[EBILinkItem] = spark.read.load(s"$sourcePath/ebi_links_dataset").as[EBILinkItem]
|
||||||
|
val lastPMIDRequested = links.map(l => l.id).select(max("value")).first.getLong(0)
|
||||||
|
|
||||||
|
log.info("Retrieving PMID to request links")
|
||||||
|
val pubmed = spark.read.load(s"$sourcePath/baseline_dataset").as[PMArticle]
|
||||||
|
pubmed.map(p => p.getPmid.toLong).where(s"value > $lastPMIDRequested").write.mode(SaveMode.Overwrite).save(s"$workingPath/id_to_request")
|
||||||
|
|
||||||
|
val pmidToReq: Dataset[Long] = spark.read.load(s"$workingPath/id_to_request").as[Long]
|
||||||
|
|
||||||
|
val total = pmidToReq.count()
|
||||||
|
|
||||||
|
spark.createDataset(pmidToReq.rdd.repartition((total / MAX_ITEM_PER_PARTITION).toInt).map(pmid => createEBILinks(pmid)).filter(l => l != null)).write.mode(SaveMode.Overwrite).save(s"$workingPath/links_update")
|
||||||
|
|
||||||
|
val updates: Dataset[EBILinkItem] = spark.read.load(s"$workingPath/links_update").as[EBILinkItem]
|
||||||
|
|
||||||
|
links.union(updates).groupByKey(_.id)
|
||||||
|
.reduceGroups { (x, y) =>
|
||||||
|
if (x == null || x.links == null)
|
||||||
|
y
|
||||||
|
if (y == null || y.links == null)
|
||||||
|
x
|
||||||
|
if (x.links.length > y.links.length)
|
||||||
|
x
|
||||||
|
else
|
||||||
|
y
|
||||||
|
}.map(_._2).write.mode(SaveMode.Overwrite).save(s"$workingPath/links_final")
|
||||||
|
}
|
||||||
|
}
|
|
@ -1,15 +1,14 @@
|
||||||
package eu.dnetlib.dhp.sx.graph.ebi
|
package eu.dnetllib.dhp.sx.bio.ebi
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||||
import eu.dnetlib.dhp.schema.oaf.Oaf
|
import eu.dnetlib.dhp.schema.oaf.Oaf
|
||||||
import eu.dnetlib.dhp.sx.graph.bio
|
import eu.dnetllib.dhp.sx.bio.BioDBToOAF
|
||||||
import eu.dnetlib.dhp.sx.graph.bio.BioDBToOAF
|
import eu.dnetllib.dhp.sx.bio.BioDBToOAF.EBILinkItem
|
||||||
import eu.dnetlib.dhp.sx.graph.bio.BioDBToOAF.EBILinkItem
|
|
||||||
import org.apache.commons.io.IOUtils
|
import org.apache.commons.io.IOUtils
|
||||||
import org.apache.spark.SparkConf
|
import org.apache.spark.SparkConf
|
||||||
import org.apache.spark.rdd.RDD
|
import org.apache.spark.sql._
|
||||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
|
||||||
import org.slf4j.{Logger, LoggerFactory}
|
import org.slf4j.{Logger, LoggerFactory}
|
||||||
|
|
||||||
object SparkEBILinksToOaf {
|
object SparkEBILinksToOaf {
|
||||||
|
|
||||||
def main(args: Array[String]): Unit = {
|
def main(args: Array[String]): Unit = {
|
||||||
|
@ -24,22 +23,17 @@ object SparkEBILinksToOaf {
|
||||||
.appName(SparkEBILinksToOaf.getClass.getSimpleName)
|
.appName(SparkEBILinksToOaf.getClass.getSimpleName)
|
||||||
.master(parser.get("master")).getOrCreate()
|
.master(parser.get("master")).getOrCreate()
|
||||||
|
|
||||||
|
|
||||||
|
import spark.implicits._
|
||||||
val sourcePath = parser.get("sourcePath")
|
val sourcePath = parser.get("sourcePath")
|
||||||
log.info(s"sourcePath -> $sourcePath")
|
log.info(s"sourcePath -> $sourcePath")
|
||||||
val targetPath = parser.get("targetPath")
|
val targetPath = parser.get("targetPath")
|
||||||
log.info(s"targetPath -> $targetPath")
|
log.info(s"targetPath -> $targetPath")
|
||||||
|
implicit val PMEncoder: Encoder[Oaf] = Encoders.kryo(classOf[Oaf])
|
||||||
|
|
||||||
import spark.implicits._
|
val ebLinks: Dataset[EBILinkItem] = spark.read.load(s"${sourcePath}_dataset").as[EBILinkItem].filter(l => l.links != null)
|
||||||
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]
|
ebLinks.flatMap(j => BioDBToOAF.parse_ebi_links(j.links))
|
||||||
|
|
||||||
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))
|
.filter(p => BioDBToOAF.EBITargetLinksFilter(p))
|
||||||
.flatMap(p => BioDBToOAF.convertEBILinksToOaf(p))
|
.flatMap(p => BioDBToOAF.convertEBILinksToOaf(p))
|
||||||
.write.mode(SaveMode.Overwrite).save(targetPath)
|
.write.mode(SaveMode.Overwrite).save(targetPath)
|
|
@ -1,5 +1,5 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.sx.graph.bio.pubmed;
|
package eu.dnetllib.dhp.sx.bio.pubmed;
|
||||||
|
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
|
@ -1,5 +1,5 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.sx.graph.bio.pubmed;
|
package eu.dnetllib.dhp.sx.bio.pubmed;
|
||||||
|
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
|
|
|
@ -1,5 +1,5 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.sx.graph.bio.pubmed;
|
package eu.dnetllib.dhp.sx.bio.pubmed;
|
||||||
|
|
||||||
public class PMGrant {
|
public class PMGrant {
|
||||||
|
|
|
@ -1,5 +1,5 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.sx.graph.bio.pubmed;
|
package eu.dnetllib.dhp.sx.bio.pubmed;
|
||||||
|
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
|
|
|
@ -1,4 +1,4 @@
|
||||||
package eu.dnetlib.dhp.sx.graph.bio.pubmed
|
package eu.dnetllib.dhp.sx.bio.pubmed
|
||||||
|
|
||||||
import scala.xml.MetaData
|
import scala.xml.MetaData
|
||||||
import scala.xml.pull.{EvElemEnd, EvElemStart, EvText, XMLEventReader}
|
import scala.xml.pull.{EvElemEnd, EvElemStart, EvText, XMLEventReader}
|
|
@ -1,5 +1,5 @@
|
||||||
|
|
||||||
package eu.dnetlib.dhp.sx.graph.bio.pubmed;
|
package eu.dnetllib.dhp.sx.bio.pubmed;
|
||||||
|
|
||||||
public class PMSubject {
|
public class PMSubject {
|
||||||
private String value;
|
private String value;
|
|
@ -1,11 +1,12 @@
|
||||||
package eu.dnetlib.dhp.sx.graph.bio.pubmed
|
package eu.dnetllib.dhp.sx.bio.pubmed
|
||||||
|
|
||||||
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup
|
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup
|
||||||
import eu.dnetlib.dhp.schema.common.ModelConstants
|
import eu.dnetlib.dhp.schema.common.ModelConstants
|
||||||
import eu.dnetlib.dhp.schema.oaf._
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.utils.{GraphCleaningFunctions, IdentifierFactory, OafMapperUtils, PidType}
|
import eu.dnetlib.dhp.schema.oaf.utils.{GraphCleaningFunctions, IdentifierFactory, OafMapperUtils, PidType}
|
||||||
|
import eu.dnetlib.dhp.schema.oaf._
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
import java.util.regex.Pattern
|
import java.util.regex.Pattern
|
||||||
import scala.collection.JavaConverters._
|
|
||||||
|
|
||||||
object PubMedToOaf {
|
object PubMedToOaf {
|
||||||
|
|
||||||
|
@ -15,7 +16,7 @@ object PubMedToOaf {
|
||||||
"doi" -> "https://dx.doi.org/"
|
"doi" -> "https://dx.doi.org/"
|
||||||
)
|
)
|
||||||
|
|
||||||
def cleanDoi(doi:String):String = {
|
def cleanDoi(doi: String): String = {
|
||||||
|
|
||||||
val regex = "^10.\\d{4,9}\\/[\\[\\]\\-\\<\\>._;()\\/:A-Z0-9]+$"
|
val regex = "^10.\\d{4,9}\\/[\\[\\]\\-\\<\\>._;()\\/:A-Z0-9]+$"
|
||||||
|
|
||||||
|
@ -71,14 +72,14 @@ object PubMedToOaf {
|
||||||
if (article.getPublicationTypes == null)
|
if (article.getPublicationTypes == null)
|
||||||
return null
|
return null
|
||||||
val i = new Instance
|
val i = new Instance
|
||||||
var pidList: List[StructuredProperty] = List(OafMapperUtils.structuredProperty(article.getPmid, PidType.pmid.toString, PidType.pmid.toString, ModelConstants.DNET_PID_TYPES, ModelConstants.DNET_PID_TYPES, dataInfo))
|
val pidList: List[StructuredProperty] = List(OafMapperUtils.structuredProperty(article.getPmid, PidType.pmid.toString, PidType.pmid.toString, ModelConstants.DNET_PID_TYPES, ModelConstants.DNET_PID_TYPES, dataInfo))
|
||||||
if (pidList == null)
|
if (pidList == null)
|
||||||
return null
|
return null
|
||||||
|
|
||||||
var alternateIdentifier :StructuredProperty = null
|
var alternateIdentifier: StructuredProperty = null
|
||||||
if (article.getDoi != null) {
|
if (article.getDoi != null) {
|
||||||
val normalizedPid = cleanDoi(article.getDoi)
|
val normalizedPid = cleanDoi(article.getDoi)
|
||||||
if (normalizedPid!= null)
|
if (normalizedPid != null)
|
||||||
alternateIdentifier = OafMapperUtils.structuredProperty(normalizedPid, PidType.doi.toString, PidType.doi.toString, ModelConstants.DNET_PID_TYPES, ModelConstants.DNET_PID_TYPES, dataInfo)
|
alternateIdentifier = OafMapperUtils.structuredProperty(normalizedPid, PidType.doi.toString, PidType.doi.toString, ModelConstants.DNET_PID_TYPES, ModelConstants.DNET_PID_TYPES, dataInfo)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -102,10 +103,10 @@ object PubMedToOaf {
|
||||||
return result
|
return result
|
||||||
result.setDataInfo(dataInfo)
|
result.setDataInfo(dataInfo)
|
||||||
i.setPid(pidList.asJava)
|
i.setPid(pidList.asJava)
|
||||||
if (alternateIdentifier!= null)
|
if (alternateIdentifier != null)
|
||||||
i.setAlternateIdentifier(List(alternateIdentifier).asJava)
|
i.setAlternateIdentifier(List(alternateIdentifier).asJava)
|
||||||
result.setInstance(List(i).asJava)
|
result.setInstance(List(i).asJava)
|
||||||
i.getPid.asScala.filter(p => "pmid".equalsIgnoreCase(p.getQualifier.getClassid)).map(p => p.getValue)(collection breakOut)
|
i.getPid.asScala.filter(p => "pmid".equalsIgnoreCase(p.getQualifier.getClassid)).map(p => p.getValue)(collection.breakOut)
|
||||||
val urlLists: List[String] = pidList
|
val urlLists: List[String] = pidList
|
||||||
.map(s => (urlMap.getOrElse(s.getQualifier.getClassid, ""), s.getValue))
|
.map(s => (urlMap.getOrElse(s.getQualifier.getClassid, ""), s.getValue))
|
||||||
.filter(t => t._1.nonEmpty)
|
.filter(t => t._1.nonEmpty)
|
||||||
|
@ -136,7 +137,7 @@ object PubMedToOaf {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
val subjects: List[StructuredProperty] = article.getSubjects.asScala.map(s => OafMapperUtils.structuredProperty(s.getValue, SUBJ_CLASS, SUBJ_CLASS, ModelConstants.DNET_SUBJECT_TYPOLOGIES, ModelConstants.DNET_SUBJECT_TYPOLOGIES, dataInfo))(collection breakOut)
|
val subjects: List[StructuredProperty] = article.getSubjects.asScala.map(s => OafMapperUtils.structuredProperty(s.getValue, SUBJ_CLASS, SUBJ_CLASS, ModelConstants.DNET_SUBJECT_TYPOLOGIES, ModelConstants.DNET_SUBJECT_TYPOLOGIES, dataInfo))(collection.breakOut)
|
||||||
if (subjects != null)
|
if (subjects != null)
|
||||||
result.setSubject(subjects.asJava)
|
result.setSubject(subjects.asJava)
|
||||||
|
|
||||||
|
@ -148,7 +149,7 @@ object PubMedToOaf {
|
||||||
author.setFullname(a.getFullName)
|
author.setFullname(a.getFullName)
|
||||||
author.setRank(index + 1)
|
author.setRank(index + 1)
|
||||||
author
|
author
|
||||||
}(collection breakOut)
|
}(collection.breakOut)
|
||||||
|
|
||||||
|
|
||||||
if (authors != null && authors.nonEmpty)
|
if (authors != null && authors.nonEmpty)
|
|
@ -0,0 +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":"h", "paramLongName":"hdfsServerUri", "paramDescription": "the working path ", "paramRequired": true}
|
||||||
|
]
|
|
@ -0,0 +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}
|
||||||
|
]
|
|
@ -1,13 +1,9 @@
|
||||||
<workflow-app name="Transform_Pubmed_Workflow" xmlns="uri:oozie:workflow:0.5">
|
<workflow-app name="Download_Transform_Pubmed_Workflow" xmlns="uri:oozie:workflow:0.5">
|
||||||
<parameters>
|
<parameters>
|
||||||
<property>
|
<property>
|
||||||
<name>baselineWorkingPath</name>
|
<name>baselineWorkingPath</name>
|
||||||
<description>the Baseline Working Path</description>
|
<description>the Baseline Working Path</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
|
||||||
<name>targetPath</name>
|
|
||||||
<description>the Target Path</description>
|
|
||||||
</property>
|
|
||||||
<property>
|
<property>
|
||||||
<name>isLookupUrl</name>
|
<name>isLookupUrl</name>
|
||||||
<description>The IS lookUp service endopoint</description>
|
<description>The IS lookUp service endopoint</description>
|
||||||
|
@ -24,8 +20,8 @@
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
<master>yarn</master>
|
<master>yarn</master>
|
||||||
<mode>cluster</mode>
|
<mode>cluster</mode>
|
||||||
<name>Convert Baseline to Dataset</name>
|
<name>Convert Baseline to OAF Dataset</name>
|
||||||
<class>eu.dnetlib.dhp.sx.graph.ebi.SparkCreateBaselineDataFrame</class>
|
<class>eu.dnetllib.dhp.sx.bio.ebi.SparkCreateBaselineDataFrame</class>
|
||||||
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
|
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
|
||||||
<spark-opts>
|
<spark-opts>
|
||||||
--executor-memory=${sparkExecutorMemory}
|
--executor-memory=${sparkExecutorMemory}
|
||||||
|
@ -38,9 +34,10 @@
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--workingPath</arg><arg>${baselineWorkingPath}</arg>
|
<arg>--workingPath</arg><arg>${baselineWorkingPath}</arg>
|
||||||
<arg>--targetPath</arg><arg>${targetPath}</arg>
|
<arg>--targetPath</arg><arg>${baselineWorkingPath}/transformed</arg>
|
||||||
<arg>--master</arg><arg>yarn</arg>
|
<arg>--master</arg><arg>yarn</arg>
|
||||||
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
|
||||||
|
<arg>--hdfsServerUri</arg><arg>${nameNode}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="End"/>
|
<ok to="End"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
|
@ -1,12 +1,10 @@
|
||||||
package eu.dnetlib.dhp.sx.graph.bio.pubmed
|
package eu.dnetllib.dhp.sx.bio
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper, SerializationFeature}
|
import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper, SerializationFeature}
|
||||||
import eu.dnetlib.dhp.schema.common.ModelConstants
|
import eu.dnetlib.dhp.aggregation.AbstractVocabularyTest
|
||||||
import eu.dnetlib.dhp.schema.oaf.utils.{CleaningFunctions, OafMapperUtils, PidType}
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.{Oaf, Relation, Result}
|
import eu.dnetlib.dhp.schema.oaf.{Oaf, Relation, Result}
|
||||||
import eu.dnetlib.dhp.sx.graph.bio.BioDBToOAF.ScholixResolved
|
import eu.dnetllib.dhp.sx.bio.BioDBToOAF.ScholixResolved
|
||||||
import eu.dnetlib.dhp.sx.graph.bio.BioDBToOAF
|
import eu.dnetllib.dhp.sx.bio.pubmed.{PMArticle, PMParser, PubMedToOaf}
|
||||||
import eu.dnetlib.dhp.sx.graph.bio.pubmed.PubMedToOaf.dataInfo
|
|
||||||
import org.json4s.DefaultFormats
|
import org.json4s.DefaultFormats
|
||||||
import org.json4s.JsonAST.{JField, JObject, JString}
|
import org.json4s.JsonAST.{JField, JObject, JString}
|
||||||
import org.json4s.jackson.JsonMethods.parse
|
import org.json4s.jackson.JsonMethods.parse
|
||||||
|
@ -50,9 +48,11 @@ class BioScholixTest extends AbstractVocabularyTest{
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def testEBIData() = {
|
def testEBIData() = {
|
||||||
val inputXML = Source.fromInputStream(getClass.getResourceAsStream("pubmed.xml")).mkString
|
val inputXML = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/bio/pubmed.xml")).mkString
|
||||||
val xml = new XMLEventReader(Source.fromBytes(inputXML.getBytes()))
|
val xml = new XMLEventReader(Source.fromBytes(inputXML.getBytes()))
|
||||||
new PMParser(xml).foreach(s =>println(mapper.writeValueAsString(s)))
|
new PMParser(xml).foreach(s =>println(mapper.writeValueAsString(s)))
|
||||||
}
|
}
|
||||||
|
@ -62,7 +62,7 @@ class BioScholixTest extends AbstractVocabularyTest{
|
||||||
def testPubmedToOaf(): Unit = {
|
def testPubmedToOaf(): Unit = {
|
||||||
assertNotNull(vocabularies)
|
assertNotNull(vocabularies)
|
||||||
assertTrue(vocabularies.vocabularyExists("dnet:publication_resource"))
|
assertTrue(vocabularies.vocabularyExists("dnet:publication_resource"))
|
||||||
val records:String =Source.fromInputStream(getClass.getResourceAsStream("pubmed_dump")).mkString
|
val records:String =Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/bio/pubmed_dump")).mkString
|
||||||
val r:List[Oaf] = records.lines.toList.map(s=>mapper.readValue(s, classOf[PMArticle])).map(a => PubMedToOaf.convert(a, vocabularies))
|
val r:List[Oaf] = records.lines.toList.map(s=>mapper.readValue(s, classOf[PMArticle])).map(a => PubMedToOaf.convert(a, vocabularies))
|
||||||
assertEquals(10, r.size)
|
assertEquals(10, r.size)
|
||||||
assertTrue(r.map(p => p.asInstanceOf[Result]).flatMap(p => p.getInstance().asScala.map(i => i.getInstancetype.getClassid)).exists(p => "0037".equalsIgnoreCase(p)))
|
assertTrue(r.map(p => p.asInstanceOf[Result]).flatMap(p => p.getInstance().asScala.map(i => i.getInstancetype.getClassid)).exists(p => "0037".equalsIgnoreCase(p)))
|
File diff suppressed because it is too large
Load Diff
|
@ -6,10 +6,7 @@ import java.util.Optional;
|
||||||
|
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.JavaPairRDD;
|
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
import org.apache.spark.api.java.function.FilterFunction;
|
|
||||||
import org.apache.spark.api.java.function.MapFunction;
|
import org.apache.spark.api.java.function.MapFunction;
|
||||||
import org.apache.spark.api.java.function.PairFunction;
|
import org.apache.spark.api.java.function.PairFunction;
|
||||||
import org.apache.spark.sql.Dataset;
|
import org.apache.spark.sql.Dataset;
|
||||||
|
@ -22,7 +19,6 @@ import org.slf4j.LoggerFactory;
|
||||||
import org.xml.sax.SAXException;
|
import org.xml.sax.SAXException;
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
|
||||||
import eu.dnetlib.dhp.oa.dedup.model.Block;
|
|
||||||
import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
import eu.dnetlib.dhp.schema.oaf.DataInfo;
|
||||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||||
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
|
||||||
|
@ -32,7 +28,6 @@ import eu.dnetlib.pace.config.DedupConfig;
|
||||||
import eu.dnetlib.pace.model.MapDocument;
|
import eu.dnetlib.pace.model.MapDocument;
|
||||||
import eu.dnetlib.pace.util.MapDocumentUtil;
|
import eu.dnetlib.pace.util.MapDocumentUtil;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
import scala.Tuple3;
|
|
||||||
|
|
||||||
public class SparkWhitelistSimRels extends AbstractSparkAction {
|
public class SparkWhitelistSimRels extends AbstractSparkAction {
|
||||||
|
|
||||||
|
|
|
@ -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>
|
<name>dbPwd</name>
|
||||||
<description>password to access the OpenOrgs database</description>
|
<description>password to access the OpenOrgs database</description>
|
||||||
</property>
|
</property>
|
||||||
|
<property>
|
||||||
|
<name>dbConnections</name>
|
||||||
|
<value>10</value>
|
||||||
|
<description>number of connections to the postgres db</description>
|
||||||
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>workingPath</name>
|
<name>workingPath</name>
|
||||||
<description>path for the working directory</description>
|
<description>path for the working directory</description>
|
||||||
|
@ -223,7 +228,7 @@
|
||||||
<arg>--dbTable</arg><arg>${dbTable}</arg>
|
<arg>--dbTable</arg><arg>${dbTable}</arg>
|
||||||
<arg>--dbUser</arg><arg>${dbUser}</arg>
|
<arg>--dbUser</arg><arg>${dbUser}</arg>
|
||||||
<arg>--dbPwd</arg><arg>${dbPwd}</arg>
|
<arg>--dbPwd</arg><arg>${dbPwd}</arg>
|
||||||
<arg>--numConnections</arg><arg>20</arg>
|
<arg>--numConnections</arg><arg>${dbConnections}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="PrepareNewOrgs"/>
|
<ok to="PrepareNewOrgs"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
@ -254,19 +259,24 @@
|
||||||
<arg>--dbTable</arg><arg>${dbTable}</arg>
|
<arg>--dbTable</arg><arg>${dbTable}</arg>
|
||||||
<arg>--dbUser</arg><arg>${dbUser}</arg>
|
<arg>--dbUser</arg><arg>${dbUser}</arg>
|
||||||
<arg>--dbPwd</arg><arg>${dbPwd}</arg>
|
<arg>--dbPwd</arg><arg>${dbPwd}</arg>
|
||||||
<arg>--numConnections</arg><arg>20</arg>
|
<arg>--numConnections</arg><arg>${dbConnections}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="update_openorgs"/>
|
<ok to="update_openorgs"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
|
||||||
<action name="update_openorgs">
|
<action name="update_openorgs">
|
||||||
<shell xmlns="uri:oozie:shell-action:0.1">
|
<java>
|
||||||
<job-tracker>${jobTracker}</job-tracker>
|
<configuration>
|
||||||
<name-node>${nameNode}</name-node>
|
<property>
|
||||||
<exec>/usr/bin/curl</exec>
|
<name>oozie.launcher.mapreduce.user.classpath.first</name>
|
||||||
<argument>${apiUrl}</argument>
|
<value>true</value>
|
||||||
</shell>
|
</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"/>
|
<ok to="End"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</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
|
||||||
|
}
|
||||||
|
]
|
|
@ -208,7 +208,7 @@ object SparkGenerateDoiBoost {
|
||||||
(r.getTarget,r)
|
(r.getTarget,r)
|
||||||
else
|
else
|
||||||
("resolved", r)
|
("resolved", r)
|
||||||
})
|
})(Encoders.tuple(Encoders.STRING, mapEncoderRel))
|
||||||
|
|
||||||
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)
|
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)
|
||||||
|
|
||||||
|
@ -222,7 +222,7 @@ object SparkGenerateDoiBoost {
|
||||||
else
|
else
|
||||||
currentRels.setTarget(currentOrgs._1)
|
currentRels.setTarget(currentOrgs._1)
|
||||||
currentRels
|
currentRels
|
||||||
}.write.save(s"$workingDirPath/doiBoostPublicationAffiliation")
|
}.filter(r=> !r.getSource.startsWith("unresolved") && !r.getTarget.startsWith("unresolved")).write.mode(SaveMode.Overwrite).save(s"$workingDirPath/doiBoostPublicationAffiliation")
|
||||||
|
|
||||||
magPubs.joinWith(a,magPubs("_1").equalTo(a("PaperId"))).map( item => {
|
magPubs.joinWith(a,magPubs("_1").equalTo(a("PaperId"))).map( item => {
|
||||||
val affiliation = item._2
|
val affiliation = item._2
|
||||||
|
|
|
@ -1,93 +0,0 @@
|
||||||
package eu.dnetlib.dhp.sx.graph.ebi
|
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
|
||||||
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup
|
|
||||||
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.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 scala.io.Source
|
|
||||||
import scala.xml.pull.XMLEventReader
|
|
||||||
|
|
||||||
object SparkCreateBaselineDataFrame {
|
|
||||||
|
|
||||||
|
|
||||||
val pmArticleAggregator: Aggregator[(String, PMArticle), PMArticle, PMArticle] = new Aggregator[(String, PMArticle), PMArticle, PMArticle] with Serializable {
|
|
||||||
override def zero: PMArticle = new PMArticle
|
|
||||||
|
|
||||||
override def reduce(b: PMArticle, a: (String, PMArticle)): PMArticle = {
|
|
||||||
if (b != null && b.getPmid!= null) b else a._2
|
|
||||||
}
|
|
||||||
|
|
||||||
override def merge(b1: PMArticle, b2: PMArticle): PMArticle = {
|
|
||||||
if (b1 != null && b1.getPmid!= null) b1 else b2
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
override def finish(reduction: PMArticle): PMArticle = reduction
|
|
||||||
|
|
||||||
override def bufferEncoder: Encoder[PMArticle] = Encoders.kryo[PMArticle]
|
|
||||||
|
|
||||||
override def outputEncoder: Encoder[PMArticle] = Encoders.kryo[PMArticle]
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
def main(args: Array[String]): Unit = {
|
|
||||||
val conf: SparkConf = new SparkConf()
|
|
||||||
val log: Logger = LoggerFactory.getLogger(getClass)
|
|
||||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkEBILinksToOaf.getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/ebi/baseline_to_oaf_params.json")))
|
|
||||||
parser.parseArgument(args)
|
|
||||||
val isLookupUrl: String = parser.get("isLookupUrl")
|
|
||||||
log.info("isLookupUrl: {}", isLookupUrl)
|
|
||||||
val workingPath = parser.get("workingPath")
|
|
||||||
log.info("workingPath: {}", workingPath)
|
|
||||||
|
|
||||||
val targetPath = parser.get("targetPath")
|
|
||||||
log.info("targetPath: {}", targetPath)
|
|
||||||
|
|
||||||
val isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl)
|
|
||||||
val vocabularies = VocabularyGroup.loadVocsFromIS(isLookupService)
|
|
||||||
val spark: SparkSession =
|
|
||||||
SparkSession
|
|
||||||
.builder()
|
|
||||||
.config(conf)
|
|
||||||
.appName(SparkEBILinksToOaf.getClass.getSimpleName)
|
|
||||||
.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])
|
|
||||||
|
|
||||||
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()))
|
|
||||||
new PMParser(xml)
|
|
||||||
|
|
||||||
} ))
|
|
||||||
|
|
||||||
ds.map(p => (p.getPmid,p))(Encoders.tuple(Encoders.STRING, PMEncoder)).groupByKey(_._1)
|
|
||||||
.agg(pmArticleAggregator.toColumn)
|
|
||||||
.map(p => p._2).write.mode(SaveMode.Overwrite).save(s"$workingPath/baseline_dataset")
|
|
||||||
|
|
||||||
val exported_dataset = spark.read.load(s"$workingPath/baseline_dataset").as[PMArticle]
|
|
||||||
exported_dataset
|
|
||||||
.map(a => PubMedToOaf.convert(a, vocabularies)).as[Result]
|
|
||||||
.filter(p => p!= null)
|
|
||||||
.write.mode(SaveMode.Overwrite).save(targetPath)
|
|
||||||
|
|
||||||
//s"$workingPath/oaf/baseline_oaf"
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,115 +0,0 @@
|
||||||
package eu.dnetlib.dhp.sx.graph.ebi
|
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
|
||||||
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.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.slf4j.{Logger, LoggerFactory}
|
|
||||||
|
|
||||||
object SparkDownloadEBILinks {
|
|
||||||
|
|
||||||
|
|
||||||
def createEBILinks(pmid:Long):EBILinkItem = {
|
|
||||||
|
|
||||||
val res = requestLinks(pmid)
|
|
||||||
if (res!=null)
|
|
||||||
return EBILinkItem(pmid, res)
|
|
||||||
null
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
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()
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
def main(args: Array[String]): Unit = {
|
|
||||||
|
|
||||||
val log: Logger = LoggerFactory.getLogger(getClass)
|
|
||||||
val MAX_ITEM_PER_PARTITION = 20000
|
|
||||||
val conf: SparkConf = new SparkConf()
|
|
||||||
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/ebi/ebi_download_update.json")))
|
|
||||||
parser.parseArgument(args)
|
|
||||||
val spark: SparkSession =
|
|
||||||
SparkSession
|
|
||||||
.builder()
|
|
||||||
.config(conf)
|
|
||||||
.appName(SparkEBILinksToOaf.getClass.getSimpleName)
|
|
||||||
.master(parser.get("master")).getOrCreate()
|
|
||||||
|
|
||||||
import spark.implicits._
|
|
||||||
|
|
||||||
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])
|
|
||||||
|
|
||||||
val sourcePath = parser.get("sourcePath")
|
|
||||||
log.info(s"sourcePath -> $sourcePath")
|
|
||||||
val workingPath = parser.get("workingPath")
|
|
||||||
log.info(s"workingPath -> $workingPath")
|
|
||||||
|
|
||||||
log.info("Getting max pubmedId where the links have been requested")
|
|
||||||
val links:Dataset[EBILinkItem] = spark.read.load(s"$sourcePath/ebi_links_dataset").as[EBILinkItem]
|
|
||||||
val lastPMIDRequested =links.map(l => l.id).select(max("value")).first.getLong(0)
|
|
||||||
|
|
||||||
log.info("Retrieving PMID to request links")
|
|
||||||
val pubmed = spark.read.load(s"$sourcePath/baseline_dataset").as[PMArticle]
|
|
||||||
pubmed.map(p => p.getPmid.toLong).where(s"value > $lastPMIDRequested").write.mode(SaveMode.Overwrite).save(s"$workingPath/id_to_request")
|
|
||||||
|
|
||||||
val pmidToReq:Dataset[Long] = spark.read.load(s"$workingPath/id_to_request").as[Long]
|
|
||||||
|
|
||||||
val total = pmidToReq.count()
|
|
||||||
|
|
||||||
spark.createDataset(pmidToReq.rdd.repartition((total/MAX_ITEM_PER_PARTITION).toInt).map(pmid =>createEBILinks(pmid)).filter(l => l!= null)).write.mode(SaveMode.Overwrite).save(s"$workingPath/links_update")
|
|
||||||
|
|
||||||
val updates:Dataset[EBILinkItem] =spark.read.load(s"$workingPath/links_update").as[EBILinkItem]
|
|
||||||
|
|
||||||
links.union(updates).groupByKey(_.id)
|
|
||||||
.reduceGroups{(x,y) =>
|
|
||||||
if (x == null || x.links ==null)
|
|
||||||
y
|
|
||||||
if (y ==null || y.links ==null)
|
|
||||||
x
|
|
||||||
if (x.links.length > y.links.length)
|
|
||||||
x
|
|
||||||
else
|
|
||||||
y
|
|
||||||
}.map(_._2).write.mode(SaveMode.Overwrite).save(s"$workingPath/links_final")
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -1,7 +1,6 @@
|
||||||
package eu.dnetlib.dhp.sx.graph.pangaea
|
package eu.dnetlib.dhp.sx.graph.pangaea
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||||
import eu.dnetlib.dhp.sx.graph.ebi.SparkEBILinksToOaf
|
|
||||||
import org.apache.spark.rdd.RDD
|
import org.apache.spark.rdd.RDD
|
||||||
import org.apache.spark.{SparkConf, SparkContext}
|
import org.apache.spark.{SparkConf, SparkContext}
|
||||||
import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}
|
import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}
|
||||||
|
|
|
@ -1,6 +0,0 @@
|
||||||
[
|
|
||||||
{"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}
|
|
||||||
]
|
|
|
@ -1,5 +0,0 @@
|
||||||
[
|
|
||||||
{"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>
|
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||||
</kill>
|
</kill>
|
||||||
|
|
||||||
|
|
||||||
<action name="GenerateBaselineDataset">
|
<action name="GenerateBaselineDataset">
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
<master>yarn-cluster</master>
|
<master>yarn-cluster</master>
|
||||||
|
@ -43,6 +42,7 @@
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||||
<arg>--master</arg><arg>yarn</arg>
|
<arg>--master</arg><arg>yarn</arg>
|
||||||
|
<arg>--hdfsServerUri</arg><arg>${nameNode}</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="End"/>
|
<ok to="End"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
@ -74,7 +74,7 @@
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
<master>yarn-cluster</master>
|
<master>yarn-cluster</master>
|
||||||
<mode>cluster</mode>
|
<mode>cluster</mode>
|
||||||
<name>Create Baselnie DataSet</name>
|
<name>Create Baseline DataSet</name>
|
||||||
|
|
||||||
<class>eu.dnetlib.dhp.sx.ebi.SparkAddLinkUpdates</class>
|
<class>eu.dnetlib.dhp.sx.ebi.SparkAddLinkUpdates</class>
|
||||||
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
|
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
|
||||||
|
|
|
@ -1,59 +1,67 @@
|
||||||
<workflow-app name="Create EBI Dataset" xmlns="uri:oozie:workflow:0.5">
|
<workflow-app name="Create EBI Dataset" xmlns="uri:oozie:workflow:0.5">
|
||||||
<parameters>
|
<parameters>
|
||||||
<property>
|
<property>
|
||||||
<name>sourcePath</name>
|
<name>sourcePath</name>
|
||||||
<description>the Working Path</description>
|
<description>the Working Path</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>workingPath</name>
|
<name>workingPath</name>
|
||||||
<description>the Working Path</description>
|
<description>the Working Path</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>sparkDriverMemory</name>
|
<name>sparkDriverMemory</name>
|
||||||
<description>memory for driver process</description>
|
<description>memory for driver process</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>sparkExecutorMemory</name>
|
<name>sparkExecutorMemory</name>
|
||||||
<description>memory for individual executor</description>
|
<description>memory for individual executor</description>
|
||||||
</property>
|
</property>
|
||||||
<property>
|
<property>
|
||||||
<name>sparkExecutorCores</name>
|
<name>sparkExecutorCores</name>
|
||||||
<description>number of cores used by single executor</description>
|
<description>number of cores used by single executor</description>
|
||||||
</property>
|
</property>
|
||||||
</parameters>
|
</parameters>
|
||||||
|
|
||||||
<start to="DownloadEBILinks"/>
|
<start to="DownloadEBILinks"/>
|
||||||
|
|
||||||
|
|
||||||
<kill name="Kill">
|
<kill name="Kill">
|
||||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||||
</kill>
|
</kill>
|
||||||
|
|
||||||
|
|
||||||
<action name="DownloadEBILinks">
|
<action name="DownloadEBILinks">
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
<master>yarn-cluster</master>
|
<master>yarn-cluster</master>
|
||||||
<mode>cluster</mode>
|
<mode>cluster</mode>
|
||||||
<name>Incremental Download EBI Links</name>
|
<name>Incremental Download EBI Links</name>
|
||||||
<class>eu.dnetlib.dhp.sx.graph.ebi.SparkDownloadEBILinks</class>
|
<class>eu.dnetlib.dhp.sx.graph.ebi.SparkDownloadEBILinks</class>
|
||||||
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
|
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
|
||||||
<spark-opts>
|
<spark-opts>
|
||||||
--executor-memory=${sparkExecutorMemory}
|
--executor-memory=${sparkExecutorMemory}
|
||||||
--executor-cores=${sparkExecutorCores}
|
--executor-cores=${sparkExecutorCores}
|
||||||
--driver-memory=${sparkDriverMemory}
|
--driver-memory=${sparkDriverMemory}
|
||||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||||
--conf spark.sql.shuffle.partitions=2000
|
--conf spark.sql.shuffle.partitions=2000
|
||||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
|
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
|
||||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||||
<arg>--master</arg><arg>yarn</arg>
|
<arg>--master</arg><arg>yarn</arg>
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="End"/>
|
<ok to="OverrideFolders"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
<action name="OverrideFolders">
|
||||||
<end name="End"/>
|
<fs>
|
||||||
</workflow-app>
|
<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>
|
|
@ -89,6 +89,26 @@ public class IndexRecordTransformerTest {
|
||||||
testRecordTransformation(record);
|
testRecordTransformation(record);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testForEOSCFutureAirQualityCopernicus() throws IOException, TransformerException {
|
||||||
|
final String record = IOUtils
|
||||||
|
.toString(getClass().getResourceAsStream("eosc-future/air-quality-copernicus.xml"));
|
||||||
|
testRecordTransformation(record);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testForEOSCFutureB2SharePlotSw() throws IOException, TransformerException {
|
||||||
|
final String record = IOUtils.toString(getClass().getResourceAsStream("eosc-future/b2share-plot-sw.xml"));
|
||||||
|
testRecordTransformation(record);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testForEOSCFutureB2SharePlotRelatedORP() throws IOException, TransformerException {
|
||||||
|
final String record = IOUtils
|
||||||
|
.toString(getClass().getResourceAsStream("eosc-future/b2share-plot-related-orp.xml"));
|
||||||
|
testRecordTransformation(record);
|
||||||
|
}
|
||||||
|
|
||||||
private void testRecordTransformation(final String record) throws IOException, TransformerException {
|
private void testRecordTransformation(final String record) throws IOException, TransformerException {
|
||||||
final String fields = IOUtils.toString(getClass().getResourceAsStream("fields.xml"));
|
final String fields = IOUtils.toString(getClass().getResourceAsStream("fields.xml"));
|
||||||
final String xslt = IOUtils.toString(getClass().getResourceAsStream("layoutToRecordTransformer.xsl"));
|
final String xslt = IOUtils.toString(getClass().getResourceAsStream("layoutToRecordTransformer.xsl"));
|
||||||
|
|
|
@ -0,0 +1,114 @@
|
||||||
|
<record>
|
||||||
|
<result xmlns:dri="http://www.driver-repository.eu/namespace/dri"
|
||||||
|
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
|
||||||
|
<header>
|
||||||
|
<dri:objIdentifier>r37b0ad08687::a8df7db30ae0e4e0b875a098df7b652f</dri:objIdentifier>
|
||||||
|
<dri:dateOfCollection>2021-10-07T01:56:56Z</dri:dateOfCollection>
|
||||||
|
<dri:status>under curation</dri:status>
|
||||||
|
<counters/>
|
||||||
|
</header>
|
||||||
|
<metadata>
|
||||||
|
<oaf:entity xmlns:oaf="http://namespace.openaire.eu/oaf"
|
||||||
|
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||||
|
xsi:schemaLocation="http://namespace.openaire.eu/oaf http://namespace.openaire.eu/oaf http://www.openaire.eu/schema/0.2/oaf-0.2.xsd">
|
||||||
|
|
||||||
|
<oaf:result>
|
||||||
|
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title"
|
||||||
|
schemename="dnet:dataCite_title">Using CAMS European air quality analysis from Copernicus
|
||||||
|
Atmosphere Monitoring with RELIANCE services
|
||||||
|
</title>
|
||||||
|
<creator rank="1" name="Simone" surname="Mantovani" orcid_pending="0000-0003-3979-3645">Simone Mantovani</creator>
|
||||||
|
<dateofacceptance>2021-10-07</dateofacceptance>
|
||||||
|
<resulttype classid="software" classname="software" schemeid="dnet:result_typologies"
|
||||||
|
schemename="dnet:result_typologies"/>
|
||||||
|
<language classid="" classname="" schemeid="dnet:languages" schemename="dnet:languages"/>
|
||||||
|
<description>
|
||||||
|
This notebook shows how to discover and access the Copernicus Atmosphere Monitoring products available in the RELIANCE datacube resources.
|
||||||
|
The process is structured in 6 steps, including example of data analysis and visualization with the Python libraries installed in the Jupyter environment
|
||||||
|
</description>
|
||||||
|
<country classid="" classname="" schemeid="" schemename=""/>
|
||||||
|
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
|
||||||
|
schemename="dnet:subject_classification_typologies">EOSC Jupyter Notebook
|
||||||
|
</subject>
|
||||||
|
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
|
||||||
|
schemename="dnet:subject_classification_typologies">RELIANCE
|
||||||
|
</subject>
|
||||||
|
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
|
||||||
|
schemename="dnet:subject_classification_typologies">Copernicus
|
||||||
|
</subject>
|
||||||
|
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
|
||||||
|
schemename="dnet:subject_classification_typologies">Air quality
|
||||||
|
</subject>
|
||||||
|
|
||||||
|
<relevantdate classid="" classname="" schemeid="" schemename=""/>
|
||||||
|
<publisher>Zenodo</publisher>
|
||||||
|
<embargoenddate/>
|
||||||
|
<journal issn="" eissn="" lissn="" ep="" iss="" sp="" vol=""/>
|
||||||
|
<source/>
|
||||||
|
<fulltext/>
|
||||||
|
<format/>
|
||||||
|
<storagedate/>
|
||||||
|
<resourcetype classid="" classname="" schemeid="" schemename=""/>
|
||||||
|
<device/>
|
||||||
|
<size/>
|
||||||
|
<version/>
|
||||||
|
<lastmetadataupdate/>
|
||||||
|
<metadataversionnumber/>
|
||||||
|
<documentationUrl/>
|
||||||
|
<codeRepositoryUrl/>
|
||||||
|
<programmingLanguage classid="" classname="" schemeid="" schemename=""/>
|
||||||
|
<contactperson/>
|
||||||
|
<contactgroup/>
|
||||||
|
<tool/>
|
||||||
|
<originalId>oai:zenodo.org:5554786</originalId>
|
||||||
|
<collectedfrom name="Zenodo" id="re3data_____::7b0ad08687b2c960d5aeef06f811d5e6"/>
|
||||||
|
<pid classid="oai" classname="Open Archives Initiative" schemeid="dnet:pid_types"
|
||||||
|
schemename="dnet:pid_types">oai:zenodo.org:5554786
|
||||||
|
</pid>
|
||||||
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
|
||||||
|
schemename="dnet:pid_types">10.5281/zenodo.5554786
|
||||||
|
</pid>
|
||||||
|
<bestaccessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes"
|
||||||
|
schemename="dnet:access_modes"/>
|
||||||
|
<datainfo>
|
||||||
|
<inferred>false</inferred>
|
||||||
|
<deletedbyinference>false</deletedbyinference>
|
||||||
|
<trust>0.9</trust>
|
||||||
|
<inferenceprovenance/>
|
||||||
|
<provenanceaction classid="user:insert" classname="user:insert"
|
||||||
|
schemeid="dnet:provenanceActions" schemename="dnet:provenanceActions"/>
|
||||||
|
</datainfo>
|
||||||
|
<rels>
|
||||||
|
<rel inferred="false" trust="0.9" inferenceprovenance="" provenanceaction="user:claim">
|
||||||
|
<to class="isProducedBy" scheme="dnet:result_project_relations" type="project">
|
||||||
|
corda__h2020::8771f523c34e38902d4921037d545ef8
|
||||||
|
</to>
|
||||||
|
<title>REsearch LIfecycle mAnagemeNt for Earth Science Communities and CopErnicus users in EOSC</title>
|
||||||
|
<code>101017501</code>
|
||||||
|
<acronym>RELIANCE</acronym>
|
||||||
|
<funding>
|
||||||
|
<funder id="ec__________::EC" shortname="EC" name="European Commission" jurisdiction="EU" />
|
||||||
|
<funding_level_0 name="H2020">ec__________::EC::H2020</funding_level_0>
|
||||||
|
<funding_level_1 name="RIA">ec__________::EC::H2020::RIA</funding_level_1>
|
||||||
|
</funding>
|
||||||
|
</rel>
|
||||||
|
</rels>
|
||||||
|
<children>
|
||||||
|
<instance id="r37b0ad08687::a8df7db30ae0e4e0b875a098df7b652f">
|
||||||
|
<instancetype classid="0029" classname="Software" schemeid="dnet:publication_resource"
|
||||||
|
schemename="dnet:publication_resource"/>
|
||||||
|
<collectedfrom name="Zenodo" id="re3data_____::7b0ad08687b2c960d5aeef06f811d5e6"/>
|
||||||
|
<hostedby name="Zenodo" id="re3data_____::7b0ad08687b2c960d5aeef06f811d5e6"/>
|
||||||
|
<accessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes"
|
||||||
|
schemename="dnet:access_modes"/>
|
||||||
|
<dateofacceptance/>
|
||||||
|
<webresource>
|
||||||
|
<url>https://zenodo.org/record/5554786</url>
|
||||||
|
</webresource>
|
||||||
|
</instance>
|
||||||
|
</children>
|
||||||
|
</oaf:result>
|
||||||
|
</oaf:entity>
|
||||||
|
</metadata>
|
||||||
|
</result>
|
||||||
|
</record>
|
|
@ -0,0 +1,288 @@
|
||||||
|
<record>
|
||||||
|
<result xmlns:dri="http://www.driver-repository.eu/namespace/dri"
|
||||||
|
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
|
||||||
|
<header>
|
||||||
|
<dri:objIdentifier>doi_dedup___::44fd8a9b5b79adb0783ac245b21e3127</dri:objIdentifier>
|
||||||
|
<dri:dateOfCollection>2019-09-19T07:43:31+0000</dri:dateOfCollection>
|
||||||
|
<dri:dateOfTransformation>2019-09-19T07:43:31+0000</dri:dateOfTransformation>
|
||||||
|
</header>
|
||||||
|
<metadata>
|
||||||
|
<oaf:entity xmlns:oaf="http://namespace.openaire.eu/oaf"
|
||||||
|
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||||
|
xsi:schemaLocation="http://namespace.openaire.eu/oaf http://namespace.openaire.eu/oaf http://www.openaire.eu/schema/0.2/oaf-0.2.xsd">
|
||||||
|
|
||||||
|
<oaf:result>
|
||||||
|
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254"/>
|
||||||
|
<collectedfrom name="B2FIND" id="re3data_____::730f562f9efe8a3b3742d2da510d4335"/>
|
||||||
|
<originalId>10.23728/b2share.ebcd2972c5fb44199f8b3fdf9f6413c6</originalId>
|
||||||
|
<originalId>10.23728/b2share.a69a7b2dcc22449e8734552dde4d3906</originalId>
|
||||||
|
<originalId>6a93c069-a167-44cb-bfe8-74c275637347</originalId>
|
||||||
|
<originalId>50|r3730f562f9e::9b434fedc00d568b8e00611a7fa19f41</originalId>
|
||||||
|
<originalId>10.23728/b2share.7c8655b6f25348358b4e6fece7ab6016</originalId>
|
||||||
|
<originalId>ada23067-496a-494f-bd82-6ffe3cf4f0fb</originalId>
|
||||||
|
<originalId>50|r3730f562f9e::b9cd774e8126b6902d56f9a4aa03e1dc</originalId>
|
||||||
|
<originalId>f3bd1041-422c-439d-8e68-c1d0711d130d</originalId>
|
||||||
|
<originalId>50|r3730f562f9e::b847821a0ca5365b0d971dd89dea6bf1</originalId>
|
||||||
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
|
||||||
|
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
|
||||||
|
trust="0.9">10.23728/b2share.ebcd2972c5fb44199f8b3fdf9f6413c6
|
||||||
|
</pid>
|
||||||
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
|
||||||
|
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
|
||||||
|
trust="0.9">10.23728/b2share.a69a7b2dcc22449e8734552dde4d3906
|
||||||
|
</pid>
|
||||||
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
|
||||||
|
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
|
||||||
|
trust="0.9">10.23728/b2share.7c8655b6f25348358b4e6fece7ab6016
|
||||||
|
</pid>
|
||||||
|
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title"
|
||||||
|
schemename="dnet:dataCite_title">HCG16 L-band VLA C+D array final data
|
||||||
|
</title>
|
||||||
|
<bestaccessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes"
|
||||||
|
schemename="dnet:access_modes"/>
|
||||||
|
<creator rank="1" name="Michael G." surname="Jones">Jones, Michael G.</creator>
|
||||||
|
<dateofacceptance>2019-01-01</dateofacceptance>
|
||||||
|
<description>These are the reduced final data associated with the paper Jones et al. 2019 submitted
|
||||||
|
to Astronomy & Astrophysics. They are used by a mybinder (https://gke.mybinder.org/)
|
||||||
|
executable environment to generate the final plots of that paper. The link for this environment
|
||||||
|
is https://mybinder.org/v2/gh/AMIGA-IAA/hcg-16/master. The raw VLA D and C array data of HCG 16
|
||||||
|
were collected by the Very Large Array (http://www.vla.nrao.edu/) in 1989 and 1999, under PI
|
||||||
|
projects of Barbara Williams. The project numbers are AW234 and AW500 respectively. The file
|
||||||
|
also includes a grz colour image and r-band image from DECaLS DR8
|
||||||
|
(http://legacysurvey.org/decamls/), a GBT HI spectrum published in Borthakur et al. 2010 (ApJ
|
||||||
|
710, 385), an HI data cube from HIPASS (https://www.atnf.csiro.au/research/multibeam/release/),
|
||||||
|
and a source mask (and associated parameters file) for the HIPASS cube generated using SoFiA
|
||||||
|
(https://github.com/SoFiA-Admin/SoFiA-2).
|
||||||
|
</description>
|
||||||
|
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
|
||||||
|
schemename="dnet:subject_classification_typologies">3.5.2.1.1 → Observational astronomy →
|
||||||
|
Radio astronomy
|
||||||
|
</subject>
|
||||||
|
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
|
||||||
|
schemename="dnet:subject_classification_typologies">HI
|
||||||
|
</subject>
|
||||||
|
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
|
||||||
|
schemename="dnet:subject_classification_typologies">VLA
|
||||||
|
</subject>
|
||||||
|
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
|
||||||
|
schemename="dnet:subject_classification_typologies">HCG16
|
||||||
|
</subject>
|
||||||
|
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
|
||||||
|
schemename="dnet:subject_classification_typologies" inferred="false"
|
||||||
|
provenanceaction="sysimport:crosswalk:repository" trust="0.9">Various
|
||||||
|
</subject>
|
||||||
|
<language classid="und" classname="Undetermined" schemeid="dnet:languages"
|
||||||
|
schemename="dnet:languages"/>
|
||||||
|
<relevantdate classid="issued" classname="issued" schemeid="dnet:dataCite_date"
|
||||||
|
schemename="dnet:dataCite_date">2019-01-01
|
||||||
|
</relevantdate>
|
||||||
|
<publisher>https://b2share.eudat.eu</publisher>
|
||||||
|
<resulttype classid="other" classname="other" schemeid="dnet:result_typologies"
|
||||||
|
schemename="dnet:result_typologies"/>
|
||||||
|
<resourcetype classid="UNKNOWN" classname="Unknown" schemeid="dnet:dataCite_resource"
|
||||||
|
schemename="dnet:dataCite_resource"/>
|
||||||
|
<datainfo>
|
||||||
|
<inferred>true</inferred>
|
||||||
|
<deletedbyinference>false</deletedbyinference>
|
||||||
|
<trust>0.8</trust>
|
||||||
|
<inferenceprovenance>dedup-similarity-result-decisiontree-v2</inferenceprovenance>
|
||||||
|
<provenanceaction classid="sysimport:dedup" classname="Inferred by OpenAIRE"
|
||||||
|
schemeid="dnet:provenanceActions" schemename="dnet:provenanceActions"/>
|
||||||
|
</datainfo>
|
||||||
|
<rels>
|
||||||
|
<rel inferred="false" trust="0.9" inferenceprovenance="" provenanceaction="user:claim">
|
||||||
|
<to class="IsRelatedTo" scheme="dnet:result_result_relations" type="software">userclaim___::ee29372a239b79db3ac4c5debe44d6e6</to>
|
||||||
|
<collectedfrom name="B2FIND" id="re3data_____::730f562f9efe8a3b3742d2da510d4335"/>
|
||||||
|
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title">Plot scripts for HCG-16 Project</title>
|
||||||
|
</rel>
|
||||||
|
</rels>
|
||||||
|
<children>
|
||||||
|
<result objidentifier="r3730f562f9e::9b434fedc00d568b8e00611a7fa19f41">
|
||||||
|
<dateofacceptance>2019-01-01</dateofacceptance>
|
||||||
|
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title"
|
||||||
|
schemename="dnet:dataCite_title" inferred="false"
|
||||||
|
provenanceaction="sysimport:crosswalk:repository" trust="0.9">HCG16 L-band VLA C+D
|
||||||
|
array final data
|
||||||
|
</title>
|
||||||
|
<collectedfrom name="B2FIND" id="re3data_____::730f562f9efe8a3b3742d2da510d4335"/>
|
||||||
|
<publisher>B2SHARE</publisher>
|
||||||
|
</result>
|
||||||
|
<result objidentifier="doi_________::929c1e415f4bb04797679d1af7cb706f">
|
||||||
|
<dateofacceptance>2019-01-01</dateofacceptance>
|
||||||
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
|
||||||
|
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
|
||||||
|
trust="0.9">10.23728/b2share.a69a7b2dcc22449e8734552dde4d3906
|
||||||
|
</pid>
|
||||||
|
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title"
|
||||||
|
schemename="dnet:dataCite_title">HCG16 L-band VLA C+D array final data
|
||||||
|
</title>
|
||||||
|
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254"/>
|
||||||
|
<publisher>https://b2share.eudat.eu</publisher>
|
||||||
|
</result>
|
||||||
|
<result objidentifier="doi_________::e24d8a6399c5d8df9a78aed032573b81">
|
||||||
|
<dateofacceptance>2019-01-01</dateofacceptance>
|
||||||
|
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title"
|
||||||
|
schemename="dnet:dataCite_title">HCG16 L-band VLA C+D array final data
|
||||||
|
</title>
|
||||||
|
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254"/>
|
||||||
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
|
||||||
|
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
|
||||||
|
trust="0.9">10.23728/b2share.7c8655b6f25348358b4e6fece7ab6016
|
||||||
|
</pid>
|
||||||
|
<publisher>https://b2share.eudat.eu</publisher>
|
||||||
|
</result>
|
||||||
|
<result objidentifier="r3730f562f9e::b9cd774e8126b6902d56f9a4aa03e1dc">
|
||||||
|
<dateofacceptance>2019-01-01</dateofacceptance>
|
||||||
|
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title"
|
||||||
|
schemename="dnet:dataCite_title" inferred="false"
|
||||||
|
provenanceaction="sysimport:crosswalk:repository" trust="0.9">HCG16 L-band VLA C+D
|
||||||
|
array final data
|
||||||
|
</title>
|
||||||
|
<collectedfrom name="B2FIND" id="re3data_____::730f562f9efe8a3b3742d2da510d4335"/>
|
||||||
|
<publisher>B2SHARE</publisher>
|
||||||
|
</result>
|
||||||
|
<result objidentifier="doi_________::44fd8a9b5b79adb0783ac245b21e3127">
|
||||||
|
<dateofacceptance>2019-01-01</dateofacceptance>
|
||||||
|
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title"
|
||||||
|
schemename="dnet:dataCite_title">HCG16 L-band VLA C+D array final data
|
||||||
|
</title>
|
||||||
|
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254"/>
|
||||||
|
<publisher>https://b2share.eudat.eu</publisher>
|
||||||
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
|
||||||
|
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
|
||||||
|
trust="0.9">10.23728/b2share.ebcd2972c5fb44199f8b3fdf9f6413c6
|
||||||
|
</pid>
|
||||||
|
</result>
|
||||||
|
<result objidentifier="r3730f562f9e::b847821a0ca5365b0d971dd89dea6bf1">
|
||||||
|
<dateofacceptance>2019-01-01</dateofacceptance>
|
||||||
|
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title"
|
||||||
|
schemename="dnet:dataCite_title" inferred="false"
|
||||||
|
provenanceaction="sysimport:crosswalk:repository" trust="0.9">HCG16 L-band VLA C+D
|
||||||
|
array final data
|
||||||
|
</title>
|
||||||
|
<collectedfrom name="B2FIND" id="re3data_____::730f562f9efe8a3b3742d2da510d4335"/>
|
||||||
|
<publisher>B2SHARE</publisher>
|
||||||
|
</result>
|
||||||
|
<instance id="openaire____::55045bd2a65019fd8e6741a755395c8c">
|
||||||
|
<accessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes"
|
||||||
|
schemename="dnet:access_modes"/>
|
||||||
|
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254"/>
|
||||||
|
<hostedby name="Unknown Repository" id="openaire____::55045bd2a65019fd8e6741a755395c8c"/>
|
||||||
|
<dateofacceptance>2019-01-01</dateofacceptance>
|
||||||
|
<instancetype classid="0020" classname="Other ORP type" schemeid="dnet:publication_resource"
|
||||||
|
schemename="dnet:publication_resource"/>
|
||||||
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
|
||||||
|
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
|
||||||
|
trust="0.9">10.23728/b2share.ebcd2972c5fb44199f8b3fdf9f6413c6
|
||||||
|
</pid>
|
||||||
|
<refereed classid="0000" classname="Unknown" schemeid="dnet:review_levels"
|
||||||
|
schemename="dnet:review_levels"/>
|
||||||
|
<webresource>
|
||||||
|
<url>https://dx.doi.org/10.23728/b2share.ebcd2972c5fb44199f8b3fdf9f6413c6</url>
|
||||||
|
</webresource>
|
||||||
|
</instance>
|
||||||
|
<instance id="openaire____::55045bd2a65019fd8e6741a755395c8c">
|
||||||
|
<accessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes"
|
||||||
|
schemename="dnet:access_modes"/>
|
||||||
|
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254"/>
|
||||||
|
<hostedby name="Unknown Repository" id="openaire____::55045bd2a65019fd8e6741a755395c8c"/>
|
||||||
|
<dateofacceptance>2019-01-01</dateofacceptance>
|
||||||
|
<instancetype classid="0020" classname="Other ORP type" schemeid="dnet:publication_resource"
|
||||||
|
schemename="dnet:publication_resource"/>
|
||||||
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
|
||||||
|
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
|
||||||
|
trust="0.9">10.23728/b2share.a69a7b2dcc22449e8734552dde4d3906
|
||||||
|
</pid>
|
||||||
|
<refereed classid="0000" classname="Unknown" schemeid="dnet:review_levels"
|
||||||
|
schemename="dnet:review_levels"/>
|
||||||
|
<webresource>
|
||||||
|
<url>https://dx.doi.org/10.23728/b2share.a69a7b2dcc22449e8734552dde4d3906</url>
|
||||||
|
</webresource>
|
||||||
|
</instance>
|
||||||
|
<instance id="re3data_____::730f562f9efe8a3b3742d2da510d4335">
|
||||||
|
<accessright classid="UNKNOWN" classname="not available" schemeid="dnet:access_modes"
|
||||||
|
schemename="dnet:access_modes"/>
|
||||||
|
<collectedfrom name="B2FIND" id="re3data_____::730f562f9efe8a3b3742d2da510d4335"/>
|
||||||
|
<hostedby name="B2FIND" id="re3data_____::730f562f9efe8a3b3742d2da510d4335"/>
|
||||||
|
<dateofacceptance>2019-01-01</dateofacceptance>
|
||||||
|
<instancetype classid="0000" classname="Unknown" schemeid="dnet:publication_resource"
|
||||||
|
schemename="dnet:publication_resource"/>
|
||||||
|
<alternateidentifier classid="doi" classname="Digital Object Identifier"
|
||||||
|
schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false"
|
||||||
|
provenanceaction="sysimport:crosswalk:repository" trust="0.9">
|
||||||
|
https://doi.org10.23728/b2share.ebcd2972c5fb44199f8b3fdf9f6413c6
|
||||||
|
</alternateidentifier>
|
||||||
|
<refereed classid="0000" classname="Unknown" schemeid="dnet:review_levels"
|
||||||
|
schemename="dnet:review_levels"/>
|
||||||
|
<webresource>
|
||||||
|
<url>
|
||||||
|
http://dx.doi.org/https://doi.org/10.23728/b2share.ebcd2972c5fb44199f8b3fdf9f6413c6
|
||||||
|
</url>
|
||||||
|
</webresource>
|
||||||
|
</instance>
|
||||||
|
<instance id="openaire____::55045bd2a65019fd8e6741a755395c8c">
|
||||||
|
<accessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes"
|
||||||
|
schemename="dnet:access_modes"/>
|
||||||
|
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254"/>
|
||||||
|
<hostedby name="Unknown Repository" id="openaire____::55045bd2a65019fd8e6741a755395c8c"/>
|
||||||
|
<dateofacceptance>2019-01-01</dateofacceptance>
|
||||||
|
<instancetype classid="0020" classname="Other ORP type" schemeid="dnet:publication_resource"
|
||||||
|
schemename="dnet:publication_resource"/>
|
||||||
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
|
||||||
|
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
|
||||||
|
trust="0.9">10.23728/b2share.7c8655b6f25348358b4e6fece7ab6016
|
||||||
|
</pid>
|
||||||
|
<refereed classid="0000" classname="Unknown" schemeid="dnet:review_levels"
|
||||||
|
schemename="dnet:review_levels"/>
|
||||||
|
<webresource>
|
||||||
|
<url>https://dx.doi.org/10.23728/b2share.7c8655b6f25348358b4e6fece7ab6016</url>
|
||||||
|
</webresource>
|
||||||
|
</instance>
|
||||||
|
<instance id="re3data_____::730f562f9efe8a3b3742d2da510d4335">
|
||||||
|
<accessright classid="UNKNOWN" classname="not available" schemeid="dnet:access_modes"
|
||||||
|
schemename="dnet:access_modes"/>
|
||||||
|
<collectedfrom name="B2FIND" id="re3data_____::730f562f9efe8a3b3742d2da510d4335"/>
|
||||||
|
<hostedby name="B2FIND" id="re3data_____::730f562f9efe8a3b3742d2da510d4335"/>
|
||||||
|
<dateofacceptance>2019-01-01</dateofacceptance>
|
||||||
|
<instancetype classid="0000" classname="Unknown" schemeid="dnet:publication_resource"
|
||||||
|
schemename="dnet:publication_resource"/>
|
||||||
|
<alternateidentifier classid="doi" classname="Digital Object Identifier"
|
||||||
|
schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false"
|
||||||
|
provenanceaction="sysimport:crosswalk:repository" trust="0.9">
|
||||||
|
https://doi.org10.23728/b2share.7c8655b6f25348358b4e6fece7ab6016
|
||||||
|
</alternateidentifier>
|
||||||
|
<refereed classid="0000" classname="Unknown" schemeid="dnet:review_levels"
|
||||||
|
schemename="dnet:review_levels"/>
|
||||||
|
<webresource>
|
||||||
|
<url>
|
||||||
|
http://dx.doi.org/https://doi.org/10.23728/b2share.7c8655b6f25348358b4e6fece7ab6016
|
||||||
|
</url>
|
||||||
|
</webresource>
|
||||||
|
</instance>
|
||||||
|
<instance id="re3data_____::730f562f9efe8a3b3742d2da510d4335">
|
||||||
|
<accessright classid="UNKNOWN" classname="not available" schemeid="dnet:access_modes"
|
||||||
|
schemename="dnet:access_modes"/>
|
||||||
|
<collectedfrom name="B2FIND" id="re3data_____::730f562f9efe8a3b3742d2da510d4335"/>
|
||||||
|
<hostedby name="B2FIND" id="re3data_____::730f562f9efe8a3b3742d2da510d4335"/>
|
||||||
|
<dateofacceptance>2019-01-01</dateofacceptance>
|
||||||
|
<instancetype classid="0000" classname="Unknown" schemeid="dnet:publication_resource"
|
||||||
|
schemename="dnet:publication_resource"/>
|
||||||
|
<alternateidentifier classid="doi" classname="Digital Object Identifier"
|
||||||
|
schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false"
|
||||||
|
provenanceaction="sysimport:crosswalk:repository" trust="0.9">
|
||||||
|
https://doi.org10.23728/b2share.a69a7b2dcc22449e8734552dde4d3906
|
||||||
|
</alternateidentifier>
|
||||||
|
<refereed classid="0000" classname="Unknown" schemeid="dnet:review_levels"
|
||||||
|
schemename="dnet:review_levels"/>
|
||||||
|
<webresource>
|
||||||
|
<url>
|
||||||
|
http://dx.doi.org/https://doi.org/10.23728/b2share.a69a7b2dcc22449e8734552dde4d3906
|
||||||
|
</url>
|
||||||
|
</webresource>
|
||||||
|
</instance>
|
||||||
|
</children>
|
||||||
|
</oaf:result>
|
||||||
|
</oaf:entity>
|
||||||
|
</metadata>
|
||||||
|
</result>
|
||||||
|
</record>
|
|
@ -0,0 +1,112 @@
|
||||||
|
<record>
|
||||||
|
<result xmlns:dri="http://www.driver-repository.eu/namespace/dri"
|
||||||
|
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
|
||||||
|
<header>
|
||||||
|
<dri:objIdentifier>userclaim___::ee29372a239b79db3ac4c5debe44d6e6</dri:objIdentifier>
|
||||||
|
<dri:dateOfCollection>2021-10-07T12:42:54Z</dri:dateOfCollection>
|
||||||
|
<counters/>
|
||||||
|
</header>
|
||||||
|
<metadata>
|
||||||
|
<oaf:entity xmlns:oaf="http://namespace.openaire.eu/oaf"
|
||||||
|
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||||
|
xsi:schemaLocation="http://namespace.openaire.eu/oaf http://namespace.openaire.eu/oaf http://www.openaire.eu/schema/0.2/oaf-0.2.xsd">
|
||||||
|
|
||||||
|
<oaf:result>
|
||||||
|
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title"
|
||||||
|
schemename="dnet:dataCite_title">Plot scripts for HCG-16 Project
|
||||||
|
</title>
|
||||||
|
<creator rank="1" name="Michael G." surname="Jones">Jones, Michael G.</creator>
|
||||||
|
<creator rank="2" name="Sebastián" surname="Luna-Valero">Jones, Michael G.</creator>
|
||||||
|
<dateofacceptance>2021-09-30</dateofacceptance>
|
||||||
|
<resulttype classid="software" classname="software" schemeid="dnet:result_typologies"
|
||||||
|
schemename="dnet:result_typologies"/>
|
||||||
|
<language classid="" classname="" schemeid="dnet:languages" schemename="dnet:languages"/>
|
||||||
|
<description>These are the notebooks to general the final data plots of the paper Jones et al. 2019
|
||||||
|
submitted to Astronomy & Astrophysics. They can be used in a notebooks environment (like
|
||||||
|
https://notebooks.egi.eu/) with the proper libraries installed. A mybinder
|
||||||
|
(https://mybinder.org/)
|
||||||
|
ready version can be started from https://mybinder.org/v2/gh/AMIGA-IAA/hcg-16/master. Data to
|
||||||
|
generate plots is also available from B2SHARE:
|
||||||
|
https://b2share.eudat.eu/records/a69a7b2dcc22449e8734552dde4d3906
|
||||||
|
</description>
|
||||||
|
<country classid="" classname="" schemeid="" schemename=""/>
|
||||||
|
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies"
|
||||||
|
schemename="dnet:subject_classification_typologies">EOSC Jupyter Notebook
|
||||||
|
</subject>
|
||||||
|
<relevantdate classid="" classname="" schemeid="" schemename=""/>
|
||||||
|
<publisher>B2SHARE</publisher>
|
||||||
|
<embargoenddate/>
|
||||||
|
<journal issn="" eissn="" lissn="" ep="" iss="" sp="" vol=""/>
|
||||||
|
<source/>
|
||||||
|
<fulltext/>
|
||||||
|
<format/>
|
||||||
|
<storagedate/>
|
||||||
|
<resourcetype classid="" classname="" schemeid="" schemename=""/>
|
||||||
|
<device/>
|
||||||
|
<size/>
|
||||||
|
<version/>
|
||||||
|
<lastmetadataupdate/>
|
||||||
|
<metadataversionnumber/>
|
||||||
|
<documentationUrl/>
|
||||||
|
<codeRepositoryUrl/>
|
||||||
|
<programmingLanguage classid="" classname="" schemeid="" schemename=""/>
|
||||||
|
<contactperson/>
|
||||||
|
<contactgroup/>
|
||||||
|
<tool/>
|
||||||
|
<originalId>userclaim___::ee29372a239b79db3ac4c5debe44d6e6</originalId>
|
||||||
|
<collectedfrom name="B2FIND" id="re3data_____::730f562f9efe8a3b3742d2da510d4335"/>
|
||||||
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
|
||||||
|
schemename="dnet:pid_types">10.23728/b2share.adf6e2e942b04561a8640c449b48c14a
|
||||||
|
</pid>
|
||||||
|
<bestaccessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes"
|
||||||
|
schemename="dnet:access_modes"/>
|
||||||
|
<datainfo>
|
||||||
|
<inferred>false</inferred>
|
||||||
|
<deletedbyinference>false</deletedbyinference>
|
||||||
|
<trust>0.9</trust>
|
||||||
|
<inferenceprovenance/>
|
||||||
|
<provenanceaction classid="user:insert" classname="user:insert"
|
||||||
|
schemeid="dnet:provenanceActions"
|
||||||
|
schemename="dnet:provenanceActions"/>
|
||||||
|
</datainfo>
|
||||||
|
<rels>
|
||||||
|
<rel inferred="false" trust="0.9" inferenceprovenance="" provenanceaction="user:claim">
|
||||||
|
<to class="IsRelatedTo" scheme="dnet:result_result_relations" type="otherresearchproduct">doi_dedup___::44fd8a9b5b79adb0783ac245b21e3127</to>
|
||||||
|
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title">HCG16 L-band VLA C+D array final data</title>
|
||||||
|
<dateofacceptance>2019-01-01</dateofacceptance>
|
||||||
|
<publisher>https://b2share.eudat.eu</publisher>
|
||||||
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
|
||||||
|
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
|
||||||
|
trust="0.9">10.23728/b2share.ebcd2972c5fb44199f8b3fdf9f6413c6
|
||||||
|
</pid>
|
||||||
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
|
||||||
|
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
|
||||||
|
trust="0.9">10.23728/b2share.a69a7b2dcc22449e8734552dde4d3906
|
||||||
|
</pid>
|
||||||
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types"
|
||||||
|
schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset"
|
||||||
|
trust="0.9">10.23728/b2share.7c8655b6f25348358b4e6fece7ab6016
|
||||||
|
</pid>
|
||||||
|
<collectedfrom name="Datacite" id="openaire____::9e3be59865b2c1c335d32dae2fe7b254"/>
|
||||||
|
<collectedfrom name="B2FIND" id="re3data_____::730f562f9efe8a3b3742d2da510d4335"/>
|
||||||
|
</rel>
|
||||||
|
</rels>
|
||||||
|
<children>
|
||||||
|
<instance id="userclaim___::ee29372a239b79db3ac4c5debe44d6e6">
|
||||||
|
<instancetype classid="0029" classname="Software" schemeid="dnet:publication_resource"
|
||||||
|
schemename="dnet:publication_resource"/>
|
||||||
|
<collectedfrom name="B2FIND" id="re3data_____::730f562f9efe8a3b3742d2da510d4335"/>
|
||||||
|
<hostedby name="B2SHARE" id="re3data_____::ad3609c351bd520edf6f10f5e0d9b877"/>
|
||||||
|
<accessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes"
|
||||||
|
schemename="dnet:access_modes"/>
|
||||||
|
<dateofacceptance>2021-09-30</dateofacceptance>
|
||||||
|
<webresource>
|
||||||
|
<url>http://dx.doi.org/10.23728/b2share.adf6e2e942b04561a8640c449b48c14a</url>
|
||||||
|
</webresource>
|
||||||
|
</instance>
|
||||||
|
</children>
|
||||||
|
</oaf:result>
|
||||||
|
</oaf:entity>
|
||||||
|
</metadata>
|
||||||
|
</result>
|
||||||
|
</record>
|
|
@ -1,26 +1,25 @@
|
||||||
<record>
|
<record>
|
||||||
<result xmlns:dri="http://www.driver-repository.eu/namespace/dri" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
|
<result xmlns:dri="http://www.driver-repository.eu/namespace/dri" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
|
||||||
<header>
|
<header>
|
||||||
<dri:objIdentifier>r37b0ad08687::dec0d8520e726f2adda9a51280ac7299</dri:objIdentifier>
|
<dri:objIdentifier>doi_dedup___::ab57f086011a9ae23d1165211dc6e04b</dri:objIdentifier>
|
||||||
<dri:dateOfCollection>2021-09-22T08:53:16Z</dri:dateOfCollection>
|
<dri:dateOfCollection>2020-11-03T05:39:50+0000</dri:dateOfCollection>
|
||||||
<dri:status>under curation</dri:status>
|
<dri:dateOfTransformation>2020-11-03T05:39:50+0000</dri:dateOfTransformation>
|
||||||
<counters />
|
|
||||||
</header>
|
</header>
|
||||||
<metadata>
|
<metadata>
|
||||||
<oaf:entity xmlns:oaf="http://namespace.openaire.eu/oaf"
|
<oaf:entity xmlns:oaf="http://namespace.openaire.eu/oaf"
|
||||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://namespace.openaire.eu/oaf http://namespace.openaire.eu/oaf http://www.openaire.eu/schema/0.2/oaf-0.2.xsd">
|
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://namespace.openaire.eu/oaf http://namespace.openaire.eu/oaf http://www.openaire.eu/schema/0.2/oaf-0.2.xsd">
|
||||||
<oaf:result>
|
<oaf:result>
|
||||||
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title">EGI-Foundation/data-transfer-pilot: Include libraries in environment.yml</title>
|
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title">EGI-Foundation/data-transfer-pilot: Include libraries in environment.yml</title>
|
||||||
<creator rank="1" name="" surname="">Giuseppe La Rocca</creator>
|
<creator rank="1" name="Giuseppe" surname="La Rocca">Giuseppe La Rocca</creator>
|
||||||
<creator rank="2" name="" surname="">Enol Fernández</creator>
|
<creator rank="2" name="Enol" surname="Fernández">Enol Fernández</creator>
|
||||||
<creator rank="3" name="" surname="">Andrea Manzi</creator>
|
<creator rank="3" name="Andrea" surname="Manzi">Andrea Manzi</creator>
|
||||||
<dateofacceptance />
|
<dateofacceptance>2020-11-03</dateofacceptance>
|
||||||
<resulttype classid="software" classname="software" schemeid="dnet:result_typologies" schemename="dnet:result_typologies" />
|
<resulttype classid="software" classname="software" schemeid="dnet:result_typologies" schemename="dnet:result_typologies" />
|
||||||
<language classid="" classname="" schemeid="dnet:languages" schemename="dnet:languages" />
|
<language classid="" classname="" schemeid="dnet:languages" schemename="dnet:languages" />
|
||||||
<description>This notebook is used to demonstrate how a scientist from one of the PaNOSC RIs can use the resources provided by EGI to perform analysis on the data sets obtained during an expirement.</description>
|
<description>This notebook is used to demonstrate how a scientist from one of the PaNOSC RIs can use the resources provided by EGI to perform analysis on the data sets obtained during an expirement.</description>
|
||||||
<country classid="" classname="" schemeid="" schemename="" />
|
<country classid="" classname="" schemeid="" schemename="" />
|
||||||
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies">EOSC Jupyter Notebook</subject>
|
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies">EOSC Jupyter Notebook</subject>
|
||||||
<relevantdate classid="" classname="" schemeid="" schemename="" />
|
<relevantdate classid="issued" classname="issued" schemeid="dnet:dataCite_date" schemename="dnet:dataCite_date">2020-11-03</relevantdate>
|
||||||
<publisher>Zenodo</publisher>
|
<publisher>Zenodo</publisher>
|
||||||
<embargoenddate />
|
<embargoenddate />
|
||||||
<journal issn="" eissn="" lissn="" ep="" iss="" sp="" vol="" />
|
<journal issn="" eissn="" lissn="" ep="" iss="" sp="" vol="" />
|
||||||
|
@ -43,8 +42,8 @@
|
||||||
<originalId>oai:zenodo.org:4218562</originalId>
|
<originalId>oai:zenodo.org:4218562</originalId>
|
||||||
<collectedfrom name="Zenodo" id="re3data_____::7b0ad08687b2c960d5aeef06f811d5e6" />
|
<collectedfrom name="Zenodo" id="re3data_____::7b0ad08687b2c960d5aeef06f811d5e6" />
|
||||||
<pid classid="oai" classname="Open Archives Initiative" schemeid="dnet:pid_types" schemename="dnet:pid_types">oai:zenodo.org:4218562</pid>
|
<pid classid="oai" classname="Open Archives Initiative" schemeid="dnet:pid_types" schemename="dnet:pid_types">oai:zenodo.org:4218562</pid>
|
||||||
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types">10.5281/zenodo.4218562</pid>
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">10.5281/zenodo.4195418</pid>
|
||||||
<bestaccessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes" schemename="dnet:access_modes" />
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">10.5281/zenodo.4218562</pid><bestaccessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes" schemename="dnet:access_modes" />
|
||||||
<datainfo>
|
<datainfo>
|
||||||
<inferred>false</inferred>
|
<inferred>false</inferred>
|
||||||
<deletedbyinference>false</deletedbyinference>
|
<deletedbyinference>false</deletedbyinference>
|
||||||
|
@ -59,7 +58,7 @@
|
||||||
<collectedfrom name="Zenodo" id="re3data_____::7b0ad08687b2c960d5aeef06f811d5e6" />
|
<collectedfrom name="Zenodo" id="re3data_____::7b0ad08687b2c960d5aeef06f811d5e6" />
|
||||||
<hostedby name="Zenodo" id="re3data_____::7b0ad08687b2c960d5aeef06f811d5e6" />
|
<hostedby name="Zenodo" id="re3data_____::7b0ad08687b2c960d5aeef06f811d5e6" />
|
||||||
<accessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes" schemename="dnet:access_modes" />
|
<accessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes" schemename="dnet:access_modes" />
|
||||||
<dateofacceptance />
|
<dateofacceptance>2020-11-03</dateofacceptance>
|
||||||
<webresource>
|
<webresource>
|
||||||
<url>https://zenodo.org/record/4218562</url>
|
<url>https://zenodo.org/record/4218562</url>
|
||||||
</webresource>
|
</webresource>
|
||||||
|
|
|
@ -1,10 +1,8 @@
|
||||||
<record>
|
<record>
|
||||||
<result xmlns:dri="http://www.driver-repository.eu/namespace/dri" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
|
<result xmlns:dri="http://www.driver-repository.eu/namespace/dri" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
|
||||||
<header>
|
<header>
|
||||||
<dri:objIdentifier>r37b0ad08687::eb430fb7438e1533ba95d6aa50a477eb</dri:objIdentifier>
|
<dri:objIdentifier>doi_dedup___::8539a8de8996e01350f0de8ca4899b7f</dri:objIdentifier>
|
||||||
<dri:dateOfCollection>2021-09-22T08:53:13Z</dri:dateOfCollection>
|
<dri:dateOfCollection>2021-09-22T08:53:13Z</dri:dateOfCollection>
|
||||||
<dri:status>under curation</dri:status>
|
|
||||||
<counters />
|
|
||||||
</header>
|
</header>
|
||||||
<metadata>
|
<metadata>
|
||||||
<oaf:entity xmlns:oaf="http://namespace.openaire.eu/oaf"
|
<oaf:entity xmlns:oaf="http://namespace.openaire.eu/oaf"
|
||||||
|
@ -13,10 +11,10 @@
|
||||||
<oaf:result>
|
<oaf:result>
|
||||||
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title">EGI-Foundation/training-notebooks-seadatanet: Version 0.4</title>
|
<title classid="main title" classname="main title" schemeid="dnet:dataCite_title" schemename="dnet:dataCite_title">EGI-Foundation/training-notebooks-seadatanet: Version 0.4</title>
|
||||||
<creator rank="1" name="" surname="">Enol Fernández</creator>
|
<creator rank="1" name="" surname="">Enol Fernández</creator>
|
||||||
<dateofacceptance />
|
<dateofacceptance>2019-12-04</dateofacceptance>
|
||||||
<resulttype classid="software" classname="software" schemeid="dnet:result_typologies" schemename="dnet:result_typologies" />
|
<resulttype classid="software" classname="software" schemeid="dnet:result_typologies" schemename="dnet:result_typologies" />
|
||||||
<language classid="" classname="" schemeid="dnet:languages" schemename="dnet:languages" />
|
<language classid="" classname="" schemeid="dnet:languages" schemename="dnet:languages" />
|
||||||
<description>A sample notebook using SeaDataNet data to plot a map that shows surface temperature of Black Sea, Arctic Sea and Baltic Sea. The data is available at EGI DataHub with PID http://hdl.handle.net/21.T15999/qVk6JWQ (run at EGI Notebooks service for easy access to data).This release updates the PID for the data.</description>
|
<description>A sample notebook using SeaDataNet data to plot a map that shows surface temperature of Black Sea, Arctic Sea and Baltic Sea. The data is available at EGI DataHub with PID http://hdl.handle.net/21.T15999/3Byz9Cw (run at EGI Notebooks service for easy access to data). This release uses the correct path of the data share from the EGI DataHub.</description>
|
||||||
<country classid="" classname="" schemeid="" schemename="" />
|
<country classid="" classname="" schemeid="" schemename="" />
|
||||||
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies">EOSC Jupyter Notebook</subject>
|
<subject classid="keyword" classname="keyword" schemeid="dnet:subject_classification_typologies" schemename="dnet:subject_classification_typologies">EOSC Jupyter Notebook</subject>
|
||||||
<relevantdate classid="" classname="" schemeid="" schemename="" />
|
<relevantdate classid="" classname="" schemeid="" schemename="" />
|
||||||
|
@ -43,6 +41,9 @@
|
||||||
<collectedfrom name="Zenodo" id="re3data_____::7b0ad08687b2c960d5aeef06f811d5e6" />
|
<collectedfrom name="Zenodo" id="re3data_____::7b0ad08687b2c960d5aeef06f811d5e6" />
|
||||||
<pid classid="oai" classname="Open Archives Initiative" schemeid="dnet:pid_types" schemename="dnet:pid_types">oai:zenodo.org:3561323</pid>
|
<pid classid="oai" classname="Open Archives Initiative" schemeid="dnet:pid_types" schemename="dnet:pid_types">oai:zenodo.org:3561323</pid>
|
||||||
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types">10.5281/zenodo.3561323</pid>
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types">10.5281/zenodo.3561323</pid>
|
||||||
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">10.5281/zenodo.3443996</pid>
|
||||||
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">10.5281/zenodo.3475539</pid>
|
||||||
|
<pid classid="doi" classname="Digital Object Identifier" schemeid="dnet:pid_types" schemename="dnet:pid_types" inferred="false" provenanceaction="sysimport:actionset" trust="0.9">10.5281/zenodo.3475785</pid>
|
||||||
<bestaccessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes" schemename="dnet:access_modes" />
|
<bestaccessright classid="OPEN" classname="Open Access" schemeid="dnet:access_modes" schemename="dnet:access_modes" />
|
||||||
<datainfo>
|
<datainfo>
|
||||||
<inferred>false</inferred>
|
<inferred>false</inferred>
|
||||||
|
|
|
@ -13,4 +13,4 @@ export SHADOW=$3
|
||||||
echo "Creating observatory database"
|
echo "Creating observatory database"
|
||||||
impala-shell -q "drop database if exists ${TARGET} cascade"
|
impala-shell -q "drop database if exists ${TARGET} cascade"
|
||||||
impala-shell -q "create database if not exists ${TARGET}"
|
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 -
|
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 -
|
||||||
|
|
|
@ -233,4 +233,50 @@ on p.id= tmp.id;
|
||||||
|
|
||||||
create table indi_pub_has_abstract stored as parquet as
|
create table indi_pub_has_abstract stored as parquet as
|
||||||
select distinct publication.id, coalesce(abstract, 1) has_abstract
|
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
|
||||||
|
|
Loading…
Reference in New Issue