Merge branch 'beta' into beta_solr_config

This commit is contained in:
Claudio Atzori 2021-10-14 11:08:44 +02:00
commit 14fbf92ad6
52 changed files with 3292 additions and 1095 deletions

View File

@ -60,14 +60,10 @@ object SparkCreateActionset {
val entities: Dataset[(String, Result)] = spark.read.load(s"$sourcePath/entities/*").as[Result].map(p => (p.getId, p))(Encoders.tuple(Encoders.STRING, resultEncoders)) val entities: Dataset[(String, Result)] = spark.read.load(s"$sourcePath/entities/*").as[Result].map(p => (p.getId, p))(Encoders.tuple(Encoders.STRING, resultEncoders))
entities.filter(r => r.isInstanceOf[Result]).map(r => r.asInstanceOf[Result])
entities entities
.joinWith(idRelation, entities("_1").equalTo(idRelation("value"))) .joinWith(idRelation, entities("_1").equalTo(idRelation("value")))
.map(p => p._1._2) .map(p => p._1._2)
.write.mode(SaveMode.Append).save(s"$workingDirFolder/actionSetOaf") .write.mode(SaveMode.Append).save(s"$workingDirFolder/actionSetOaf")
} }
} }

View File

@ -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")
} }

View File

@ -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)
} }
} }

View File

@ -0,0 +1,200 @@
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)
IOUtils.copy(i, fsDataOutputStream)
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 skipUpdate = parser.get("skipUpdate")
log.info("skipUpdate: {}", skipUpdate)
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])
if (!"true".equalsIgnoreCase(skipUpdate)) {
downloadBaseLineUpdate(s"$workingPath/baseline", hdfsServerUri)
val k: RDD[(String, String)] = sc.wholeTextFiles(s"$workingPath/baseline", 2000)
val ds: Dataset[PMArticle] = spark.createDataset(k.filter(i => i._1.endsWith(".gz")).flatMap(i => {
val xml = new XMLEventReader(Source.fromBytes(i._2.getBytes()))
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)
}
}

View File

@ -1,33 +1,28 @@
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.sx.graph.bio.BioDBToOAF.EBILinkItem import eu.dnetllib.dhp.sx.bio.BioDBToOAF.EBILinkItem
import eu.dnetlib.dhp.sx.graph.bio.pubmed.{PMArticle, PMAuthor, PMJournal} import eu.dnetllib.dhp.sx.bio.pubmed.{PMArticle, PMAuthor, PMJournal}
import org.apache.commons.io.IOUtils import org.apache.commons.io.IOUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.http.client.config.RequestConfig import org.apache.http.client.config.RequestConfig
import org.apache.http.client.methods.{HttpGet, HttpUriRequest} import org.apache.http.client.methods.HttpGet
import org.apache.http.impl.client.HttpClientBuilder import org.apache.http.impl.client.HttpClientBuilder
import org.apache.spark.SparkConf import org.apache.spark.SparkConf
import org.apache.spark.sql.expressions.Aggregator
import org.apache.spark.sql.functions.max import org.apache.spark.sql.functions.max
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession} import org.apache.spark.sql._
import org.slf4j.{Logger, LoggerFactory} import org.slf4j.{Logger, LoggerFactory}
object SparkDownloadEBILinks { object SparkDownloadEBILinks {
def createEBILinks(pmid: Long): EBILinkItem = {
def createEBILinks(pmid:Long):EBILinkItem = {
val res = requestLinks(pmid) val res = requestLinks(pmid)
if (res!=null) if (res != null)
return EBILinkItem(pmid, res) return EBILinkItem(pmid, res)
null null
} }
def requestPage(url: String): String = {
def requestPage(url:String):String = {
val r = new HttpGet(url) val r = new HttpGet(url)
val timeout = 60; // seconds val timeout = 60; // seconds
val config = RequestConfig.custom() val config = RequestConfig.custom()
@ -61,46 +56,11 @@ object SparkDownloadEBILinks {
} }
} }
def requestLinks(PMID: Long): String = {
def requestBaseLineUpdatePage():List[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") ).map(s => s"https://ftp.ncbi.nlm.nih.gov/pubmed/updatefiles/$s").toList
result
}
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)
while (files.hasNext) {
val c = files.next()
c.getPath
}
}
def requestLinks(PMID:Long):String = {
requestPage(s"https://www.ebi.ac.uk/europepmc/webservices/rest/MED/$PMID/datalinks?format=json") requestPage(s"https://www.ebi.ac.uk/europepmc/webservices/rest/MED/$PMID/datalinks?format=json")
} }
def main(args: Array[String]): Unit = { def main(args: Array[String]): Unit = {
val log: Logger = LoggerFactory.getLogger(getClass) val log: Logger = LoggerFactory.getLogger(getClass)
@ -117,9 +77,9 @@ object SparkDownloadEBILinks {
import spark.implicits._ import spark.implicits._
implicit val PMEncoder: Encoder[PMArticle] = Encoders.kryo(classOf[PMArticle]) implicit val PMEncoder: Encoder[PMArticle] = Encoders.kryo(classOf[PMArticle])
implicit val PMJEncoder: Encoder[PMJournal] = Encoders.kryo(classOf[PMJournal]) implicit val PMJEncoder: Encoder[PMJournal] = Encoders.kryo(classOf[PMJournal])
implicit val PMAEncoder: Encoder[PMAuthor] = Encoders.kryo(classOf[PMAuthor]) implicit val PMAEncoder: Encoder[PMAuthor] = Encoders.kryo(classOf[PMAuthor])
val sourcePath = parser.get("sourcePath") val sourcePath = parser.get("sourcePath")
log.info(s"sourcePath -> $sourcePath") log.info(s"sourcePath -> $sourcePath")
@ -127,29 +87,29 @@ object SparkDownloadEBILinks {
log.info(s"workingPath -> $workingPath") log.info(s"workingPath -> $workingPath")
log.info("Getting max pubmedId where the links have been requested") 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 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) val lastPMIDRequested = links.map(l => l.id).select(max("value")).first.getLong(0)
log.info("Retrieving PMID to request links") log.info("Retrieving PMID to request links")
val pubmed = spark.read.load(s"$sourcePath/baseline_dataset").as[PMArticle] 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") 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 pmidToReq: Dataset[Long] = spark.read.load(s"$workingPath/id_to_request").as[Long]
val total = pmidToReq.count() 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") 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] val updates: Dataset[EBILinkItem] = spark.read.load(s"$workingPath/links_update").as[EBILinkItem]
links.union(updates).groupByKey(_.id) links.union(updates).groupByKey(_.id)
.reduceGroups{(x,y) => .reduceGroups { (x, y) =>
if (x == null || x.links ==null) if (x == null || x.links == null)
y y
if (y ==null || y.links ==null) if (y == null || y.links == null)
x x
if (x.links.length > y.links.length) if (x.links.length > y.links.length)
x x
else else
y y
}.map(_._2).write.mode(SaveMode.Overwrite).save(s"$workingPath/links_final") }.map(_._2).write.mode(SaveMode.Overwrite).save(s"$workingPath/links_final")

View File

@ -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,17 +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 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))
ebLinks.flatMap(j =>BioDBToOAF.parse_ebi_links(j.links))
.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)

View File

@ -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;

View File

@ -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;

View File

@ -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 {

View File

@ -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;

View File

@ -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}

View File

@ -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;

View File

@ -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)

View File

@ -0,0 +1,8 @@
[
{"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":"s", "paramLongName":"skipUpdate", "paramDescription": "skip update ", "paramRequired": false},
{"paramName":"h", "paramLongName":"hdfsServerUri", "paramDescription": "the working path ", "paramRequired": true}
]

View File

@ -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}
]

View File

@ -1,17 +1,22 @@
<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>
</property> </property>
<property>
<name>targetPath</name>
<description>The target path</description>
</property>
<property>
<name>skipUpdate</name>
<value>false</value>
<description>The request block size</description>
</property>
</parameters> </parameters>
<start to="ConvertDataset"/> <start to="ConvertDataset"/>
@ -24,9 +29,9 @@
<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-aggregation-${projectVersion}.jar</jar>
<spark-opts> <spark-opts>
--executor-memory=${sparkExecutorMemory} --executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores} --executor-cores=${sparkExecutorCores}
@ -41,6 +46,8 @@
<arg>--targetPath</arg><arg>${targetPath}</arg> <arg>--targetPath</arg><arg>${targetPath}</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>
<arg>--skipUpdate</arg><arg>${skipUpdate}</arg>
</spark> </spark>
<ok to="End"/> <ok to="End"/>
<error to="Kill"/> <error to="Kill"/>

View File

@ -1,13 +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 eu.dnetlib.dhp.sx.graph.ebi.SparkDownloadEBILinks
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
@ -51,16 +48,11 @@ class BioScholixTest extends AbstractVocabularyTest{
} }
@Test
def testDownloadEBIUpdate() = {
val data = SparkDownloadEBILinks.requestBaseLineUpdatePage()
println(data)
}
@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)))
} }
@ -70,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)))

View File

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

View File

@ -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
}

View File

@ -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>

View File

@ -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
}
]

View File

@ -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

View File

@ -48,7 +48,7 @@
<!-- MAG Parameters --> <!-- MAG Parameters -->
<property> <property>
<name>inputPathMAG</name> <name>inputPathMAG</name>
<description>the MAG working path</description> <description>the MAG input path</description>
</property> </property>
@ -138,7 +138,7 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts> </spark-opts>
<arg>--sourcePath</arg><arg>${inputPathMAG}/dataset</arg> <arg>--sourcePath</arg><arg>${inputPathMAG}/dataset</arg>
<arg>--workingPath</arg><arg>${inputPathMAG}/process_p</arg> <arg>--workingPath</arg><arg>${workingPath}/MAG</arg>
<arg>--targetPath</arg><arg>${workingPath}</arg> <arg>--targetPath</arg><arg>${workingPath}</arg>
<arg>--master</arg><arg>yarn-cluster</arg> <arg>--master</arg><arg>yarn-cluster</arg>
</spark> </spark>

View File

@ -69,7 +69,7 @@ public class PropagationConstant {
PROPAGATION_DATA_INFO_TYPE, PROPAGATION_DATA_INFO_TYPE,
PROPAGATION_COUNTRY_INSTREPO_CLASS_ID, PROPAGATION_COUNTRY_INSTREPO_CLASS_ID,
PROPAGATION_COUNTRY_INSTREPO_CLASS_NAME, PROPAGATION_COUNTRY_INSTREPO_CLASS_NAME,
ModelConstants.DNET_PROVENANCE_ACTIONS)); ModelConstants.DNET_PROVENANCE_ACTIONS));
return nc; return nc;
} }
@ -84,7 +84,8 @@ public class PropagationConstant {
return di; return di;
} }
public static Qualifier getQualifier(String inference_class_id, String inference_class_name, String qualifierSchema) { public static Qualifier getQualifier(String inference_class_id, String inference_class_name,
String qualifierSchema) {
Qualifier pa = new Qualifier(); Qualifier pa = new Qualifier();
pa.setClassid(inference_class_id); pa.setClassid(inference_class_id);
pa.setClassname(inference_class_name); pa.setClassname(inference_class_name);
@ -108,7 +109,11 @@ public class PropagationConstant {
r.setRelClass(rel_class); r.setRelClass(rel_class);
r.setRelType(rel_type); r.setRelType(rel_type);
r.setSubRelType(subrel_type); r.setSubRelType(subrel_type);
r.setDataInfo(getDataInfo(inference_provenance, inference_class_id, inference_class_name, ModelConstants.DNET_PROVENANCE_ACTIONS)); r
.setDataInfo(
getDataInfo(
inference_provenance, inference_class_id, inference_class_name,
ModelConstants.DNET_PROVENANCE_ACTIONS));
return r; return r;
} }

View File

@ -173,14 +173,17 @@ public class SparkOrcidToResultFromSemRelJob {
if (toaddpid) { if (toaddpid) {
StructuredProperty p = new StructuredProperty(); StructuredProperty p = new StructuredProperty();
p.setValue(autoritative_author.getOrcid()); p.setValue(autoritative_author.getOrcid());
p.setQualifier(getQualifier(ModelConstants.ORCID_PENDING, ModelConstants.ORCID_CLASSNAME, ModelConstants.DNET_PID_TYPES)); p
.setQualifier(
getQualifier(
ModelConstants.ORCID_PENDING, ModelConstants.ORCID_CLASSNAME, ModelConstants.DNET_PID_TYPES));
p p
.setDataInfo( .setDataInfo(
getDataInfo( getDataInfo(
PROPAGATION_DATA_INFO_TYPE, PROPAGATION_DATA_INFO_TYPE,
PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID,
PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME,
ModelConstants.DNET_PROVENANCE_ACTIONS)); ModelConstants.DNET_PROVENANCE_ACTIONS));
Optional<List<StructuredProperty>> authorPid = Optional.ofNullable(author.getPid()); Optional<List<StructuredProperty>> authorPid = Optional.ofNullable(author.getPid());
if (authorPid.isPresent()) { if (authorPid.isPresent()) {

View File

@ -10,7 +10,6 @@ import java.util.List;
import java.util.Optional; import java.util.Optional;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import eu.dnetlib.dhp.schema.common.ModelConstants;
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.function.MapFunction; import org.apache.spark.api.java.function.MapFunction;
@ -22,6 +21,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.Context; import eu.dnetlib.dhp.schema.oaf.Context;
import eu.dnetlib.dhp.schema.oaf.Result; import eu.dnetlib.dhp.schema.oaf.Result;
import scala.Tuple2; import scala.Tuple2;
@ -130,7 +130,7 @@ public class SparkResultToCommunityFromOrganizationJob {
PROPAGATION_DATA_INFO_TYPE, PROPAGATION_DATA_INFO_TYPE,
PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID,
PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME, PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME,
ModelConstants.DNET_PROVENANCE_ACTIONS))); ModelConstants.DNET_PROVENANCE_ACTIONS)));
propagatedContexts.add(newContext); propagatedContexts.add(newContext);
} }
} }

View File

@ -7,7 +7,6 @@ import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession;
import java.util.*; import java.util.*;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import eu.dnetlib.dhp.schema.common.ModelConstants;
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.function.MapFunction; import org.apache.spark.api.java.function.MapFunction;
@ -20,6 +19,7 @@ import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.oaf.*;
import scala.Tuple2; import scala.Tuple2;
@ -126,7 +126,7 @@ public class SparkResultToCommunityThroughSemRelJob {
PROPAGATION_DATA_INFO_TYPE, PROPAGATION_DATA_INFO_TYPE,
PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID,
PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME,
ModelConstants.DNET_PROVENANCE_ACTIONS))); ModelConstants.DNET_PROVENANCE_ACTIONS)));
return newContext; return newContext;
} }
return null; return null;

View File

@ -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"
}
}

View File

@ -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}

View File

@ -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}
]

View File

@ -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}
]

View File

@ -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"/>

View File

@ -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>

View File

@ -84,7 +84,28 @@ public class IndexRecordTransformerTest {
@Test @Test
public void testForEOSCFutureTraining() throws IOException, TransformerException { public void testForEOSCFutureTraining() throws IOException, TransformerException {
final String record = IOUtils.toString(getClass().getResourceAsStream("eosc-future/training-notebooks-seadatanet.xml")); final String record = IOUtils
.toString(getClass().getResourceAsStream("eosc-future/training-notebooks-seadatanet.xml"));
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); testRecordTransformation(record);
} }

View File

@ -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>

View File

@ -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 &amp; 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>

View File

@ -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 &amp; 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>

View File

@ -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>

View File

@ -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>

View File

@ -11,9 +11,13 @@ TARGET_DB=$2
TMP=/tmp/stats-update-`tr -dc A-Za-z0-9 </dev/urandom | head -c 6` TMP=/tmp/stats-update-`tr -dc A-Za-z0-9 </dev/urandom | head -c 6`
echo "Downloading context data" echo "Downloading context ids"
curl -L ${CONTEXT_API}/contexts/?type=ri,community -H "accept: application/json" | /usr/local/sbin/jq -r '.[] | "\(.id),\(.label)"' > contexts.csv curl -L ${CONTEXT_API}/contexts/?type=ri,community -H "accept: application/json" | /usr/local/sbin/jq -r '.[] | "\(.id),\(.label)"' > contexts.csv
echo "Downloading categories data"
cat contexts.csv | cut -d , -f1 | xargs -I {} curl -L ${CONTEXT_API}/context/{}/?all=true | /usr/local/sbin/jq -r '.[]|"\(.id|split(":")[0]),\(.id),\(.label)"' > categories.csv cat contexts.csv | cut -d , -f1 | xargs -I {} curl -L ${CONTEXT_API}/context/{}/?all=true | /usr/local/sbin/jq -r '.[]|"\(.id|split(":")[0]),\(.id),\(.label)"' > categories.csv
echo "Downloading concepts data"
cat categories.csv | cut -d , -f2 | sed 's/:/%3A/g'| xargs -I {} curl -L ${CONTEXT_API}/context/category/{}/?all=true | /usr/local/sbin/jq -r '.[]|"\(.id|split("::")[0])::\(.id|split("::")[1]),\(.id),\(.label)"' > concepts.csv cat categories.csv | cut -d , -f2 | sed 's/:/%3A/g'| xargs -I {} curl -L ${CONTEXT_API}/context/category/{}/?all=true | /usr/local/sbin/jq -r '.[]|"\(.id|split("::")[0])::\(.id|split("::")[1]),\(.id),\(.label)"' > concepts.csv
cat contexts.csv | sed 's/^\(.*\),\(.*\)/\1,\1::other,\2/' >> categories.csv cat contexts.csv | sed 's/^\(.*\),\(.*\)/\1,\1::other,\2/' >> categories.csv
cat categories.csv | sed 's/^.*,\(.*\),\(.*\)/\1,\1::other,\2/' >> concepts.csv cat categories.csv | sed 's/^.*,\(.*\),\(.*\)/\1,\1::other,\2/' >> concepts.csv

View File

@ -753,7 +753,7 @@
<mockito-core.version>3.3.3</mockito-core.version> <mockito-core.version>3.3.3</mockito-core.version>
<mongodb.driver.version>3.4.2</mongodb.driver.version> <mongodb.driver.version>3.4.2</mongodb.driver.version>
<vtd.version>[2.12,3.0)</vtd.version> <vtd.version>[2.12,3.0)</vtd.version>
<dhp-schemas.version>[2.7.18]</dhp-schemas.version> <dhp-schemas.version>[2.8.20]</dhp-schemas.version>
<dnet-actionmanager-api.version>[4.0.3]</dnet-actionmanager-api.version> <dnet-actionmanager-api.version>[4.0.3]</dnet-actionmanager-api.version>
<dnet-actionmanager-common.version>[6.0.5]</dnet-actionmanager-common.version> <dnet-actionmanager-common.version>[6.0.5]</dnet-actionmanager-common.version>
<dnet-openaire-broker-common.version>[3.1.6]</dnet-openaire-broker-common.version> <dnet-openaire-broker-common.version>[3.1.6]</dnet-openaire-broker-common.version>