[Dump] resolved conflicts with beta and merging

This commit is contained in:
Miriam Baglioni 2021-12-14 15:03:45 +01:00
commit 56409d1281
85 changed files with 3818 additions and 894 deletions

2
.gitignore vendored
View File

@ -3,8 +3,6 @@
*.iws
*.ipr
*.iml
*.ipr
*.iws
*~
.vscode
.metals

View File

@ -57,9 +57,17 @@ public class VocabularyGroup implements Serializable {
final String syn = arr[2].trim();
vocs.addSynonyms(vocId, termId, syn);
}
}
// add the term names as synonyms
vocs.vocs.values().forEach(voc -> {
voc.getTerms().values().forEach(term -> {
voc.addSynonym(term.getName().toLowerCase(), term.getId());
});
});
return vocs;
}

View File

@ -16,6 +16,8 @@ import com.github.sisyphsu.dateparser.DateParserUtils;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import eu.dnetlib.dhp.common.vocabulary.Vocabulary;
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
@ -86,6 +88,22 @@ public class GraphCleaningFunctions extends CleaningFunctions {
}
public static <T extends Oaf> boolean filter(T value) {
if (Boolean.TRUE
.equals(
Optional
.ofNullable(value)
.map(
o -> Optional
.ofNullable(o.getDataInfo())
.map(
d -> Optional
.ofNullable(d.getInvisible())
.orElse(true))
.orElse(true))
.orElse(true))) {
return false;
}
if (value instanceof Datasource) {
// nothing to evaluate here
} else if (value instanceof Project) {
@ -115,7 +133,7 @@ public class GraphCleaningFunctions extends CleaningFunctions {
return true;
}
public static <T extends Oaf> T cleanup(T value) {
public static <T extends Oaf> T cleanup(T value, VocabularyGroup vocs) {
if (value instanceof Datasource) {
// nothing to clean here
} else if (value instanceof Project) {
@ -212,6 +230,15 @@ public class GraphCleaningFunctions extends CleaningFunctions {
.map(GraphCleaningFunctions::cleanValue)
.collect(Collectors.toList()));
}
if (Objects.nonNull(r.getFormat())) {
r
.setFormat(
r
.getFormat()
.stream()
.map(GraphCleaningFunctions::cleanValue)
.collect(Collectors.toList()));
}
if (Objects.nonNull(r.getDescription())) {
r
.setDescription(
@ -234,6 +261,38 @@ public class GraphCleaningFunctions extends CleaningFunctions {
if (Objects.nonNull(r.getInstance())) {
for (Instance i : r.getInstance()) {
if (!vocs.termExists(ModelConstants.DNET_PUBLICATION_RESOURCE, i.getInstancetype().getClassid())) {
if (r instanceof Publication) {
i
.setInstancetype(
OafMapperUtils
.qualifier(
"0038", "Other literature type", ModelConstants.DNET_PUBLICATION_RESOURCE,
ModelConstants.DNET_PUBLICATION_RESOURCE));
} else if (r instanceof Dataset) {
i
.setInstancetype(
OafMapperUtils
.qualifier(
"0039", "Other dataset type", ModelConstants.DNET_PUBLICATION_RESOURCE,
ModelConstants.DNET_PUBLICATION_RESOURCE));
} else if (r instanceof Software) {
i
.setInstancetype(
OafMapperUtils
.qualifier(
"0040", "Other software type", ModelConstants.DNET_PUBLICATION_RESOURCE,
ModelConstants.DNET_PUBLICATION_RESOURCE));
} else if (r instanceof OtherResearchProduct) {
i
.setInstancetype(
OafMapperUtils
.qualifier(
"0020", "Other ORP type", ModelConstants.DNET_PUBLICATION_RESOURCE,
ModelConstants.DNET_PUBLICATION_RESOURCE));
}
}
if (Objects.nonNull(i.getPid())) {
i.setPid(processPidCleaning(i.getPid()));
}

View File

@ -107,7 +107,7 @@ class OafMapperUtilsTest {
assertEquals("2006-01-02", GraphCleaningFunctions.doCleanDate("2006-01-02T15:04:05+0000").get());
assertEquals("2009-08-13", GraphCleaningFunctions.doCleanDate("2009-08-12T22:15:09-07:00").get());
assertEquals("2009-08-12", GraphCleaningFunctions.doCleanDate("2009-08-12T22:15:09").get());
assertEquals("2009-08-12", GraphCleaningFunctions.doCleanDate("2009-08-12T22:15:09Z").get());
assertEquals("2009-08-13", GraphCleaningFunctions.doCleanDate("2009-08-12T22:15:09Z").get());
assertEquals("2014-04-26", GraphCleaningFunctions.doCleanDate("2014-04-26 17:24:37.3186369").get());
assertEquals("2012-08-03", GraphCleaningFunctions.doCleanDate("2012-08-03 18:31:59.257000000").get());
assertEquals("2014-04-26", GraphCleaningFunctions.doCleanDate("2014-04-26 17:24:37.123").get());

View File

@ -1,69 +0,0 @@
package eu.dnetlib.dhp.actionmanager.scholix
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.schema.oaf.{Oaf, Relation, Result}
import org.apache.spark.SparkConf
import org.apache.spark.sql._
import org.slf4j.{Logger, LoggerFactory}
import scala.io.Source
object SparkCreateActionset {
def main(args: Array[String]): Unit = {
val log: Logger = LoggerFactory.getLogger(getClass)
val conf: SparkConf = new SparkConf()
val parser = new ArgumentApplicationParser(Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/actionset/generate_actionset.json")).mkString)
parser.parseArgument(args)
val spark: SparkSession =
SparkSession
.builder()
.config(conf)
.appName(getClass.getSimpleName)
.master(parser.get("master")).getOrCreate()
val sourcePath = parser.get("sourcePath")
log.info(s"sourcePath -> $sourcePath")
val targetPath = parser.get("targetPath")
log.info(s"targetPath -> $targetPath")
val workingDirFolder = parser.get("workingDirFolder")
log.info(s"workingDirFolder -> $workingDirFolder")
implicit val oafEncoders: Encoder[Oaf] = Encoders.kryo[Oaf]
implicit val resultEncoders: Encoder[Result] = Encoders.kryo[Result]
implicit val relationEncoders: Encoder[Relation] = Encoders.kryo[Relation]
import spark.implicits._
val relation = spark.read.load(s"$sourcePath/relation").as[Relation]
relation.filter(r => (r.getDataInfo == null || r.getDataInfo.getDeletedbyinference == false) && !r.getRelClass.toLowerCase.contains("merge"))
.flatMap(r => List(r.getSource, r.getTarget)).distinct().write.mode(SaveMode.Overwrite).save(s"$workingDirFolder/id_relation")
val idRelation = spark.read.load(s"$workingDirFolder/id_relation").as[String]
log.info("extract source and target Identifier involved in relations")
log.info("save relation filtered")
relation.filter(r => (r.getDataInfo == null || r.getDataInfo.getDeletedbyinference == false) && !r.getRelClass.toLowerCase.contains("merge"))
.write.mode(SaveMode.Overwrite).save(s"$workingDirFolder/actionSetOaf")
log.info("saving entities")
val entities: Dataset[(String, Result)] = spark.read.load(s"$sourcePath/entities/*").as[Result].map(p => (p.getId, p))(Encoders.tuple(Encoders.STRING, resultEncoders))
entities
.joinWith(idRelation, entities("_1").equalTo(idRelation("value")))
.map(p => p._1._2)
.write.mode(SaveMode.Append).save(s"$workingDirFolder/actionSetOaf")
}
}

View File

@ -1,86 +0,0 @@
package eu.dnetlib.dhp.actionmanager.scholix
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.schema.action.AtomicAction
import eu.dnetlib.dhp.schema.oaf.{Oaf, Dataset => OafDataset,Publication, Software, OtherResearchProduct, Relation}
import org.apache.hadoop.io.Text
import org.apache.hadoop.io.compress.GzipCodec
import org.apache.hadoop.mapred.SequenceFileOutputFormat
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Encoder, Encoders, SparkSession}
import org.slf4j.{Logger, LoggerFactory}
import scala.io.Source
object SparkSaveActionSet {
def toActionSet(item: Oaf): (String, String) = {
val mapper = new ObjectMapper()
item match {
case dataset: OafDataset =>
val a: AtomicAction[OafDataset] = new AtomicAction[OafDataset]
a.setClazz(classOf[OafDataset])
a.setPayload(dataset)
(dataset.getClass.getCanonicalName, mapper.writeValueAsString(a))
case publication: Publication =>
val a: AtomicAction[Publication] = new AtomicAction[Publication]
a.setClazz(classOf[Publication])
a.setPayload(publication)
(publication.getClass.getCanonicalName, mapper.writeValueAsString(a))
case software: Software =>
val a: AtomicAction[Software] = new AtomicAction[Software]
a.setClazz(classOf[Software])
a.setPayload(software)
(software.getClass.getCanonicalName, mapper.writeValueAsString(a))
case orp: OtherResearchProduct =>
val a: AtomicAction[OtherResearchProduct] = new AtomicAction[OtherResearchProduct]
a.setClazz(classOf[OtherResearchProduct])
a.setPayload(orp)
(orp.getClass.getCanonicalName, mapper.writeValueAsString(a))
case relation: Relation =>
val a: AtomicAction[Relation] = new AtomicAction[Relation]
a.setClazz(classOf[Relation])
a.setPayload(relation)
(relation.getClass.getCanonicalName, mapper.writeValueAsString(a))
case _ =>
null
}
}
def main(args: Array[String]): Unit = {
val log: Logger = LoggerFactory.getLogger(getClass)
val conf: SparkConf = new SparkConf()
val parser = new ArgumentApplicationParser(Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/actionset/save_actionset.json")).mkString)
parser.parseArgument(args)
val spark: SparkSession =
SparkSession
.builder()
.config(conf)
.appName(getClass.getSimpleName)
.master(parser.get("master")).getOrCreate()
val sourcePath = parser.get("sourcePath")
log.info(s"sourcePath -> $sourcePath")
val targetPath = parser.get("targetPath")
log.info(s"targetPath -> $targetPath")
implicit val oafEncoders: Encoder[Oaf] = Encoders.kryo[Oaf]
implicit val tEncoder: Encoder[(String, String)] = Encoders.tuple(Encoders.STRING, Encoders.STRING)
spark.read.load(sourcePath).as[Oaf]
.map(o => toActionSet(o))
.filter(o => o != null)
.rdd.map(s => (new Text(s._1), new Text(s._2))).saveAsHadoopFile(s"$targetPath", classOf[Text], classOf[Text], classOf[SequenceFileOutputFormat[Text, Text]], classOf[GzipCodec])
}
}

View File

@ -1,8 +1,8 @@
package eu.dnetlib.dhp.datacite
import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf.{DataInfo, KeyValue}
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils
import eu.dnetlib.dhp.schema.oaf.{DataInfo, KeyValue}
import java.io.InputStream
import java.time.format.DateTimeFormatter

View File

@ -6,7 +6,7 @@ import eu.dnetlib.dhp.datacite.DataciteModelConstants._
import eu.dnetlib.dhp.schema.action.AtomicAction
import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf.utils.{IdentifierFactory, OafMapperUtils}
import eu.dnetlib.dhp.schema.oaf.{AccessRight, Author, DataInfo, Instance, KeyValue, Oaf, OtherResearchProduct, Publication, Qualifier, Relation, Result, Software, StructuredProperty, Dataset => OafDataset}
import eu.dnetlib.dhp.schema.oaf.{Dataset => OafDataset, _}
import eu.dnetlib.dhp.utils.DHPUtils
import org.apache.commons.lang3.StringUtils
import org.json4s.DefaultFormats
@ -29,6 +29,7 @@ object DataciteToOAFTransformation {
/**
* This method should skip record if json contains invalid text
* defined in gile datacite_filter
*
* @param json
* @return True if the record should be skipped
*/

View File

@ -7,6 +7,7 @@ import org.json4s.DefaultFormats
import org.json4s.JsonAST.{JField, JObject, JString}
import org.json4s.jackson.JsonMethods.{compact, parse, render}
import collection.JavaConverters._
object BioDBToOAF {
case class EBILinkItem(id: Long, links: String) {}

View File

@ -1,9 +1,9 @@
package eu.dnetlib.dhp.sx.bio
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.schema.oaf.Oaf
import BioDBToOAF.ScholixResolved
import eu.dnetlib.dhp.collection.CollectionUtils
import eu.dnetlib.dhp.schema.oaf.Oaf
import eu.dnetlib.dhp.sx.bio.BioDBToOAF.ScholixResolved
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}

View File

@ -3,7 +3,7 @@ package eu.dnetlib.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.sx.bio.pubmed.{PMArticle, PMAuthor, PMJournal, PMParser, PubMedToOaf}
import eu.dnetlib.dhp.sx.bio.pubmed._
import eu.dnetlib.dhp.utils.ISLookupClientFactory
import org.apache.commons.io.IOUtils
import org.apache.hadoop.conf.Configuration

View File

@ -1,9 +1,8 @@
package eu.dnetlib.dhp.sx.bio.ebi
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.sx.bio.pubmed.{PMArticle, PMAuthor, PMJournal}
import eu.dnetlib.dhp.sx.bio.BioDBToOAF.EBILinkItem
import eu.dnetlib.dhp.sx.bio.pubmed.PMJournal
import eu.dnetlib.dhp.sx.bio.pubmed.{PMArticle, PMAuthor, PMJournal}
import org.apache.commons.io.IOUtils
import org.apache.http.client.config.RequestConfig
import org.apache.http.client.methods.HttpGet

View File

@ -1,11 +1,10 @@
package eu.dnetlib.dhp.sx.bio.ebi
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.collection.CollectionUtils
import eu.dnetlib.dhp.schema.oaf.Oaf
import eu.dnetlib.dhp.sx.bio.BioDBToOAF
import eu.dnetlib.dhp.sx.bio.BioDBToOAF.EBILinkItem
import BioDBToOAF.EBILinkItem
import eu.dnetlib.dhp.collection.CollectionUtils
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql._

View File

@ -4,7 +4,7 @@ import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup
import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf.utils.{GraphCleaningFunctions, IdentifierFactory, OafMapperUtils, PidType}
import eu.dnetlib.dhp.schema.oaf._
import scala.collection.JavaConverters._
import collection.JavaConverters._
import java.util.regex.Pattern
@ -22,10 +22,10 @@ object PubMedToOaf {
val collectedFrom: KeyValue = OafMapperUtils.keyValue(ModelConstants.EUROPE_PUBMED_CENTRAL_ID, "Europe PubMed Central")
/**
* Cleaning the DOI Applying regex in order to
* remove doi starting with URL
*
* @param doi input DOI
* @return cleaned DOI
*/
@ -93,7 +93,6 @@ object PubMedToOaf {
* @param vocabularyName the input vocabulary name
* @param vocabularies all the vocabularies
* @param term the term to find
*
* @return the cleaned term value
*/
def getVocabularyTerm(vocabularyName: String, vocabularies: VocabularyGroup, term: String): Qualifier = {
@ -106,7 +105,6 @@ object PubMedToOaf {
/**
* Map the Pubmed Article into the OAF instance
*
*
* @param article the pubmed articles
* @param vocabularies the vocabularies
* @return The OAF instance if the mapping did not fail
@ -185,7 +183,6 @@ object PubMedToOaf {
//--------------------------------------------------------------------------------------
// RESULT MAPPING
//--------------------------------------------------------------------------------------
result.setDateofacceptance(OafMapperUtils.field(GraphCleaningFunctions.cleanDate(article.getDate), dataInfo))

View File

@ -89,13 +89,13 @@ public class CreateOpenCitationsASTest {
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet"
workingDir.toString() + "/actionSet1"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet", Text.class, Text.class)
.sequenceFile(workingDir.toString() + "/actionSet1", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Relation) aa.getPayload()));
@ -121,13 +121,13 @@ public class CreateOpenCitationsASTest {
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet"
workingDir.toString() + "/actionSet2"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet", Text.class, Text.class)
.sequenceFile(workingDir.toString() + "/actionSet2", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Relation) aa.getPayload()));
@ -153,13 +153,13 @@ public class CreateOpenCitationsASTest {
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet"
workingDir.toString() + "/actionSet3"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet", Text.class, Text.class)
.sequenceFile(workingDir.toString() + "/actionSet3", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Relation) aa.getPayload()));
@ -186,13 +186,13 @@ public class CreateOpenCitationsASTest {
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet"
workingDir.toString() + "/actionSet4"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet", Text.class, Text.class)
.sequenceFile(workingDir.toString() + "/actionSet4", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Relation) aa.getPayload()));
@ -226,13 +226,13 @@ public class CreateOpenCitationsASTest {
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet"
workingDir.toString() + "/actionSet5"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet", Text.class, Text.class)
.sequenceFile(workingDir.toString() + "/actionSet5", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Relation) aa.getPayload()));
@ -261,13 +261,13 @@ public class CreateOpenCitationsASTest {
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet"
workingDir.toString() + "/actionSet6"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet", Text.class, Text.class)
.sequenceFile(workingDir.toString() + "/actionSet6", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Relation) aa.getPayload()));
@ -306,13 +306,13 @@ public class CreateOpenCitationsASTest {
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet"
workingDir.toString() + "/actionSet7"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet", Text.class, Text.class)
.sequenceFile(workingDir.toString() + "/actionSet7", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Relation) aa.getPayload()));

View File

@ -8,6 +8,7 @@ import org.apache.commons.io.FileUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql.functions.{col, count}
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.extension.ExtendWith
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.mockito.junit.jupiter.MockitoExtension
@ -17,7 +18,7 @@ import java.nio.file.{Files, Path}
import java.text.SimpleDateFormat
import java.util.Locale
import scala.io.Source
import org.junit.jupiter.api.Assertions._
@ExtendWith(Array(classOf[MockitoExtension]))
class DataciteToOAFTest extends AbstractVocabularyTest{

View File

@ -90,7 +90,7 @@ public class ReadBlacklistFromDB implements Closeable {
inverse.setSource(target_direct);
String encoding = rs.getString("relationship");
RelationInverse ri = ModelSupport.relationInverseMap.get(encoding);
RelationInverse ri = ModelSupport.findInverse(encoding);
direct.setRelClass(ri.getRelClass());
inverse.setRelClass(ri.getInverseRelClass());
direct.setRelType(ri.getRelType());

View File

@ -1,21 +1,19 @@
package eu.dnetlib.doiboost
import java.time.LocalDate
import java.time.format.DateTimeFormatter
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.schema.action.AtomicAction
import eu.dnetlib.dhp.schema.oaf.{AccessRight, DataInfo, Dataset, Field, Instance, KeyValue, Oaf, OpenAccessRoute, Organization, Publication, Qualifier, Relation, Result, StructuredProperty}
import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf._
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils
import eu.dnetlib.dhp.utils.DHPUtils
import org.apache.commons.lang3.StringUtils
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils
import eu.dnetlib.doiboost.DoiBoostMappingUtil.{getClosedAccessQualifier, getEmbargoedAccessQualifier, getUnknownQualifier}
import org.json4s
import org.json4s.DefaultFormats
import org.json4s.jackson.JsonMethods.parse
import org.slf4j.{Logger, LoggerFactory}
import java.time.LocalDate
import java.time.format.DateTimeFormatter
import scala.collection.JavaConverters._

View File

@ -8,11 +8,12 @@ import org.apache.hadoop.io.Text
import org.apache.hadoop.io.compress.GzipCodec
import org.apache.hadoop.mapred.SequenceFileOutputFormat
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession}
import org.slf4j.{Logger, LoggerFactory}
object SparkGenerateDOIBoostActionSet {
val logger: Logger = LoggerFactory.getLogger(getClass)
def main(args: Array[String]): Unit = {
val conf: SparkConf = new SparkConf()
@ -55,7 +56,6 @@ object SparkGenerateDOIBoostActionSet {
.map(d => DoiBoostMappingUtil.toActionSet(d))(Encoders.tuple(Encoders.STRING, Encoders.STRING))
val asCRelation = spark.read.load(crossRefRelation).as[Relation]
.filter(r => r != null && r.getSource != null && r.getTarget != null)
.map(d => DoiBoostMappingUtil.toActionSet(d))(Encoders.tuple(Encoders.STRING, Encoders.STRING))
@ -65,23 +65,12 @@ object SparkGenerateDOIBoostActionSet {
.map(d => DoiBoostMappingUtil.toActionSet(d))(Encoders.tuple(Encoders.STRING, Encoders.STRING))
val d: Dataset[(String, String)] = asDataset.union(asPublication).union(asOrganization).union(asCRelation).union(asRelAffiliation)
d.rdd.repartition(6000).map(s => (new Text(s._1), new Text(s._2))).saveAsHadoopFile(s"$sequenceFilePath", classOf[Text], classOf[Text], classOf[SequenceFileOutputFormat[Text, Text]], classOf[GzipCodec])
}
}

View File

@ -9,14 +9,12 @@ import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql.expressions.Aggregator
import org.apache.spark.sql.functions.col
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
import org.slf4j.{Logger, LoggerFactory}
import scala.collection.JavaConverters._
import org.apache.spark.sql._
import org.json4s.DefaultFormats
import org.json4s.JsonAST.{JField, JObject, JString,JArray}
import org.json4s.JsonAST.{JField, JObject, JString}
import org.json4s.jackson.JsonMethods.parse
import org.slf4j.{Logger, LoggerFactory}
import scala.collection.JavaConverters._
object SparkGenerateDoiBoost {
@ -38,7 +36,6 @@ object SparkGenerateDoiBoost {
}
def main(args: Array[String]): Unit = {
val logger: Logger = LoggerFactory.getLogger(getClass)
@ -118,8 +115,7 @@ object SparkGenerateDoiBoost {
val crossrefPublication: Dataset[(String, Publication)] = spark.read.load(s"$workingDirPath/crossrefPublication").as[Publication].map(p => (p.getId, p))
val uwPublication: Dataset[(String, Publication)] = spark.read.load(s"$workingDirPath/uwPublication").as[Publication].map(p => (p.getId, p))
def applyMerge(item:((String, Publication), (String, Publication))) : Publication =
{
def applyMerge(item: ((String, Publication), (String, Publication))): Publication = {
val crossrefPub = item._1._2
if (item._2 != null) {
val otherPub = item._2._2
@ -130,6 +126,7 @@ object SparkGenerateDoiBoost {
}
crossrefPub
}
crossrefPublication.joinWith(uwPublication, crossrefPublication("_1").equalTo(uwPublication("_1")), "left").map(applyMerge).write.mode(SaveMode.Overwrite).save(s"$workingDirPath/firstJoin")
logger.info("Phase 3) Join Result with ORCID")
val fj: Dataset[(String, Publication)] = spark.read.load(s"$workingDirPath/firstJoin").as[Publication].map(p => (p.getId, p))
@ -169,7 +166,6 @@ object SparkGenerateDoiBoost {
.select(col("_1.PaperId"), col("_2.AffiliationId"), col("_2.GridId"), col("_2.OfficialPage"), col("_2.DisplayName")).as[DoiBoostAffiliation]
val magPubs: Dataset[(String, Publication)] = spark.read.load(s"$workingDirPath/doiBoostPublicationFiltered").as[Publication]
.map(p => (ConversionUtil.extractMagIdentifier(p.getOriginalId.asScala), p))(tupleForJoinEncoder).filter(s => s._1 != null)
@ -198,8 +194,6 @@ object SparkGenerateDoiBoost {
})(mapEncoderRel).write.mode(SaveMode.Overwrite).save(s"$workingDirPath/doiBoostPublicationAffiliation_unresolved")
val unresolvedRels: Dataset[(String, Relation)] = spark.read.load(s"$workingDirPath/doiBoostPublicationAffiliation_unresolved").as[Relation].map(r => {
if (r.getSource.startsWith("unresolved"))

View File

@ -4,20 +4,19 @@ import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf._
import eu.dnetlib.dhp.schema.oaf.utils.{IdentifierFactory, OafMapperUtils}
import eu.dnetlib.dhp.utils.DHPUtils
import eu.dnetlib.doiboost.DoiBoostMappingUtil.{decideAccessRight, _}
import eu.dnetlib.doiboost.DoiBoostMappingUtil
import eu.dnetlib.doiboost.DoiBoostMappingUtil._
import org.apache.commons.lang.StringUtils
import org.json4s
import org.json4s.DefaultFormats
import org.json4s.JsonAST.{JValue, _}
import org.json4s.JsonAST._
import org.json4s.jackson.JsonMethods._
import org.slf4j.{Logger, LoggerFactory}
import java.util
import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.util.matching.Regex
import java.util
import eu.dnetlib.doiboost.DoiBoostMappingUtil
case class CrossrefDT(doi: String, json:String, timestamp: Long) {}

View File

@ -6,7 +6,7 @@ import org.apache.commons.io.IOUtils
import org.apache.hadoop.io.{IntWritable, Text}
import org.apache.spark.SparkConf
import org.apache.spark.sql.expressions.Aggregator
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
import org.apache.spark.sql.{Dataset, Encoder, SaveMode, SparkSession}
import org.json4s
import org.json4s.DefaultFormats
import org.json4s.jackson.JsonMethods.parse
@ -30,7 +30,6 @@ object CrossrefDataset {
def main(args: Array[String]): Unit = {
val conf: SparkConf = new SparkConf()
val parser = new ArgumentApplicationParser(IOUtils.toString(CrossrefDataset.getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/crossref_to_dataset_params.json")))
parser.parseArgument(args)

View File

@ -2,17 +2,12 @@ package eu.dnetlib.doiboost.crossref
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.doiboost.DoiBoostMappingUtil
import eu.dnetlib.doiboost.crossref.CrossrefDataset.to_item
import eu.dnetlib.doiboost.crossref.UnpackCrtossrefEntries.getClass
import org.apache.hadoop.io.{IntWritable, Text}
import org.apache.hadoop.io.compress.GzipCodec
import org.apache.spark.rdd.RDD
import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}
import org.apache.spark.{SparkConf, SparkContext}
import org.json4s
import org.json4s.DefaultFormats
import org.json4s.JsonAST.JArray
import org.json4s.jackson.JsonMethods.{compact, parse, render}
import org.json4s.jackson.JsonMethods.parse
import org.slf4j.{Logger, LoggerFactory}
import scala.io.Source
@ -24,7 +19,6 @@ object GenerateCrossrefDataset {
implicit val mrEncoder: Encoder[CrossrefDT] = Encoders.kryo[CrossrefDT]
def crossrefElement(meta: String): CrossrefDT = {
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
lazy val json: json4s.JValue = parse(meta)

View File

@ -4,10 +4,8 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.schema.oaf
import eu.dnetlib.dhp.schema.oaf.{Oaf, Publication, Relation, Dataset => OafDataset}
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
import org.apache.spark.sql._
import org.slf4j.{Logger, LoggerFactory}

View File

@ -2,8 +2,8 @@ package eu.dnetlib.doiboost.crossref
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import org.apache.hadoop.io.compress.GzipCodec
import org.apache.spark.sql.SparkSession
import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}
import org.json4s
import org.json4s.DefaultFormats
import org.json4s.JsonAST.JArray
@ -17,8 +17,6 @@ object UnpackCrtossrefEntries {
val log: Logger = LoggerFactory.getLogger(UnpackCrtossrefEntries.getClass)
def extractDump(input: String): List[String] = {
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
lazy val json: json4s.JValue = parse(input)
@ -30,7 +28,6 @@ object UnpackCrtossrefEntries {
}
def main(args: Array[String]): Unit = {
val conf = new SparkConf
val parser = new ArgumentApplicationParser(Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/crossref_dump_reader/generate_dataset_params.json")).mkString)

View File

@ -5,10 +5,10 @@ import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory
import eu.dnetlib.dhp.schema.oaf.{Instance, Journal, Publication, StructuredProperty}
import eu.dnetlib.doiboost.DoiBoostMappingUtil
import eu.dnetlib.doiboost.DoiBoostMappingUtil._
import org.json4s
import org.json4s.DefaultFormats
import org.json4s.jackson.JsonMethods.parse
import eu.dnetlib.doiboost.DoiBoostMappingUtil._
import scala.collection.JavaConverters._
import scala.collection.mutable

View File

@ -3,8 +3,8 @@ package eu.dnetlib.doiboost.mag
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql.{SaveMode, SparkSession}
import org.apache.spark.sql.types._
import org.apache.spark.sql.{SaveMode, SparkSession}
import org.slf4j.{Logger, LoggerFactory}
object SparkImportMagIntoDataset {
@ -75,7 +75,6 @@ object SparkImportMagIntoDataset {
.master(parser.get("master")).getOrCreate()
stream.foreach { case (k, v) =>
val s: StructType = getSchema(k)
val df = spark.read

View File

@ -5,13 +5,10 @@ import eu.dnetlib.dhp.schema.oaf.Publication
import eu.dnetlib.doiboost.DoiBoostMappingUtil
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.functions._
import org.apache.spark.sql.functions.{col, collect_list, struct}
import org.apache.spark.sql._
import org.slf4j.{Logger, LoggerFactory}
import scala.collection.JavaConverters._
object SparkProcessMAG {
def getDistinctResults(d: Dataset[MagPapers]): Dataset[MagPapers] = {
@ -20,7 +17,7 @@ object SparkProcessMAG {
.reduceGroups((p1: MagPapers, p2: MagPapers) => ConversionUtil.choiceLatestMagArtitcle(p1, p2))
.map(_._2)(Encoders.product[MagPapers])
.map(mp => {
new MagPapers(mp.PaperId, mp.Rank, DoiBoostMappingUtil.normalizeDoi(mp.Doi),
MagPapers(mp.PaperId, mp.Rank, DoiBoostMappingUtil.normalizeDoi(mp.Doi),
mp.DocType, mp.PaperTitle, mp.OriginalTitle,
mp.BookTitle, mp.Year, mp.Date, mp.Publisher: String,
mp.JournalId, mp.ConferenceSeriesId, mp.ConferenceInstanceId,
@ -153,6 +150,5 @@ object SparkProcessMAG {
.write.mode(SaveMode.Overwrite).save(s"$targetPath/magPublication")
}
}

View File

@ -4,17 +4,16 @@ import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory
import eu.dnetlib.dhp.schema.oaf.{Author, DataInfo, Publication}
import eu.dnetlib.dhp.schema.orcid.{AuthorData, OrcidDOI}
import eu.dnetlib.doiboost.DoiBoostMappingUtil
import eu.dnetlib.doiboost.DoiBoostMappingUtil.{createSP, generateDataInfo}
import org.apache.commons.lang.StringUtils
import org.slf4j.{Logger, LoggerFactory}
import scala.collection.JavaConverters._
import org.json4s
import org.json4s.DefaultFormats
import org.json4s.JsonAST._
import org.json4s.jackson.JsonMethods._
import org.slf4j.{Logger, LoggerFactory}
import scala.collection.JavaConverters._
case class ORCIDItem(doi:String, authors:List[OrcidAuthor]){}

View File

@ -1,15 +1,12 @@
package eu.dnetlib.doiboost.orcid
import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper}
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.oa.merge.AuthorMerger
import eu.dnetlib.dhp.schema.oaf.Publication
import eu.dnetlib.dhp.schema.orcid.OrcidDOI
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.functions._
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
import org.apache.spark.sql.functions.{col, collect_list}
import org.apache.spark.sql._
import org.slf4j.{Logger, LoggerFactory}
object SparkPreprocessORCID {

View File

@ -1,16 +1,14 @@
package eu.dnetlib.doiboost.uw
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.schema.oaf.Publication
import eu.dnetlib.doiboost.crossref.SparkMapDumpIntoOAF
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
import org.apache.spark.sql._
import org.slf4j.{Logger, LoggerFactory}
object SparkMapUnpayWallToOAF {
def main(args: Array[String]): Unit = {

View File

@ -4,14 +4,13 @@ import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory
import eu.dnetlib.dhp.schema.oaf.{AccessRight, Instance, OpenAccessRoute, Publication}
import eu.dnetlib.doiboost.DoiBoostMappingUtil
import eu.dnetlib.doiboost.DoiBoostMappingUtil._
import org.json4s
import org.json4s.DefaultFormats
import org.json4s.jackson.JsonMethods.parse
import org.slf4j.{Logger, LoggerFactory}
import scala.collection.JavaConverters._
import eu.dnetlib.doiboost.DoiBoostMappingUtil._
import eu.dnetlib.doiboost.uw.UnpayWallToOAF.get_unpaywall_color

View File

@ -1,70 +0,0 @@
package eu.dnetlib.dhp.doiboost
import eu.dnetlib.dhp.schema.oaf.{Publication, Dataset => OafDataset}
import eu.dnetlib.doiboost.{DoiBoostMappingUtil, HostedByItemType}
import eu.dnetlib.doiboost.SparkGenerateDoiBoost.getClass
import eu.dnetlib.doiboost.mag.ConversionUtil
import eu.dnetlib.doiboost.orcid.ORCIDElement
import org.apache.spark.SparkConf
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
import org.codehaus.jackson.map.{ObjectMapper, SerializationConfig}
import org.junit.jupiter.api.Test
import scala.io.Source
class DoiBoostHostedByMapTest {
// @Test
// def testMerge():Unit = {
// val conf: SparkConf = new SparkConf()
// val spark: SparkSession =
// SparkSession
// .builder()
// .config(conf)
// .appName(getClass.getSimpleName)
// .master("local[*]").getOrCreate()
//
//
//
// implicit val mapEncoderPub: Encoder[Publication] = Encoders.kryo[Publication]
// implicit val mapEncoderDataset: Encoder[OafDataset] = Encoders.kryo[OafDataset]
// implicit val tupleForJoinEncoder: Encoder[(String, Publication)] = Encoders.tuple(Encoders.STRING, mapEncoderPub)
//
//
// import spark.implicits._
// val dataset:RDD[String]= spark.sparkContext.textFile("/home/sandro/Downloads/hbMap.gz")
//
//
// val hbMap:Dataset[(String, HostedByItemType)] =spark.createDataset(dataset.map(DoiBoostMappingUtil.toHostedByItem))
//
//
// hbMap.show()
//
//
//
//
//
//
//
//
//
//
// }
@Test
def idDSGeneration():Unit = {
val s ="doajarticles::0066-782X"
println(DoiBoostMappingUtil.generateDSId(s))
}
}

View File

@ -0,0 +1,20 @@
package eu.dnetlib.dhp.doiboost
import eu.dnetlib.doiboost.DoiBoostMappingUtil
import org.junit.jupiter.api.Test
class DoiBoostHostedByMapTest {
@Test
def idDSGeneration():Unit = {
val s ="doajarticles::0066-782X"
println(DoiBoostMappingUtil.generateDSId(s))
}
}

View File

@ -1,7 +1,8 @@
package eu.dnetlib.doiboost.crossref
package eu.dnetlib.dhp.doiboost.crossref
import eu.dnetlib.dhp.schema.oaf._
import eu.dnetlib.dhp.utils.DHPUtils
import eu.dnetlib.doiboost.crossref.Crossref2Oaf
import org.codehaus.jackson.map.{ObjectMapper, SerializationConfig}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
@ -21,9 +22,9 @@ class CrossrefMappingTest {
@Test
def testFunderRelationshipsMapping(): Unit = {
val template = Source.fromInputStream(getClass.getResourceAsStream("article_funder_template.json")).mkString
val funder_doi = Source.fromInputStream(getClass.getResourceAsStream("funder_doi")).mkString
val funder_name = Source.fromInputStream(getClass.getResourceAsStream("funder_doi")).mkString
val template = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/article_funder_template.json")).mkString
val funder_doi = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/funder_doi")).mkString
val funder_name = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/funder_doi")).mkString
for (line <- funder_doi.lines) {
@ -72,7 +73,7 @@ class CrossrefMappingTest {
@Test
def testOrcidID() :Unit = {
val json = Source.fromInputStream(getClass.getResourceAsStream("orcid_data.json")).mkString
val json = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/orcid_data.json")).mkString
assertNotNull(json)
@ -93,7 +94,7 @@ class CrossrefMappingTest {
@Test
def testEmptyTitle() :Unit = {
val json = Source.fromInputStream(getClass.getResourceAsStream("empty_title.json")).mkString
val json = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/empty_title.json")).mkString
assertNotNull(json)
@ -115,7 +116,7 @@ class CrossrefMappingTest {
@Test
def testPeerReviewed(): Unit = {
val json = Source.fromInputStream(getClass.getResourceAsStream("prwTest.json")).mkString
val json = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/prwTest.json")).mkString
mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT)
assertNotNull(json)
@ -156,7 +157,7 @@ class CrossrefMappingTest {
@Test
def testJournalRelation(): Unit = {
val json = Source.fromInputStream(getClass.getResourceAsStream("awardTest.json")).mkString
val json = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/awardTest.json")).mkString
assertNotNull(json)
assertFalse(json.isEmpty)
@ -177,7 +178,7 @@ class CrossrefMappingTest {
@Test
def testConvertBookFromCrossRef2Oaf(): Unit = {
val json = Source.fromInputStream(getClass.getResourceAsStream("book.json")).mkString
val json = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/book.json")).mkString
assertNotNull(json)
assertFalse(json.isEmpty);
@ -233,7 +234,7 @@ class CrossrefMappingTest {
@Test
def testConvertPreprintFromCrossRef2Oaf(): Unit = {
val json = Source.fromInputStream(getClass.getResourceAsStream("preprint.json")).mkString
val json = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/preprint.json")).mkString
assertNotNull(json)
assertFalse(json.isEmpty);
@ -291,7 +292,7 @@ class CrossrefMappingTest {
@Test
def testConvertDatasetFromCrossRef2Oaf(): Unit = {
val json = Source.fromInputStream(getClass.getResourceAsStream("dataset.json")).mkString
val json = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/dataset.json")).mkString
assertNotNull(json)
assertFalse(json.isEmpty);
@ -332,7 +333,7 @@ class CrossrefMappingTest {
@Test
def testConvertArticleFromCrossRef2Oaf(): Unit = {
val json = Source.fromInputStream(getClass.getResourceAsStream("article.json")).mkString
val json = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/article.json")).mkString
assertNotNull(json)
assertFalse(json.isEmpty);
@ -400,7 +401,7 @@ class CrossrefMappingTest {
@Test
def testSetDateOfAcceptanceCrossRef2Oaf(): Unit = {
val json = Source.fromInputStream(getClass.getResourceAsStream("dump_file.json")).mkString
val json = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/dump_file.json")).mkString
assertNotNull(json)
assertFalse(json.isEmpty);
@ -415,55 +416,12 @@ class CrossrefMappingTest {
assert(items.size == 1)
val result: Result = items.head.asInstanceOf[Publication]
assertNotNull(result)
logger.info(mapper.writeValueAsString(result));
// assertNotNull(result.getDataInfo, "Datainfo test not null Failed");
// assertNotNull(
// result.getDataInfo.getProvenanceaction,
// "DataInfo/Provenance test not null Failed");
// assertFalse(
// result.getDataInfo.getProvenanceaction.getClassid.isEmpty,
// "DataInfo/Provenance/classId test not null Failed");
// assertFalse(
// result.getDataInfo.getProvenanceaction.getClassname.isEmpty,
// "DataInfo/Provenance/className test not null Failed");
// assertFalse(
// result.getDataInfo.getProvenanceaction.getSchemeid.isEmpty,
// "DataInfo/Provenance/SchemeId test not null Failed");
// assertFalse(
// result.getDataInfo.getProvenanceaction.getSchemename.isEmpty,
// "DataInfo/Provenance/SchemeName test not null Failed");
//
// assertNotNull(result.getCollectedfrom, "CollectedFrom test not null Failed");
// assertFalse(result.getCollectedfrom.isEmpty);
//
// val collectedFromList = result.getCollectedfrom.asScala
// assert(collectedFromList.exists(c => c.getKey.equalsIgnoreCase("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2")), "Wrong collected from assertion")
//
// assert(collectedFromList.exists(c => c.getValue.equalsIgnoreCase("crossref")), "Wrong collected from assertion")
//
//
// val relevantDates = result.getRelevantdate.asScala
//
// assert(relevantDates.exists(d => d.getQualifier.getClassid.equalsIgnoreCase("created")), "Missing relevant date of type created")
//
// val rels = resultList.filter(p => p.isInstanceOf[Relation]).asInstanceOf[List[Relation]]
// assertFalse(rels.isEmpty)
// rels.foreach(relation => {
// assertNotNull(relation)
// assertFalse(relation.getSource.isEmpty)
// assertFalse(relation.getTarget.isEmpty)
// assertFalse(relation.getRelClass.isEmpty)
// assertFalse(relation.getRelType.isEmpty)
// assertFalse(relation.getSubRelType.isEmpty)
//
// })
}
@Test
def testNormalizeDOI(): Unit = {
val template = Source.fromInputStream(getClass.getResourceAsStream("article_funder_template.json")).mkString
val template = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/article_funder_template.json")).mkString
val line :String = "\"funder\": [{\"name\": \"Wellcome Trust Masters Fellowship\",\"award\": [\"090633\"]}],"
val json = template.replace("%s", line)
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
@ -479,7 +437,7 @@ class CrossrefMappingTest {
@Test
def testNormalizeDOI2(): Unit = {
val template = Source.fromInputStream(getClass.getResourceAsStream("article.json")).mkString
val template = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/article.json")).mkString
val resultList: List[Oaf] = Crossref2Oaf.convert(template)
assertTrue(resultList.nonEmpty)
@ -494,7 +452,7 @@ class CrossrefMappingTest {
@Test
def testLicenseVorClosed() :Unit = {
val json = Source.fromInputStream(getClass.getResourceAsStream("publication_license_vor.json")).mkString
val json = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/publication_license_vor.json")).mkString
assertNotNull(json)
@ -521,7 +479,7 @@ class CrossrefMappingTest {
@Test
def testLicenseOpen() :Unit = {
val json = Source.fromInputStream(getClass.getResourceAsStream("publication_license_open.json")).mkString
val json = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/publication_license_open.json")).mkString
assertNotNull(json)
@ -544,7 +502,7 @@ class CrossrefMappingTest {
@Test
def testLicenseEmbargoOpen() :Unit = {
val json = Source.fromInputStream(getClass.getResourceAsStream("publication_license_embargo_open.json")).mkString
val json = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/publication_license_embargo_open.json")).mkString
assertNotNull(json)
@ -567,7 +525,7 @@ class CrossrefMappingTest {
@Test
def testLicenseEmbargo() :Unit = {
val json = Source.fromInputStream(getClass.getResourceAsStream("publication_license_embargo.json")).mkString
val json = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/publication_license_embargo.json")).mkString
assertNotNull(json)
@ -591,7 +549,7 @@ class CrossrefMappingTest {
@Test
def testLicenseEmbargoDateTime() :Unit = {
val json = Source.fromInputStream(getClass.getResourceAsStream("publication_license_embargo_datetime.json")).mkString
val json = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/publication_license_embargo_datetime.json")).mkString
assertNotNull(json)
@ -614,7 +572,7 @@ class CrossrefMappingTest {
@Test
def testMultipleURLs() :Unit = {
val json = Source.fromInputStream(getClass.getResourceAsStream("multiple_urls.json")).mkString
val json = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/multiple_urls.json")).mkString
assertNotNull(json)

View File

@ -1,11 +1,12 @@
package eu.dnetlib.doiboost.mag
package eu.dnetlib.dhp.doiboost.mag
import eu.dnetlib.doiboost.mag.{ConversionUtil, MagPapers, SparkProcessMAG}
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Dataset, SparkSession}
import org.codehaus.jackson.map.ObjectMapper
import org.json4s.DefaultFormats
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
import org.json4s.DefaultFormats
import org.slf4j.{Logger, LoggerFactory}
import java.sql.Timestamp
@ -47,7 +48,7 @@ class MAGMappingTest {
@Test
def buildInvertedIndexTest(): Unit = {
val json_input = Source.fromInputStream(getClass.getResourceAsStream("invertedIndex.json")).mkString
val json_input = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/mag/invertedIndex.json")).mkString
val description = ConversionUtil.convertInvertedIndexString(json_input)
assertNotNull(description)
assertTrue(description.nonEmpty)
@ -71,7 +72,7 @@ class MAGMappingTest {
.appName(getClass.getSimpleName)
.config(conf)
.getOrCreate()
val path = getClass.getResource("magPapers.json").getPath
val path = getClass.getResource("/eu/dnetlib/doiboost/mag/magPapers.json").getPath
import org.apache.spark.sql.Encoders
val schema = Encoders.product[MagPapers].schema
@ -101,7 +102,7 @@ class MAGMappingTest {
.appName(getClass.getSimpleName)
.config(conf)
.getOrCreate()
val path = getClass.getResource("duplicatedMagPapers.json").getPath
val path = getClass.getResource("/eu/dnetlib/doiboost/mag/duplicatedMagPapers.json").getPath
import org.apache.spark.sql.Encoders
val schema = Encoders.product[MagPapers].schema

View File

@ -1,7 +1,8 @@
package eu.dnetlib.doiboost.orcid
package eu.dnetlib.dhp.doiboost.orcid
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.schema.oaf.Publication
import eu.dnetlib.doiboost.orcid._
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession}
import org.junit.jupiter.api.Assertions._
@ -10,9 +11,8 @@ import org.junit.jupiter.api.io.TempDir
import org.slf4j.{Logger, LoggerFactory}
import java.nio.file.Path
import scala.io.Source
import scala.collection.JavaConversions._
import scala.io.Source
class MappingORCIDToOAFTest {
val logger: Logger = LoggerFactory.getLogger(ORCIDToOAF.getClass)
@ -20,7 +20,7 @@ class MappingORCIDToOAFTest {
@Test
def testExtractData():Unit ={
val json = Source.fromInputStream(getClass.getResourceAsStream("dataOutput")).mkString
val json = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/orcid/dataOutput")).mkString
assertNotNull(json)
assertFalse(json.isEmpty)
json.lines.foreach(s => {

View File

@ -1,13 +1,13 @@
package eu.dnetlib.doiboost.uw
package eu.dnetlib.dhp.doiboost.uw
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.schema.oaf.OpenAccessRoute
import eu.dnetlib.doiboost.uw.UnpayWallToOAF
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
import org.slf4j.{Logger, LoggerFactory}
import scala.io.Source
import org.junit.jupiter.api.Assertions._
import org.slf4j.{Logger, LoggerFactory}
class UnpayWallMappingTest {
@ -18,7 +18,7 @@ class UnpayWallMappingTest {
@Test
def testMappingToOAF():Unit ={
val Ilist = Source.fromInputStream(getClass.getResourceAsStream("input.json")).mkString
val Ilist = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/uw/input.json")).mkString
var i:Int = 0
for (line <-Ilist.lines) {

View File

@ -88,7 +88,7 @@ public class CleanGraphSparkJob {
readTableFromPath(spark, inputPath, clazz)
.map((MapFunction<T, T>) GraphCleaningFunctions::fixVocabularyNames, Encoders.bean(clazz))
.map((MapFunction<T, T>) value -> OafCleaner.apply(value, mapping), Encoders.bean(clazz))
.map((MapFunction<T, T>) GraphCleaningFunctions::cleanup, Encoders.bean(clazz))
.map((MapFunction<T, T>) value -> GraphCleaningFunctions.cleanup(value, vocs), Encoders.bean(clazz))
.filter((FilterFunction<T>) GraphCleaningFunctions::filter)
.write()
.mode(SaveMode.Overwrite)

View File

@ -1,8 +1,8 @@
package eu.dnetlib.dhp.oa.graph.hostedbymap
import eu.dnetlib.dhp.oa.graph.hostedbymap.model.EntityInfo
import org.apache.spark.sql.{Dataset, Encoder, Encoders, TypedColumn}
import org.apache.spark.sql.expressions.Aggregator
import org.apache.spark.sql.{Dataset, Encoder, Encoders, TypedColumn}
case class HostedByItemType(id: String, officialname: String, issn: String, eissn: String, lissn: String, openAccess: Boolean) {}

View File

@ -2,13 +2,12 @@ package eu.dnetlib.dhp.oa.graph.hostedbymap
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.oa.graph.hostedbymap.SparkApplyHostedByMapToResult.{applyHBtoPubs, getClass}
import eu.dnetlib.dhp.oa.graph.hostedbymap.model.EntityInfo
import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf.{Datasource, Publication}
import eu.dnetlib.dhp.schema.oaf.Datasource
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
import org.apache.spark.sql._
import org.json4s.DefaultFormats
import org.slf4j.{Logger, LoggerFactory}

View File

@ -5,16 +5,14 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.oa.graph.hostedbymap.model.EntityInfo
import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils
import eu.dnetlib.dhp.schema.oaf.{Datasource, Instance, OpenAccessRoute, Publication}
import eu.dnetlib.dhp.schema.oaf.{Instance, OpenAccessRoute, Publication}
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
import org.apache.spark.sql._
import org.json4s.DefaultFormats
import org.slf4j.{Logger, LoggerFactory}
import scala.collection.JavaConverters._
object SparkApplyHostedByMapToResult {
def applyHBtoPubs(join: Dataset[EntityInfo], pubs: Dataset[Publication]) = {
@ -25,7 +23,7 @@ object SparkApplyHostedByMapToResult {
val ei: EntityInfo = t2._2
val i = p.getInstance().asScala
if (i.size == 1) {
val inst: Instance = i(0)
val inst: Instance = i.head
inst.getHostedby.setKey(ei.getHostedById)
inst.getHostedby.setValue(ei.getName)
if (ei.getOpenAccess) {
@ -39,6 +37,7 @@ object SparkApplyHostedByMapToResult {
p
})(Encoders.bean(classOf[Publication]))
}
def main(args: Array[String]): Unit = {

View File

@ -3,18 +3,15 @@ package eu.dnetlib.dhp.oa.graph.hostedbymap
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.oa.graph.hostedbymap.model.EntityInfo
import eu.dnetlib.dhp.schema.oaf.{Journal, Publication}
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
import org.apache.spark.sql._
import org.json4s
import org.json4s.DefaultFormats
import org.json4s.jackson.JsonMethods.parse
import org.slf4j.{Logger, LoggerFactory}
object SparkPrepareHostedByInfoToApply {
implicit val mapEncoderPInfo: Encoder[EntityInfo] = Encoders.bean(classOf[EntityInfo])

View File

@ -1,22 +1,20 @@
package eu.dnetlib.dhp.oa.graph.hostedbymap
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.oa.graph.hostedbymap.model.{DOAJModel, UnibiGoldModel}
import eu.dnetlib.dhp.schema.oaf.Datasource
import org.apache.commons.io.IOUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.io.compress.GzipCodec
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession}
import org.json4s.DefaultFormats
import org.slf4j.{Logger, LoggerFactory}
import com.fasterxml.jackson.databind.ObjectMapper
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.FileSystem
import org.apache.hadoop.fs.Path
import java.io.PrintWriter
import org.apache.hadoop.io.compress.GzipCodec
object SparkProduceHostedByMap {
@ -52,7 +50,6 @@ object SparkProduceHostedByMap {
}
def getHostedByItemType(id: String, officialname: String, issn: String, eissn: String, issnl: String, oa: Boolean): HostedByItemType = {
if (issn != null) {
if (eissn != null) {
@ -163,7 +160,6 @@ object SparkProduceHostedByMap {
}
def writeToHDFS(input: Array[String], outputPath: String, hdfsNameNode: String): Unit = {
val conf = new Configuration()
@ -182,7 +178,6 @@ object SparkProduceHostedByMap {
}
def main(args: Array[String]): Unit = {
val logger: Logger = LoggerFactory.getLogger(getClass)

View File

@ -4,17 +4,11 @@ import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.common.HdfsSupport
import eu.dnetlib.dhp.schema.common.ModelSupport
import eu.dnetlib.dhp.schema.mdstore.MDStoreWithInfo
import eu.dnetlib.dhp.schema.oaf.Oaf
import eu.dnetlib.dhp.utils.DHPUtils
import org.apache.commons.io.IOUtils
import org.apache.commons.lang3.StringUtils
import org.apache.http.client.methods.HttpGet
import org.apache.http.impl.client.HttpClients
import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}
import org.apache.spark.{SparkConf, SparkContext}
import org.slf4j.LoggerFactory
import scala.collection.JavaConverters._
import scala.io.Source

View File

@ -2,9 +2,8 @@ package eu.dnetlib.dhp.oa.graph.resolution
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.common.HdfsSupport
import eu.dnetlib.dhp.schema.common.EntityType
import eu.dnetlib.dhp.schema.oaf.{OtherResearchProduct, Publication, Result, Software, Dataset => OafDataset}
import eu.dnetlib.dhp.schema.oaf.{Dataset => OafDataset,_}
import org.apache.commons.io.IOUtils
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.spark.SparkConf

View File

@ -3,7 +3,7 @@ package eu.dnetlib.dhp.oa.graph.resolution
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.common.HdfsSupport
import eu.dnetlib.dhp.schema.oaf.{Relation, Result}
import eu.dnetlib.dhp.schema.oaf.Relation
import eu.dnetlib.dhp.utils.DHPUtils
import org.apache.commons.io.IOUtils
import org.apache.hadoop.fs.{FileSystem, Path}

View File

@ -18,7 +18,6 @@ object SparkDataciteToOAF {
.config(conf)
.appName(getClass.getSimpleName)
.master(parser.get("master")).getOrCreate()
import spark.implicits._
val sc = spark.sparkContext

View File

@ -2,7 +2,7 @@ package eu.dnetlib.dhp.sx.graph
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.schema.oaf.{Oaf, OtherResearchProduct, Publication, Result, Software, Dataset => OafDataset}
import eu.dnetlib.dhp.schema.oaf.Result
import org.apache.commons.io.IOUtils
import org.apache.hadoop.io.compress.GzipCodec
import org.apache.spark.SparkConf

View File

@ -5,10 +5,10 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.schema.sx.scholix.Scholix
import eu.dnetlib.dhp.schema.sx.summary.ScholixSummary
import org.apache.commons.io.IOUtils
import org.apache.hadoop.io.compress.GzipCodec
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession}
import org.slf4j.{Logger, LoggerFactory}
import org.apache.hadoop.io.compress._
object SparkConvertObjectToJson {

View File

@ -2,11 +2,12 @@ package eu.dnetlib.dhp.sx.graph
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.schema.oaf.{OtherResearchProduct, Publication, Relation, Result, Software, Dataset => OafDataset}
import eu.dnetlib.dhp.schema.oaf.{OtherResearchProduct, Publication, Relation, Software,Dataset => OafDataset}
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}
import org.slf4j.{Logger, LoggerFactory}
object SparkConvertRDDtoDataset {
def main(args: Array[String]): Unit = {

View File

@ -1,14 +1,12 @@
package eu.dnetlib.dhp.sx.graph
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.schema.oaf.{Oaf, OtherResearchProduct, Publication, Relation, Result, Software, Dataset => OafDataset}
import eu.dnetlib.dhp.schema.oaf.{Dataset => OafDataset,_}
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
import org.apache.spark.sql._
import org.slf4j.{Logger, LoggerFactory}
object SparkCreateInputGraph {
def main(args: Array[String]): Unit = {
@ -41,9 +39,6 @@ object SparkCreateInputGraph {
implicit val relEncoder: Encoder[Relation] = Encoders.kryo(classOf[Relation])
val sourcePath = parser.get("sourcePath")
log.info(s"sourcePath -> $sourcePath")
val targetPath = parser.get("targetPath")

View File

@ -9,7 +9,7 @@ import eu.dnetlib.dhp.sx.graph.scholix.ScholixUtils.RelatedEntities
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql.functions.count
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
import org.apache.spark.sql._
import org.slf4j.{Logger, LoggerFactory}
object SparkCreateScholix {

View File

@ -6,7 +6,7 @@ import eu.dnetlib.dhp.schema.sx.summary.ScholixSummary
import eu.dnetlib.dhp.sx.graph.scholix.ScholixUtils
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
import org.apache.spark.sql._
import org.slf4j.{Logger, LoggerFactory}
object SparkCreateSummaryObject {

View File

@ -5,6 +5,7 @@ import org.apache.spark.sql.{Encoder, Encoders}
import org.json4s
import org.json4s.DefaultFormats
import org.json4s.jackson.JsonMethods.parse
import java.util.regex.Pattern
import scala.language.postfixOps
import scala.xml.{Elem, Node, XML}

View File

@ -2,11 +2,11 @@ package eu.dnetlib.dhp.sx.graph.pangaea
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import org.apache.spark.rdd.RDD
import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}
import org.apache.spark.{SparkConf, SparkContext}
import org.slf4j.{Logger, LoggerFactory}
import scala.collection.JavaConverters._
import scala.io.Source
object SparkGeneratePanagaeaDataset {
@ -46,7 +46,4 @@ object SparkGeneratePanagaeaDataset {
}
}

View File

@ -1,6 +1,5 @@
package eu.dnetlib.dhp.sx.graph.scholix
import eu.dnetlib.dhp.schema.oaf.{Publication, Relation, Result, StructuredProperty}
import eu.dnetlib.dhp.schema.sx.scholix._
import eu.dnetlib.dhp.schema.sx.summary.{CollectedFromType, SchemeValue, ScholixSummary, Typology}
@ -10,10 +9,8 @@ import org.apache.spark.sql.{Encoder, Encoders}
import org.json4s
import org.json4s.DefaultFormats
import org.json4s.jackson.JsonMethods.parse
import scala.collection.JavaConverters._
import scala.io.Source
import scala.language.postfixOps
object ScholixUtils {
@ -21,6 +18,7 @@ object ScholixUtils {
val DNET_IDENTIFIER_SCHEMA: String = "DNET Identifier"
val DATE_RELATION_KEY: String = "RelationDate"
case class RelationVocabulary(original: String, inverse: String) {}
case class RelatedEntities(id: String, relatedDataset: Long, relatedPublication: Long) {}
@ -66,7 +64,6 @@ object ScholixUtils {
}
val statsAggregator: Aggregator[(String, String, Long), RelatedEntities, RelatedEntities] = new Aggregator[(String, String, Long), RelatedEntities, RelatedEntities] with Serializable {
override def zero: RelatedEntities = null
@ -85,8 +82,7 @@ object ScholixUtils {
if (b1 != null && b2 != null)
RelatedEntities(b1.id, b1.relatedDataset + b2.relatedDataset, b1.relatedPublication + b2.relatedPublication)
else
if (b1!= null)
else if (b1 != null)
b1
else
b2
@ -144,7 +140,6 @@ object ScholixUtils {
s
}
@ -203,8 +198,7 @@ object ScholixUtils {
if (summaryObject.getDate != null && !summaryObject.getDate.isEmpty)
r.setPublicationDate(summaryObject.getDate.get(0))
if (summaryObject.getPublisher!= null && !summaryObject.getPublisher.isEmpty)
{
if (summaryObject.getPublisher != null && !summaryObject.getPublisher.isEmpty) {
val plist: List[ScholixEntityId] = summaryObject.getPublisher.asScala.map(p => new ScholixEntityId(p, null)).toList
if (plist.nonEmpty)
@ -228,9 +222,6 @@ object ScholixUtils {
}
def scholixFromSource(relation: Relation, source: ScholixSummary): Scholix = {
if (relation == null || source == null)

View File

@ -74,6 +74,53 @@ public class GraphCleaningFunctionsTest {
}
}
@Test
void testFilter_false() throws Exception {
assertNotNull(vocabularies);
assertNotNull(mapping);
String json = IOUtils
.toString(getClass().getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/result_invisible.json"));
Publication p_in = MAPPER.readValue(json, Publication.class);
assertTrue(p_in instanceof Result);
assertTrue(p_in instanceof Publication);
assertEquals(false, GraphCleaningFunctions.filter(p_in));
}
@Test
void testFilter_true() throws Exception {
assertNotNull(vocabularies);
assertNotNull(mapping);
String json = IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/result.json"));
Publication p_in = MAPPER.readValue(json, Publication.class);
assertTrue(p_in instanceof Result);
assertTrue(p_in instanceof Publication);
assertEquals(true, GraphCleaningFunctions.filter(p_in));
}
@Test
void testFilter_missing_invisible() throws Exception {
assertNotNull(vocabularies);
assertNotNull(mapping);
String json = IOUtils
.toString(getClass().getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/result_missing_invisible.json"));
Publication p_in = MAPPER.readValue(json, Publication.class);
assertTrue(p_in instanceof Result);
assertTrue(p_in instanceof Publication);
assertEquals(true, GraphCleaningFunctions.filter(p_in));
}
@Test
void testCleaning() throws Exception {
@ -104,6 +151,12 @@ public class GraphCleaningFunctionsTest {
assertEquals("0018", p_out.getInstance().get(0).getInstancetype().getClassid());
assertEquals("Annotation", p_out.getInstance().get(0).getInstancetype().getClassname());
assertEquals("0027", p_out.getInstance().get(1).getInstancetype().getClassid());
assertEquals("Model", p_out.getInstance().get(1).getInstancetype().getClassname());
assertEquals("xyz", p_out.getInstance().get(2).getInstancetype().getClassid());
assertEquals("xyz", p_out.getInstance().get(2).getInstancetype().getClassname());
assertEquals("CLOSED", p_out.getInstance().get(0).getAccessright().getClassid());
assertEquals("Closed Access", p_out.getInstance().get(0).getAccessright().getClassname());
@ -117,7 +170,7 @@ public class GraphCleaningFunctionsTest {
List<Instance> poi = p_out.getInstance();
assertNotNull(poi);
assertEquals(1, poi.size());
assertEquals(3, poi.size());
final Instance poii = poi.get(0);
assertNotNull(poii);
@ -145,7 +198,7 @@ public class GraphCleaningFunctionsTest {
assertEquals(5, p_out.getTitle().size());
Publication p_cleaned = GraphCleaningFunctions.cleanup(p_out);
Publication p_cleaned = GraphCleaningFunctions.cleanup(p_out, vocabularies);
assertEquals(3, p_cleaned.getTitle().size());
@ -164,9 +217,12 @@ public class GraphCleaningFunctionsTest {
assertEquals("1970-10-07", p_cleaned.getDateofacceptance().getValue());
assertEquals("0038", p_cleaned.getInstance().get(2).getInstancetype().getClassid());
assertEquals("Other literature type", p_cleaned.getInstance().get(2).getInstancetype().getClassname());
final List<Instance> pci = p_cleaned.getInstance();
assertNotNull(pci);
assertEquals(1, pci.size());
assertEquals(3, pci.size());
final Instance pcii = pci.get(0);
assertNotNull(pcii);
@ -234,7 +290,7 @@ public class GraphCleaningFunctionsTest {
.toString(getClass().getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/doiboostpub.json"));
Publication p_in = MAPPER.readValue(json, Publication.class);
Publication p_out = OafCleaner.apply(GraphCleaningFunctions.fixVocabularyNames(p_in), mapping);
Publication cleaned = GraphCleaningFunctions.cleanup(p_out);
Publication cleaned = GraphCleaningFunctions.cleanup(p_out, vocabularies);
Assertions.assertEquals(true, GraphCleaningFunctions.filter(cleaned));
}
@ -245,7 +301,7 @@ public class GraphCleaningFunctionsTest {
.toString(getClass().getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/doiboostpub2.json"));
Publication p_in = MAPPER.readValue(json, Publication.class);
Publication p_out = OafCleaner.apply(GraphCleaningFunctions.fixVocabularyNames(p_in), mapping);
Publication cleaned = GraphCleaningFunctions.cleanup(p_out);
Publication cleaned = GraphCleaningFunctions.cleanup(p_out, vocabularies);
Assertions.assertEquals(true, GraphCleaningFunctions.filter(cleaned));

View File

@ -171,30 +171,6 @@ public class DumpJobTest {
GraphResult gr = verificationDataset.first();
Assertions.assertEquals(1, gr.getInstance().size());
Assertions.assertEquals(2, gr.getInstance().get(0).getMeasures().size());
Assertions
.assertTrue(
gr
.getInstance()
.get(0)
.getMeasures()
.stream()
.anyMatch(
m -> m.getKey().equals("influence")
&& m.getValue().equals("1.62759106106e-08")));
Assertions
.assertTrue(
gr
.getInstance()
.get(0)
.getMeasures()
.stream()
.anyMatch(
m -> m.getKey().equals("popularity")
&& m.getValue().equals("0.22519296")));
Assertions.assertEquals(6, gr.getAuthor().size());
Assertions
.assertTrue(
@ -363,8 +339,6 @@ public class DumpJobTest {
Assertions.assertEquals("50|pensoft_____::00ea4a1cd53806a97d62ea6bf268f2a2", gr.getId());
System.out.println(gr.getOriginalId().size());
Assertions.assertEquals(1, gr.getOriginalId().size());
Assertions
.assertTrue(

View File

@ -708,7 +708,7 @@ class MappersTest {
assertEquals(1, p.getTitle().size());
assertTrue(StringUtils.isNotBlank(p.getTitle().get(0).getValue()));
final Publication p_cleaned = cleanup(fixVocabularyNames(p));
final Publication p_cleaned = cleanup(fixVocabularyNames(p), vocs);
assertNotNull(p_cleaned.getTitle());
assertFalse(p_cleaned.getTitle().isEmpty());

View File

@ -403,6 +403,178 @@
"http://juuli.fi/Record/0275158616",
"http://dx.doi.org/10.1007/s109090161569x"
]
},
{
"pid": [
{
"dataInfo": null,
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1002/s21010127267xy"
},
{
"dataInfo": null,
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1008/abcd"
}
],
"alternateIdentifier": [
{
"dataInfo": null,
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1007/s109090161569x"
},
{
"dataInfo": null,
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1009/qwerty"
}
],
"accessright": {
"classid": "CLOSED",
"classname": "CLOSED",
"schemeid": "dnet:access_modes",
"schemename": "dnet:access_modes"
},
"collectedfrom": {
"key": "10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747",
"value": "VIRTA"
},
"dateofacceptance": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"value": "2016-01-01"
},
"distributionlocation": "",
"hostedby": {
"key": "10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747",
"value": "VIRTA"
},
"instancetype": {
"classid": "Model",
"classname": "Model",
"schemeid": "dnet:publication_resource",
"schemename": "dnet:publication_resource"
},
"url": [
"http://dx.doi.org/10.1002/s21010127267xy"
]
},
{
"pid": [
{
"dataInfo": null,
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1002/s21010127267xy"
},
{
"dataInfo": null,
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1008/abcd"
}
],
"alternateIdentifier": [
{
"dataInfo": null,
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1007/s109090161569x"
},
{
"dataInfo": null,
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1009/qwerty"
}
],
"accessright": {
"classid": "CLOSED",
"classname": "CLOSED",
"schemeid": "dnet:access_modes",
"schemename": "dnet:access_modes"
},
"collectedfrom": {
"key": "10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747",
"value": "VIRTA"
},
"dateofacceptance": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"value": "2016-01-01"
},
"distributionlocation": "",
"hostedby": {
"key": "10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747",
"value": "VIRTA"
},
"instancetype": {
"classid": "xyz",
"classname": "xyz",
"schemeid": "dnet:publication_resource",
"schemename": "dnet:publication_resource"
},
"url": [
"http://dx.doi.org/10.1002/t32121238378t"
]
}
],
"journal": {

View File

@ -0,0 +1,958 @@
{
"author": [
{
"affiliation": [
],
"fullname": "Brien, Tom",
"name": "Tom",
"pid": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "ORCID12",
"classname": "ORCID12",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "0000-0001-9613-6639"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "ORCID12",
"classname": "ORCID12",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "https://orcid.org/0000-0001-9613-6639"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:entityregistry",
"classname": "sysimport:crosswalk:entityregistry",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "orcid",
"classname": "ORCID12",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "0000-0001-9613-6639"
}
],
"rank": 1,
"surname": "Brien"
},
{
"affiliation": [
],
"fullname": "Ade, Peter",
"name": "Peter",
"pid": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "xyz",
"classname": "XYZ",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "qwerty"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "ORCID",
"classname": "ORCID",
"schemeid": "",
"schemename": ""
},
"value": "asdasd"
}
],
"rank": 2,
"surname": "Ade"
},
{
"affiliation": [
],
"fullname": "Barry, Peter S.",
"name": "Peter S.",
"pid": null,
"rank": 3,
"surname": "Barry"
},
{
"affiliation": [
],
"fullname": "Dunscombe, Chris J.",
"name": "Chris J.",
"pid": [
],
"rank": 4,
"surname": "Dunscombe"
},
{
"affiliation": [
],
"fullname": "Leadley, David R.",
"name": "David R.",
"pid": [
],
"rank": 5,
"surname": "Leadley"
},
{
"affiliation": [
],
"fullname": "Morozov, Dmitry V.",
"name": "Dmitry V.",
"pid": [
],
"rank": 6,
"surname": "Morozov"
},
{
"affiliation": [
],
"fullname": "Myronov, Maksym",
"name": "Maksym",
"pid": [
],
"rank": 7,
"surname": "Myronov"
},
{
"affiliation": [
],
"fullname": "Parker, Evan",
"name": "Evan",
"pid": [
],
"rank": 8,
"surname": "Parker"
},
{
"affiliation": [
],
"fullname": "Prest, Martin J.",
"name": "Martin J.",
"pid": [
],
"rank": 9,
"surname": "Prest"
},
{
"affiliation": [
],
"fullname": "Prunnila, Mika",
"name": "Mika",
"pid": [
],
"rank": 10,
"surname": "Prunnila"
},
{
"affiliation": [
],
"fullname": "Sudiwala, Rashmi V.",
"name": "Rashmi V.",
"pid": [
],
"rank": 11,
"surname": "Sudiwala"
},
{
"affiliation": [
],
"fullname": "Whall, Terry E.",
"name": "Terry E.",
"pid": [
],
"rank": 12,
"surname": "Whall"
},
{
"affiliation": [
],
"fullname": "Mauskopf",
"name": "",
"pid": [
],
"rank": 13,
"surname": ""
},
{
"affiliation": [
],
"fullname": " P. D. ",
"name": "",
"pid": [
],
"rank": 14,
"surname": ""
}
],
"bestaccessright": null,
"publisher": {
"value": null
},
"collectedfrom": [
{
"key": "10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747",
"value": "VIRTA"
}
],
"context": [
],
"contributor": [
],
"country": [
{
"classid": "DE",
"classname": "DE",
"schemeid": "dnet:countries",
"schemename": "dnet:countries"
}
],
"coverage": [
],
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": true,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"dateofacceptance": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"value": "7 oct 1970"
},
"dateofcollection": "",
"dateoftransformation": "2020-04-22T12:34:08.009Z",
"description": [
],
"externalReference": [
],
"extraInfo": [
],
"format": [
],
"fulltext": [
],
"id": "50|CSC_________::2250a70c903c6ac6e4c01438259e9375",
"instance": [
{
"pid": [
{
"dataInfo": null,
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1007/s109090161569x"
},
{
"dataInfo": null,
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1008/abcd"
}
],
"alternateIdentifier": [
{
"dataInfo": null,
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1007/s109090161569x"
},
{
"dataInfo": null,
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1009/qwerty"
}
],
"accessright": {
"classid": "CLOSED",
"classname": "CLOSED",
"schemeid": "dnet:access_modes",
"schemename": "dnet:access_modes"
},
"collectedfrom": {
"key": "10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747",
"value": "VIRTA"
},
"dateofacceptance": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"value": "2016-01-01"
},
"distributionlocation": "",
"hostedby": {
"key": "10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747",
"value": "VIRTA"
},
"instancetype": {
"classid": "Comment/debate",
"classname": "Comment/debate",
"schemeid": "dnet:publication_resource",
"schemename": "dnet:publication_resource"
},
"url": [
"http://juuli.fi/Record/0275158616",
"http://dx.doi.org/10.1007/s109090161569x"
]
}
],
"journal": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"edition": "",
"ep": " 7",
"iss": "9 March",
"issnLinking": "",
"issnOnline": "",
"issnPrinted": "0022-2291",
"name": "Journal of Low Temperature Physics - Early Acces",
"sp": "1 ",
"vol": ""
},
"language": {
"classid": "UNKNOWN",
"classname": "UNKNOWN",
"schemeid": "dnet:languages",
"schemename": "dnet:languages"
},
"lastupdatetimestamp": 1591283286319,
"oaiprovenance": {
"originDescription": {
"altered": true,
"baseURL": "https%3A%2F%2Fvirta-jtp.csc.fi%2Fapi%2Fcerif",
"datestamp": "2019-07-30",
"harvestDate": "2020-04-22T11:04:38.685Z",
"identifier": "oai:virta-jtp.csc.fi:Publications/0275158616",
"metadataNamespace": ""
}
},
"originalId": [
"CSC_________::2250a70c903c6ac6e4c01438259e9375"
],
"pid": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1007/s109090161569x"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1007/s109090161569x"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": ""
}
],
"relevantdate": [
],
"resourcetype": {
"classid": "0001",
"classname": "0001",
"schemeid": "dnet:dataCite_resource",
"schemename": "dnet:dataCite_resource"
},
"resulttype": {
"classid": "publication",
"classname": "publication",
"schemeid": "dnet:result_typologies",
"schemename": "dnet:result_typologies"
},
"source": [
],
"subject": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "ta213"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "infrared detectors"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "lens antennas"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "silicon"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "slot antennas"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "strained silicon"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "cold electron bolometers"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "doped silicon"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "measure noise"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "noise equivalent power"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "optical characterisation"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "optical response"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "photon noise"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "silicon absorbers"
}
],
"title": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "main title",
"classname": "main title",
"schemeid": "dnet:dataCite_title",
"schemename": "dnet:dataCite_title"
},
"value": "Optical response of strained- and unstrained-silicon cold-electron bolometers test"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "main title",
"classname": "main title",
"schemeid": "dnet:dataCite_title",
"schemename": "dnet:dataCite_title"
},
"value": "test test 123 test"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "main title",
"classname": "main title",
"schemeid": "dnet:dataCite_title",
"schemename": "dnet:dataCite_title"
},
"value": "omic"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "main title",
"classname": "main title",
"schemeid": "dnet:dataCite_title",
"schemename": "dnet:dataCite_title"
},
"value": "「マキャベリ的知性と心の理論の進化論」 リチャード・バーン, アンドリュー・ホワイトゥン 編/藤田和生, 山下博志, 友永雅巳 監訳"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "main title",
"classname": "main title",
"schemeid": "dnet:dataCite_title",
"schemename": "dnet:dataCite_title"
},
"value": "-"
}
]
}

View File

@ -0,0 +1,957 @@
{
"author": [
{
"affiliation": [
],
"fullname": "Brien, Tom",
"name": "Tom",
"pid": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "ORCID12",
"classname": "ORCID12",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "0000-0001-9613-6639"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "ORCID12",
"classname": "ORCID12",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "https://orcid.org/0000-0001-9613-6639"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:entityregistry",
"classname": "sysimport:crosswalk:entityregistry",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "orcid",
"classname": "ORCID12",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "0000-0001-9613-6639"
}
],
"rank": 1,
"surname": "Brien"
},
{
"affiliation": [
],
"fullname": "Ade, Peter",
"name": "Peter",
"pid": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "xyz",
"classname": "XYZ",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "qwerty"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "ORCID",
"classname": "ORCID",
"schemeid": "",
"schemename": ""
},
"value": "asdasd"
}
],
"rank": 2,
"surname": "Ade"
},
{
"affiliation": [
],
"fullname": "Barry, Peter S.",
"name": "Peter S.",
"pid": null,
"rank": 3,
"surname": "Barry"
},
{
"affiliation": [
],
"fullname": "Dunscombe, Chris J.",
"name": "Chris J.",
"pid": [
],
"rank": 4,
"surname": "Dunscombe"
},
{
"affiliation": [
],
"fullname": "Leadley, David R.",
"name": "David R.",
"pid": [
],
"rank": 5,
"surname": "Leadley"
},
{
"affiliation": [
],
"fullname": "Morozov, Dmitry V.",
"name": "Dmitry V.",
"pid": [
],
"rank": 6,
"surname": "Morozov"
},
{
"affiliation": [
],
"fullname": "Myronov, Maksym",
"name": "Maksym",
"pid": [
],
"rank": 7,
"surname": "Myronov"
},
{
"affiliation": [
],
"fullname": "Parker, Evan",
"name": "Evan",
"pid": [
],
"rank": 8,
"surname": "Parker"
},
{
"affiliation": [
],
"fullname": "Prest, Martin J.",
"name": "Martin J.",
"pid": [
],
"rank": 9,
"surname": "Prest"
},
{
"affiliation": [
],
"fullname": "Prunnila, Mika",
"name": "Mika",
"pid": [
],
"rank": 10,
"surname": "Prunnila"
},
{
"affiliation": [
],
"fullname": "Sudiwala, Rashmi V.",
"name": "Rashmi V.",
"pid": [
],
"rank": 11,
"surname": "Sudiwala"
},
{
"affiliation": [
],
"fullname": "Whall, Terry E.",
"name": "Terry E.",
"pid": [
],
"rank": 12,
"surname": "Whall"
},
{
"affiliation": [
],
"fullname": "Mauskopf",
"name": "",
"pid": [
],
"rank": 13,
"surname": ""
},
{
"affiliation": [
],
"fullname": " P. D. ",
"name": "",
"pid": [
],
"rank": 14,
"surname": ""
}
],
"bestaccessright": null,
"publisher": {
"value": null
},
"collectedfrom": [
{
"key": "10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747",
"value": "VIRTA"
}
],
"context": [
],
"contributor": [
],
"country": [
{
"classid": "DE",
"classname": "DE",
"schemeid": "dnet:countries",
"schemename": "dnet:countries"
}
],
"coverage": [
],
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"dateofacceptance": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"value": "7 oct 1970"
},
"dateofcollection": "",
"dateoftransformation": "2020-04-22T12:34:08.009Z",
"description": [
],
"externalReference": [
],
"extraInfo": [
],
"format": [
],
"fulltext": [
],
"id": "50|CSC_________::2250a70c903c6ac6e4c01438259e9375",
"instance": [
{
"pid": [
{
"dataInfo": null,
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1007/s109090161569x"
},
{
"dataInfo": null,
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1008/abcd"
}
],
"alternateIdentifier": [
{
"dataInfo": null,
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1007/s109090161569x"
},
{
"dataInfo": null,
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1009/qwerty"
}
],
"accessright": {
"classid": "CLOSED",
"classname": "CLOSED",
"schemeid": "dnet:access_modes",
"schemename": "dnet:access_modes"
},
"collectedfrom": {
"key": "10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747",
"value": "VIRTA"
},
"dateofacceptance": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"value": "2016-01-01"
},
"distributionlocation": "",
"hostedby": {
"key": "10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747",
"value": "VIRTA"
},
"instancetype": {
"classid": "Comment/debate",
"classname": "Comment/debate",
"schemeid": "dnet:publication_resource",
"schemename": "dnet:publication_resource"
},
"url": [
"http://juuli.fi/Record/0275158616",
"http://dx.doi.org/10.1007/s109090161569x"
]
}
],
"journal": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"edition": "",
"ep": " 7",
"iss": "9 March",
"issnLinking": "",
"issnOnline": "",
"issnPrinted": "0022-2291",
"name": "Journal of Low Temperature Physics - Early Acces",
"sp": "1 ",
"vol": ""
},
"language": {
"classid": "UNKNOWN",
"classname": "UNKNOWN",
"schemeid": "dnet:languages",
"schemename": "dnet:languages"
},
"lastupdatetimestamp": 1591283286319,
"oaiprovenance": {
"originDescription": {
"altered": true,
"baseURL": "https%3A%2F%2Fvirta-jtp.csc.fi%2Fapi%2Fcerif",
"datestamp": "2019-07-30",
"harvestDate": "2020-04-22T11:04:38.685Z",
"identifier": "oai:virta-jtp.csc.fi:Publications/0275158616",
"metadataNamespace": ""
}
},
"originalId": [
"CSC_________::2250a70c903c6ac6e4c01438259e9375"
],
"pid": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1007/s109090161569x"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1007/s109090161569x"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": ""
}
],
"relevantdate": [
],
"resourcetype": {
"classid": "0001",
"classname": "0001",
"schemeid": "dnet:dataCite_resource",
"schemename": "dnet:dataCite_resource"
},
"resulttype": {
"classid": "publication",
"classname": "publication",
"schemeid": "dnet:result_typologies",
"schemename": "dnet:result_typologies"
},
"source": [
],
"subject": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "ta213"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "infrared detectors"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "lens antennas"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "silicon"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "slot antennas"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "strained silicon"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "cold electron bolometers"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "doped silicon"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "measure noise"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "noise equivalent power"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "optical characterisation"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "optical response"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "photon noise"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"value": "silicon absorbers"
}
],
"title": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "main title",
"classname": "main title",
"schemeid": "dnet:dataCite_title",
"schemename": "dnet:dataCite_title"
},
"value": "Optical response of strained- and unstrained-silicon cold-electron bolometers test"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "main title",
"classname": "main title",
"schemeid": "dnet:dataCite_title",
"schemename": "dnet:dataCite_title"
},
"value": "test test 123 test"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "main title",
"classname": "main title",
"schemeid": "dnet:dataCite_title",
"schemename": "dnet:dataCite_title"
},
"value": "omic"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "main title",
"classname": "main title",
"schemeid": "dnet:dataCite_title",
"schemename": "dnet:dataCite_title"
},
"value": "「マキャベリ的知性と心の理論の進化論」 リチャード・バーン, アンドリュー・ホワイトゥン 編/藤田和生, 山下博志, 友永雅巳 監訳"
},
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "sysimport:crosswalk:datasetarchive",
"classname": "sysimport:crosswalk:datasetarchive",
"schemeid": "dnet:provenanceActions",
"schemename": "dnet:provenanceActions"
},
"trust": "0.9"
},
"qualifier": {
"classid": "main title",
"classname": "main title",
"schemeid": "dnet:dataCite_title",
"schemename": "dnet:dataCite_title"
},
"value": "-"
}
]
}

View File

@ -3,13 +3,9 @@ package eu.dnetlib.dhp.oa.graph.hostedbymap
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.oa.graph.hostedbymap.SparkPrepareHostedByInfoToApply.{joinResHBM, prepareResultInfo, toEntityInfo}
import eu.dnetlib.dhp.oa.graph.hostedbymap.model.EntityInfo
import eu.dnetlib.dhp.schema.oaf.{Datasource, OpenAccessRoute, Publication}
import javax.management.openmbean.OpenMBeanAttributeInfo
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession}
import org.json4s
import org.json4s.DefaultFormats
import eu.dnetlib.dhp.schema.common.ModelConstants
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
import org.junit.jupiter.api.Test

View File

@ -4,10 +4,9 @@ import eu.dnetlib.dhp.schema.oaf.Datasource
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession}
import org.json4s.DefaultFormats
import org.junit.jupiter.api.Assertions.{assertNotNull, assertTrue}
import org.junit.jupiter.api.Test
import org.junit.jupiter.api.Assertions._
import org.json4s.jackson.Serialization.write
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
class TestPreprocess extends java.io.Serializable{

View File

@ -37,7 +37,8 @@ class ScholixGraphTest extends AbstractVocabularyTest{
val input = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/scholix/result.json")).mkString
val res =SparkResolveRelation.extractPidsFromRecord(input)
assertNotNull(res)
assertTrue(res._2.size == 2)
assertEquals(1,res._2.size)
}

View File

@ -0,0 +1,158 @@
package eu.dnetlib.dhp.oa.provision.model;
import java.util.Set;
import com.google.common.collect.Sets;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.*;
import scala.Serializable;
public class XmlInstance implements Serializable {
public static final AccessRight UNKNOWN_ACCESS_RIGHT;
static {
UNKNOWN_ACCESS_RIGHT = new AccessRight();
UNKNOWN_ACCESS_RIGHT.setClassid(ModelConstants.UNKNOWN);
UNKNOWN_ACCESS_RIGHT.setClassname(ModelConstants.UNKNOWN);
UNKNOWN_ACCESS_RIGHT.setSchemeid(ModelConstants.DNET_ACCESS_MODES);
UNKNOWN_ACCESS_RIGHT.setSchemename(ModelConstants.DNET_ACCESS_MODES);
}
private String url;
private AccessRight accessright;
private Set<KeyValue> collectedfrom = Sets.newHashSet();
private Set<KeyValue> hostedby = Sets.newHashSet();
private Set<Qualifier> instancetype = Sets.newHashSet();
private Set<String> license = Sets.newHashSet();
// other research products specifc
private Set<String> distributionlocation = Sets.newHashSet();
private Set<StructuredProperty> pid = Sets.newHashSet();
private Set<StructuredProperty> alternateIdentifier = Sets.newHashSet();
private Set<String> dateofacceptance = Sets.newHashSet();
// ( article | book ) processing charges. Defined here to cope with possible wrongly typed
// results
private String processingchargeamount;
// currency - alphabetic code describe in ISO-4217. Defined here to cope with possible wrongly
// typed results
private String processingchargecurrency;
private Set<Qualifier> refereed = Sets.newHashSet();; // peer-review status
public String getUrl() {
return url;
}
public void setUrl(String url) {
this.url = url;
}
public AccessRight getAccessright() {
return accessright;
}
public void setAccessright(AccessRight accessright) {
this.accessright = accessright;
}
public Set<KeyValue> getCollectedfrom() {
return collectedfrom;
}
public void setCollectedfrom(Set<KeyValue> collectedfrom) {
this.collectedfrom = collectedfrom;
}
public Set<KeyValue> getHostedby() {
return hostedby;
}
public void setHostedby(Set<KeyValue> hostedby) {
this.hostedby = hostedby;
}
public Set<Qualifier> getInstancetype() {
return instancetype;
}
public void setInstancetype(Set<Qualifier> instancetype) {
this.instancetype = instancetype;
}
public Set<String> getLicense() {
return license;
}
public void setLicense(Set<String> license) {
this.license = license;
}
public Set<String> getDistributionlocation() {
return distributionlocation;
}
public void setDistributionlocation(Set<String> distributionlocation) {
this.distributionlocation = distributionlocation;
}
public Set<StructuredProperty> getPid() {
return pid;
}
public void setPid(Set<StructuredProperty> pid) {
this.pid = pid;
}
public Set<StructuredProperty> getAlternateIdentifier() {
return alternateIdentifier;
}
public void setAlternateIdentifier(Set<StructuredProperty> alternateIdentifier) {
this.alternateIdentifier = alternateIdentifier;
}
public Set<String> getDateofacceptance() {
return dateofacceptance;
}
public void setDateofacceptance(Set<String> dateofacceptance) {
this.dateofacceptance = dateofacceptance;
}
public String getProcessingchargeamount() {
return processingchargeamount;
}
public void setProcessingchargeamount(String processingchargeamount) {
this.processingchargeamount = processingchargeamount;
}
public String getProcessingchargecurrency() {
return processingchargecurrency;
}
public void setProcessingchargecurrency(String processingchargecurrency) {
this.processingchargecurrency = processingchargecurrency;
}
public Set<Qualifier> getRefereed() {
return refereed;
}
public void setRefereed(Set<Qualifier> refereed) {
this.refereed = refereed;
}
}

View File

@ -8,11 +8,16 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import javax.swing.text.html.Option;
import org.apache.commons.lang3.StringUtils;
import org.stringtemplate.v4.ST;
import com.google.common.collect.Lists;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.OafEntity;
@ -94,13 +99,15 @@ public class TemplateFactory {
}
public String getInstance(
final String resultId, final List<String> instancemetadata, final List<String> webresources) {
final List<String> instancemetadata, final String url) {
return getTemplate(resources.getInstance())
.add("instanceId", escapeXml(removePrefix(resultId)))
.add("metadata", instancemetadata)
.add(
"webresources",
(webresources != null ? webresources : new ArrayList<String>())
Optional
.ofNullable(url)
.map(u -> Lists.newArrayList(url))
.orElse(Lists.newArrayList())
.stream()
.filter(StringUtils::isNotBlank)
.map(this::getWebResource)

View File

@ -10,24 +10,18 @@ import java.io.IOException;
import java.io.Serializable;
import java.io.StringReader;
import java.io.StringWriter;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.net.URL;
import java.util.*;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import javax.xml.transform.OutputKeys;
import javax.xml.transform.Transformer;
import javax.xml.transform.TransformerConfigurationException;
import javax.xml.transform.TransformerException;
import javax.xml.transform.TransformerFactory;
import javax.xml.transform.*;
import javax.xml.transform.dom.DOMSource;
import javax.xml.transform.stream.StreamResult;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.tuple.ImmutablePair;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.spark.util.LongAccumulator;
import org.dom4j.Document;
import org.dom4j.DocumentException;
@ -49,25 +43,10 @@ import com.mycila.xmltool.XMLTag;
import eu.dnetlib.dhp.oa.provision.model.JoinedEntity;
import eu.dnetlib.dhp.oa.provision.model.RelatedEntity;
import eu.dnetlib.dhp.oa.provision.model.RelatedEntityWrapper;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.MainEntityType;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.Dataset;
import eu.dnetlib.dhp.schema.oaf.Datasource;
import eu.dnetlib.dhp.schema.oaf.ExternalReference;
import eu.dnetlib.dhp.schema.oaf.ExtraInfo;
import eu.dnetlib.dhp.schema.oaf.Instance;
import eu.dnetlib.dhp.schema.oaf.Journal;
import eu.dnetlib.dhp.schema.oaf.KeyValue;
import eu.dnetlib.dhp.schema.oaf.OafEntity;
import eu.dnetlib.dhp.schema.oaf.Organization;
import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct;
import eu.dnetlib.dhp.schema.oaf.Project;
import eu.dnetlib.dhp.schema.oaf.Publication;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.oa.provision.model.XmlInstance;
import eu.dnetlib.dhp.schema.common.*;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.Software;
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory;
public class XmlRecordFactory implements Serializable {
@ -1112,37 +1091,64 @@ public class XmlRecordFactory implements Serializable {
if (MainEntityType.result.toString().equals(ModelSupport.getMainType(entityType))) {
final List<Instance> instances = ((Result) entity).getInstance();
if (instances != null) {
for (final Instance instance : ((Result) entity).getInstance()) {
groupInstancesByUrl(((Result) entity).getInstance()).forEach(instance -> {
final List<String> fields = Lists.newArrayList();
if (instance.getAccessright() != null && !instance.getAccessright().isBlank()) {
fields
.add(XmlSerializationUtils.mapQualifier("accessright", instance.getAccessright()));
}
if (instance.getCollectedfrom() != null && kvNotBlank(instance.getCollectedfrom())) {
if (instance.getCollectedfrom() != null) {
fields
.add(XmlSerializationUtils.mapKeyValue("collectedfrom", instance.getCollectedfrom()));
.addAll(
instance
.getCollectedfrom()
.stream()
.filter(cf -> kvNotBlank(cf))
.map(cf -> XmlSerializationUtils.mapKeyValue("collectedfrom", cf))
.collect(Collectors.toList()));
}
if (instance.getHostedby() != null && kvNotBlank(instance.getHostedby())) {
fields.add(XmlSerializationUtils.mapKeyValue("hostedby", instance.getHostedby()));
}
if (instance.getDateofacceptance() != null
&& isNotBlank(instance.getDateofacceptance().getValue())) {
if (instance.getHostedby() != null) {
fields
.add(
XmlSerializationUtils
.asXmlElement("dateofacceptance", instance.getDateofacceptance().getValue()));
.addAll(
instance
.getHostedby()
.stream()
.filter(hb -> kvNotBlank(hb))
.map(hb -> XmlSerializationUtils.mapKeyValue("hostedby", hb))
.collect(Collectors.toList()));
}
if (instance.getInstancetype() != null && !instance.getInstancetype().isBlank()) {
if (instance.getDateofacceptance() != null) {
fields
.add(XmlSerializationUtils.mapQualifier("instancetype", instance.getInstancetype()));
.addAll(
instance
.getDateofacceptance()
.stream()
.filter(d -> isNotBlank(d))
.map(d -> XmlSerializationUtils.asXmlElement("dateofacceptance", d))
.collect(Collectors.toList()));
}
if (isNotBlank(instance.getDistributionlocation())) {
if (instance.getInstancetype() != null) {
fields
.add(
XmlSerializationUtils
.asXmlElement("distributionlocation", instance.getDistributionlocation()));
.addAll(
instance
.getInstancetype()
.stream()
.filter(t -> !t.isBlank())
.map(t -> XmlSerializationUtils.mapQualifier("instancetype", t))
.collect(Collectors.toList()));
}
if (instance.getDistributionlocation() != null) {
fields
.addAll(
instance
.getDistributionlocation()
.stream()
.filter(d -> isNotBlank(d))
.map(d -> XmlSerializationUtils.asXmlElement("distributionlocation", d))
.collect(Collectors.toList()));
}
if (instance.getPid() != null) {
fields
@ -1165,32 +1171,50 @@ public class XmlRecordFactory implements Serializable {
.collect(Collectors.toList()));
}
if (instance.getRefereed() != null && !instance.getRefereed().isBlank()) {
if (instance.getRefereed() != null) {
fields
.add(XmlSerializationUtils.mapQualifier("refereed", instance.getRefereed()));
.addAll(
instance
.getRefereed()
.stream()
.filter(Objects::nonNull)
.filter(r -> !r.isBlank())
.map(r -> XmlSerializationUtils.mapQualifier("refereed", r))
.collect(Collectors.toList()));
}
if (instance.getProcessingchargeamount() != null
&& isNotBlank(instance.getProcessingchargeamount().getValue())) {
&& isNotBlank(instance.getProcessingchargeamount())) {
fields
.add(
XmlSerializationUtils
.asXmlElement(
"processingchargeamount", instance.getProcessingchargeamount().getValue()));
"processingchargeamount", instance.getProcessingchargeamount()));
}
if (instance.getProcessingchargecurrency() != null
&& isNotBlank(instance.getProcessingchargecurrency().getValue())) {
&& isNotBlank(instance.getProcessingchargecurrency())) {
fields
.add(
XmlSerializationUtils
.asXmlElement(
"processingchargecurrency", instance.getProcessingchargecurrency().getValue()));
"processingchargecurrency", instance.getProcessingchargecurrency()));
}
if (instance.getLicense() != null) {
fields
.addAll(
instance
.getLicense()
.stream()
.filter(d -> isNotBlank(d))
.map(d -> XmlSerializationUtils.asXmlElement("license", d))
.collect(Collectors.toList()));
}
children
.add(
templateFactory
.getInstance(instance.getHostedby().getKey(), fields, instance.getUrl()));
}
.getInstance(fields, instance.getUrl()));
});
}
final List<ExternalReference> ext = ((Result) entity).getExternalReference();
if (ext != null) {
@ -1234,6 +1258,97 @@ public class XmlRecordFactory implements Serializable {
return children;
}
private Stream<XmlInstance> groupInstancesByUrl(List<Instance> instance) {
return instance
.stream()
.filter(i -> Objects.nonNull(i.getUrl()))
.map(i -> {
i
.setUrl(
i
.getUrl()
.stream()
.filter(this::isValidUrl)
.collect(Collectors.toList()));
return i;
})
.filter(
i -> Optional
.ofNullable(i.getUrl())
.map(u -> !u.isEmpty())
.orElse(false))
.map(this::pickByUrl)
.collect(Collectors.groupingBy(ImmutablePair::getLeft))
.values()
.stream()
.filter(Objects::nonNull)
.map(this::mergeInstances);
}
private boolean isValidUrl(String url) {
try {
new URL(url).toURI();
return true;
} catch (Exception e) {
return false;
}
}
private ImmutablePair<String, Instance> pickByUrl(Instance i) {
return new ImmutablePair<>(i.getUrl().get(0), i);
}
private XmlInstance mergeInstances(List<ImmutablePair<String, Instance>> instances) {
final XmlInstance instance = new XmlInstance();
instance.setUrl(instances.get(0).getLeft());
instance
.setAccessright(
instances
.stream()
.map(Pair::getValue)
.map(Instance::getAccessright)
.min(new AccessRightComparator<AccessRight>())
.orElse(XmlInstance.UNKNOWN_ACCESS_RIGHT));
instances.forEach(p -> {
final Instance i = p.getRight();
instance.getCollectedfrom().add(i.getCollectedfrom());
instance.getHostedby().add(i.getHostedby());
instance.getInstancetype().add(i.getInstancetype());
instance.getRefereed().add(i.getRefereed());
instance
.setProcessingchargeamount(
Optional.ofNullable(i.getProcessingchargeamount()).map(apc -> apc.getValue()).orElse(null));
instance
.setProcessingchargecurrency(
Optional.ofNullable(i.getProcessingchargecurrency()).map(c -> c.getValue()).orElse(null));
Optional
.ofNullable(i.getPid())
.ifPresent(pid -> instance.getPid().addAll(pid));
Optional
.ofNullable(i.getAlternateIdentifier())
.ifPresent(altId -> instance.getAlternateIdentifier().addAll(altId));
Optional
.ofNullable(i.getDateofacceptance())
.ifPresent(d -> instance.getDateofacceptance().add(d.getValue()));
Optional
.ofNullable(i.getLicense())
.ifPresent(license -> instance.getLicense().add(license.getValue()));
Optional
.ofNullable(i.getDistributionlocation())
.ifPresent(dl -> instance.getDistributionlocation().add(dl));
});
if (instance.getHostedby().size() > 1
&& instance.getHostedby().stream().anyMatch(hb -> ModelConstants.UNKNOWN_REPOSITORY.equals(hb))) {
instance.getHostedby().remove(ModelConstants.UNKNOWN_REPOSITORY);
}
return instance;
}
private boolean isDuplicate(final RelatedEntityWrapper link) {
return ModelConstants.DEDUP.equalsIgnoreCase(link.getRelation().getSubRelType());
}

View File

@ -1,4 +1,4 @@
<instance id="$instanceId$">
<instance>
$metadata:{ it | $it$ }$
$webresources:{ it | $it$ }$
</instance>

View File

@ -12,7 +12,6 @@ import org.apache.commons.io.IOUtils;
import org.dom4j.Document;
import org.dom4j.DocumentException;
import org.dom4j.io.SAXReader;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import com.fasterxml.jackson.databind.DeserializationFeature;
@ -54,18 +53,19 @@ public class XmlRecordFactoryTest {
System.out.println(doc.asXML());
Assertions.assertEquals("0000-0001-9613-6638", doc.valueOf("//creator[@rank = '1']/@orcid"));
Assertions.assertEquals("0000-0001-9613-6639", doc.valueOf("//creator[@rank = '1']/@orcid_pending"));
assertEquals("0000-0001-9613-6638", doc.valueOf("//creator[@rank = '1']/@orcid"));
assertEquals("0000-0001-9613-6639", doc.valueOf("//creator[@rank = '1']/@orcid_pending"));
Assertions.assertEquals("0000-0001-9613-9956", doc.valueOf("//creator[@rank = '2']/@orcid"));
Assertions.assertEquals("", doc.valueOf("//creator[@rank = '2']/@orcid_pending"));
assertEquals("0000-0001-9613-9956", doc.valueOf("//creator[@rank = '2']/@orcid"));
assertEquals("", doc.valueOf("//creator[@rank = '2']/@orcid_pending"));
Assertions.assertEquals("doi", doc.valueOf("//instance/pid/@classid"));
Assertions.assertEquals("10.1109/TED.2018.2853550", doc.valueOf("//instance/pid/text()"));
assertEquals("doi", doc.valueOf("//instance/pid/@classid"));
assertEquals("10.1109/TED.2018.2853550", doc.valueOf("//instance/pid/text()"));
Assertions.assertEquals("doi", doc.valueOf("//instance/alternateidentifier/@classid"));
Assertions.assertEquals("10.5689/LIB.2018.2853550", doc.valueOf("//instance/alternateidentifier/text()"));
// TODO add assertions based of values extracted from the XML record
assertEquals("doi", doc.valueOf("//instance/alternateidentifier/@classid"));
assertEquals("10.5689/LIB.2018.2853550", doc.valueOf("//instance/alternateidentifier/text()"));
assertEquals(3, doc.selectNodes("//instance").size());
}
@Test
@ -96,7 +96,7 @@ public class XmlRecordFactoryTest {
final Document doc = new SAXReader().read(new StringReader(xml));
assertNotNull(doc);
System.out.println(doc.asXML());
Assertions.assertEquals("2021-01-01", doc.valueOf("//validated/@date"));
assertEquals("2021-01-01", doc.valueOf("//validated/@date"));
}
@Test

View File

@ -412,9 +412,800 @@
"value": ""
},
"url": [
"http://juuli.fi/Record/0331473718",
"http://dx.doi.org/10.1109/TED.2018.2853550"
]
},
{
"pid": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1109/TED.2018.2853550"
}
],
"alternateIdentifier": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.5689/LIB.2018.2853550"
}
],
"accessright": {
"classid": "CLOSED",
"classname": "Closed Access",
"schemeid": "dnet:access_modes",
"schemename": "dnet:access_modes"
},
"collectedfrom": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"key": "10|od______3456::b2b9ce8435390bcbfc05f3cae3948567",
"value": "A wonderful repository"
},
"dateofacceptance": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"value": "2020-01-01"
},
"distributionlocation": "",
"hostedby": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"key": "10|openaire____::55045bd2a65019fd8e6741a755395c8c",
"value": "Unknown Repository"
},
"instancetype": {
"classid": "0001",
"classname": "Article",
"schemeid": "dnet:dataCite_resource",
"schemename": "dnet:dataCite_resource"
},
"license": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"value": "CC-BY"
},
"url": [
"http://dx.doi.org/10.1109/TED.2018.2853550"
]
},
{
"pid": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1109/TED.2018.2853550"
}
],
"alternateIdentifier": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.5689/LIB.2018.2853550"
}
],
"accessright": {
"classid": "OPEN",
"classname": "Open Access",
"schemeid": "dnet:access_modes",
"schemename": "dnet:access_modes"
},
"collectedfrom": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"key": "10|od______2367::dfb9c4r4353ghjcbfbnhf3cyu79484rf",
"value": "Another repository"
},
"dateofacceptance": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"value": "2018-01-01"
},
"distributionlocation": "",
"hostedby": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"key": "10|CSC_________::a2b9ce8435390bcbfc05f3cae3948747",
"value": "VIRTA"
},
"instancetype": {
"classid": "0001",
"classname": "Article",
"schemeid": "dnet:dataCite_resource",
"schemename": "dnet:dataCite_resource"
},
"license": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"value": ""
},
"url": [
"http://dx.doi.org/10.1109/TED.2018.2853551"
]
},
{
"pid": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1109/TED.2018.2853550"
}
],
"alternateIdentifier": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.5689/LIB.2018.2853550"
}
],
"accessright": {
"classid": "OPEN",
"classname": "Open Access",
"schemeid": "dnet:access_modes",
"schemename": "dnet:access_modes"
},
"collectedfrom": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"key": "10|od______2367::dfb9c4r4353ghjcbfbnhf3cyu79484rf",
"value": "Another repository"
},
"dateofacceptance": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"value": "2018-01-01"
},
"distributionlocation": "",
"hostedby": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"key": "10|openaire____::55045bd2a65019fd8e6741a755395c8c",
"value": "Unknown Repository"
},
"instancetype": {
"classid": "0001",
"classname": "Article",
"schemeid": "dnet:dataCite_resource",
"schemename": "dnet:dataCite_resource"
},
"license": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"value": ""
},
"url": [
"http://dx.doi.org/10.1109/TED.2018.2853552",
"http://dx.doi.org/10.1109/TED.2018.2853554"
]
},
{
"pid": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1109/TED.2018.2853550"
}
],
"alternateIdentifier": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.5689/LIB.2018.2853550"
}
],
"accessright": {
"classid": "OPEN",
"classname": "Open Access",
"schemeid": "dnet:access_modes",
"schemename": "dnet:access_modes"
},
"collectedfrom": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"key": "10|od______2367::dfb9c4r4353ghjcbfbnhf3cyu79484rf",
"value": "Another repository"
},
"dateofacceptance": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"value": "2018-01-01"
},
"distributionlocation": "",
"hostedby": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"key": "10|openaire____::55045bd2a65019fd8e6741a755395c8c",
"value": "Unknown Repository"
},
"instancetype": {
"classid": "0001",
"classname": "Article",
"schemeid": "dnet:dataCite_resource",
"schemename": "dnet:dataCite_resource"
},
"license": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"value": ""
},
"url": [
]
},
{
"pid": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1109/TED.2018.2853550"
}
],
"alternateIdentifier": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.5689/LIB.2018.2853550"
}
],
"accessright": {
"classid": "OPEN",
"classname": "Open Access",
"schemeid": "dnet:access_modes",
"schemename": "dnet:access_modes"
},
"collectedfrom": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"key": "10|od______2367::dfb9c4r4353ghjcbfbnhf3cyu79484rf",
"value": "Another repository"
},
"dateofacceptance": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"value": "2018-01-01"
},
"distributionlocation": "",
"hostedby": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"key": "10|openaire____::55045bd2a65019fd8e6741a755395c8c",
"value": "Unknown Repository"
},
"instancetype": {
"classid": "0001",
"classname": "Article",
"schemeid": "dnet:dataCite_resource",
"schemename": "dnet:dataCite_resource"
},
"license": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"value": ""
},
"url": [
""
]
},
{
"pid": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.1109/TED.2018.2853550"
}
],
"alternateIdentifier": [
{
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"qualifier": {
"classid": "doi",
"classname": "doi",
"schemeid": "dnet:pid_types",
"schemename": "dnet:pid_types"
},
"value": "10.5689/LIB.2018.2853550"
}
],
"accessright": {
"classid": "OPEN",
"classname": "Open Access",
"schemeid": "dnet:access_modes",
"schemename": "dnet:access_modes"
},
"collectedfrom": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"key": "10|od______2367::dfb9c4r4353ghjcbfbnhf3cyu79484rf",
"value": "Another repository"
},
"dateofacceptance": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"value": "2018-01-01"
},
"distributionlocation": "",
"hostedby": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"key": "10|openaire____::55045bd2a65019fd8e6741a755395c8c",
"value": "Unknown Repository"
},
"instancetype": {
"classid": "0001",
"classname": "Article",
"schemeid": "dnet:dataCite_resource",
"schemename": "dnet:dataCite_resource"
},
"license": {
"dataInfo": {
"deletedbyinference": false,
"inferenceprovenance": "",
"inferred": false,
"invisible": false,
"provenanceaction": {
"classid": "",
"classname": "",
"schemeid": "",
"schemename": ""
},
"trust": ""
},
"value": ""
},
"url": [
"asdasd://not a URL"
]
}
],
"journal": {