forked from antonis.lempesis/dnet-hadoop
Merge branch 'stable_ids' of https://code-repo.d4science.org/D-Net/dnet-hadoop into stable_ids
This commit is contained in:
commit
2dd5449c13
|
@ -1,6 +1,11 @@
|
||||||
package eu.dnetlib.doiboost.crossref
|
package eu.dnetlib.doiboost.crossref
|
||||||
|
|
||||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||||
|
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.{SparkConf, SparkContext}
|
||||||
import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}
|
import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}
|
||||||
import org.json4s
|
import org.json4s
|
||||||
|
@ -17,13 +22,6 @@ object GenerateCrossrefDataset {
|
||||||
|
|
||||||
implicit val mrEncoder: Encoder[CrossrefDT] = Encoders.kryo[CrossrefDT]
|
implicit val mrEncoder: Encoder[CrossrefDT] = Encoders.kryo[CrossrefDT]
|
||||||
|
|
||||||
def extractDump(input:String):List[String] = {
|
|
||||||
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
|
|
||||||
lazy val json: json4s.JValue = parse(input)
|
|
||||||
|
|
||||||
val a = (json \ "items").extract[JArray]
|
|
||||||
a.arr.map(s => compact(render(s)))
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
def crossrefElement(meta: String): CrossrefDT = {
|
def crossrefElement(meta: String): CrossrefDT = {
|
||||||
|
@ -44,7 +42,7 @@ object GenerateCrossrefDataset {
|
||||||
val targetPath = parser.get("targetPath")
|
val targetPath = parser.get("targetPath")
|
||||||
|
|
||||||
val spark: SparkSession = SparkSession.builder().config(conf)
|
val spark: SparkSession = SparkSession.builder().config(conf)
|
||||||
.appName(GenerateCrossrefDataset.getClass.getSimpleName)
|
.appName(UnpackCrtossrefEntries.getClass.getSimpleName)
|
||||||
.master(master)
|
.master(master)
|
||||||
.getOrCreate()
|
.getOrCreate()
|
||||||
val sc: SparkContext = spark.sparkContext
|
val sc: SparkContext = spark.sparkContext
|
||||||
|
@ -52,19 +50,14 @@ object GenerateCrossrefDataset {
|
||||||
import spark.implicits._
|
import spark.implicits._
|
||||||
|
|
||||||
|
|
||||||
def extractDump(input:String):List[String] = {
|
val tmp : RDD[String] = sc.textFile(sourcePath,6000)
|
||||||
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
|
|
||||||
lazy val json: json4s.JValue = parse(input)
|
|
||||||
|
|
||||||
val a = (json \ "items").extract[JArray]
|
spark.createDataset(tmp)
|
||||||
a.arr.map(s => compact(render(s)))
|
.map(entry => crossrefElement(entry))
|
||||||
}
|
.write.mode(SaveMode.Overwrite).save(targetPath)
|
||||||
|
// .map(meta => crossrefElement(meta))
|
||||||
|
// .toDS.as[CrossrefDT]
|
||||||
sc.wholeTextFiles(sourcePath,6000).flatMap(d =>extractDump(d._2))
|
// .write.mode(SaveMode.Overwrite).save(targetPath)
|
||||||
.map(meta => crossrefElement(meta))
|
|
||||||
.toDS()//.as[CrossrefDT]
|
|
||||||
.write.mode(SaveMode.Overwrite).save(targetPath)
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -4,9 +4,9 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||||
import eu.dnetlib.dhp.schema.oaf
|
import eu.dnetlib.dhp.schema.oaf
|
||||||
import eu.dnetlib.dhp.schema.oaf.{Oaf, Publication, Relation, Dataset => OafDataset}
|
import eu.dnetlib.dhp.schema.oaf.{Oaf, Publication, Relation, Dataset => OafDataset}
|
||||||
import org.apache.commons.io.IOUtils
|
import org.apache.commons.io.IOUtils
|
||||||
import org.apache.hadoop.io.{IntWritable, Text}
|
|
||||||
import org.apache.spark.SparkConf
|
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.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||||
import org.slf4j.{Logger, LoggerFactory}
|
import org.slf4j.{Logger, LoggerFactory}
|
||||||
|
|
||||||
|
@ -17,6 +17,7 @@ object SparkMapDumpIntoOAF {
|
||||||
|
|
||||||
def main(args: Array[String]): Unit = {
|
def main(args: Array[String]): Unit = {
|
||||||
|
|
||||||
|
implicit val mrEncoder: Encoder[CrossrefDT] = Encoders.kryo[CrossrefDT]
|
||||||
|
|
||||||
val logger: Logger = LoggerFactory.getLogger(SparkMapDumpIntoOAF.getClass)
|
val logger: Logger = LoggerFactory.getLogger(SparkMapDumpIntoOAF.getClass)
|
||||||
val conf: SparkConf = new SparkConf()
|
val conf: SparkConf = new SparkConf()
|
||||||
|
@ -35,7 +36,6 @@ object SparkMapDumpIntoOAF {
|
||||||
implicit val mapEncoderDatasets: Encoder[oaf.Dataset] = Encoders.kryo[OafDataset]
|
implicit val mapEncoderDatasets: Encoder[oaf.Dataset] = Encoders.kryo[OafDataset]
|
||||||
|
|
||||||
val targetPath = parser.get("targetPath")
|
val targetPath = parser.get("targetPath")
|
||||||
import spark.implicits._
|
|
||||||
|
|
||||||
spark.read.load(parser.get("sourcePath")).as[CrossrefDT]
|
spark.read.load(parser.get("sourcePath")).as[CrossrefDT]
|
||||||
.flatMap(k => Crossref2Oaf.convert(k.json))
|
.flatMap(k => Crossref2Oaf.convert(k.json))
|
||||||
|
|
|
@ -0,0 +1,54 @@
|
||||||
|
package eu.dnetlib.doiboost.crossref
|
||||||
|
|
||||||
|
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||||
|
import org.apache.hadoop.io.compress.GzipCodec
|
||||||
|
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
|
||||||
|
import org.json4s.jackson.JsonMethods.{compact, parse, render}
|
||||||
|
import org.slf4j.{Logger, LoggerFactory}
|
||||||
|
|
||||||
|
import scala.io.Source
|
||||||
|
|
||||||
|
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)
|
||||||
|
|
||||||
|
val a = (json \ "items").extract[JArray]
|
||||||
|
a.arr.map(s => compact(render(s)))
|
||||||
|
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
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)
|
||||||
|
parser.parseArgument(args)
|
||||||
|
val master = parser.get("master")
|
||||||
|
val sourcePath = parser.get("sourcePath")
|
||||||
|
val targetPath = parser.get("targetPath")
|
||||||
|
|
||||||
|
val spark: SparkSession = SparkSession.builder().config(conf)
|
||||||
|
.appName(UnpackCrtossrefEntries.getClass.getSimpleName)
|
||||||
|
.master(master)
|
||||||
|
.getOrCreate()
|
||||||
|
val sc: SparkContext = spark.sparkContext
|
||||||
|
|
||||||
|
sc.wholeTextFiles(sourcePath,6000).flatMap(d =>extractDump(d._2))
|
||||||
|
.saveAsTextFile(targetPath, classOf[GzipCodec])
|
||||||
|
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -24,7 +24,7 @@
|
||||||
|
|
||||||
</parameters>
|
</parameters>
|
||||||
|
|
||||||
<start to="ImportCrossRef"/>
|
<start to="GenerateCrossrefDataset"/>
|
||||||
|
|
||||||
<kill name="Kill">
|
<kill name="Kill">
|
||||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||||
|
@ -40,11 +40,37 @@
|
||||||
<arg>--workingPath</arg><arg>${crossrefDumpPath}</arg>
|
<arg>--workingPath</arg><arg>${crossrefDumpPath}</arg>
|
||||||
<arg>--outputPath</arg><arg>${workingDir}/files/</arg>
|
<arg>--outputPath</arg><arg>${workingDir}/files/</arg>
|
||||||
</java>
|
</java>
|
||||||
<ok to="generateCrossrefDataset"/>
|
<ok to="UnpackCrossrefEntries"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
|
||||||
<action name="generateCrossrefDataset">
|
<action name="UnpackCrossrefEntries">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn-cluster</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>SparkGenerateCrossrefDataset</name>
|
||||||
|
<class>eu.dnetlib.doiboost.crossref.UnpackCrtossrefEntries</class>
|
||||||
|
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||||
|
<spark-opts>
|
||||||
|
--executor-memory=${sparkExecutorMemory}
|
||||||
|
--executor-cores=${sparkExecutorCores}
|
||||||
|
--driver-memory=${sparkDriverMemory}
|
||||||
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
|
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||||
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
|
</spark-opts>
|
||||||
|
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||||
|
<arg>--sourcePath</arg><arg>${crossrefDumpPath}/files</arg>
|
||||||
|
<arg>--targetPath</arg><arg>${inputPathCrossref}/crossref_ds</arg>
|
||||||
|
|
||||||
|
</spark>
|
||||||
|
<ok to="GenerateCrossrefDataset"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
<action name="GenerateCrossrefDataset">
|
||||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
<master>yarn-cluster</master>
|
<master>yarn-cluster</master>
|
||||||
<mode>cluster</mode>
|
<mode>cluster</mode>
|
||||||
|
@ -62,31 +88,31 @@
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||||
<arg>--sourcePath</arg><arg>${workingDir}/files</arg>
|
<arg>--sourcePath</arg><arg>${inputPathCrossref}/crossref_ds</arg>
|
||||||
<arg>--targetPath</arg><arg>${inputPathCrossref}/crossref_ds_updated</arg>
|
<arg>--targetPath</arg><arg>${inputPathCrossref}/crossref_ds_updates</arg>
|
||||||
|
|
||||||
</spark>
|
</spark>
|
||||||
<ok to="removeFiles"/>
|
|
||||||
<error to="Kill"/>
|
|
||||||
</action>
|
|
||||||
|
|
||||||
<action name="removeFiles">
|
|
||||||
<fs>
|
|
||||||
<delete path="${workingDir}/files"/>
|
|
||||||
</fs>
|
|
||||||
<ok to="renameDataset"/>
|
|
||||||
<error to="Kill"/>
|
|
||||||
</action>
|
|
||||||
|
|
||||||
<action name="renameDataset">
|
|
||||||
<fs>
|
|
||||||
<delete path="${inputPathCrossref}/crossref_ds"/>
|
|
||||||
<move source="${inputPathCrossref}/crossref_ds_updated"
|
|
||||||
target="${inputPathCrossref}/crossref_ds"/>
|
|
||||||
</fs>
|
|
||||||
<ok to="End"/>
|
<ok to="End"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
|
||||||
|
<!-- <action name="removeFiles">-->
|
||||||
|
<!-- <fs>-->
|
||||||
|
<!-- <delete path="${workingDir}/files"/>-->
|
||||||
|
<!-- </fs>-->
|
||||||
|
<!-- <ok to="renameDataset"/>-->
|
||||||
|
<!-- <error to="Kill"/>-->
|
||||||
|
<!-- </action>-->
|
||||||
|
|
||||||
|
<!-- <action name="renameDataset">-->
|
||||||
|
<!-- <fs>-->
|
||||||
|
<!-- <delete path="${inputPathCrossref}/crossref_ds"/>-->
|
||||||
|
<!-- <move source="${inputPathCrossref}/crossref_ds_updated"-->
|
||||||
|
<!-- target="${inputPathCrossref}/crossref_ds"/>-->
|
||||||
|
<!-- </fs>-->
|
||||||
|
<!-- <ok to="End"/>-->
|
||||||
|
<!-- <error to="Kill"/>-->
|
||||||
|
<!-- </action>-->
|
||||||
|
|
||||||
<end name="End"/>
|
<end name="End"/>
|
||||||
</workflow-app>
|
</workflow-app>
|
|
@ -142,6 +142,32 @@
|
||||||
<arg>--workingPath</arg><arg>${crossrefDumpPath}</arg>
|
<arg>--workingPath</arg><arg>${crossrefDumpPath}</arg>
|
||||||
<arg>--outputPath</arg><arg>${crossrefDumpPath}/files/</arg>
|
<arg>--outputPath</arg><arg>${crossrefDumpPath}/files/</arg>
|
||||||
</java>
|
</java>
|
||||||
|
<ok to="UnpackCrossrefEntries"/>
|
||||||
|
<error to="Kill"/>
|
||||||
|
</action>
|
||||||
|
|
||||||
|
<action name="UnpackCrossrefEntries">
|
||||||
|
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||||
|
<master>yarn-cluster</master>
|
||||||
|
<mode>cluster</mode>
|
||||||
|
<name>SparkUnpackCrossrefEntries</name>
|
||||||
|
<class>eu.dnetlib.doiboost.crossref.UnpackCrtossrefEntries</class>
|
||||||
|
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||||
|
<spark-opts>
|
||||||
|
--executor-memory=${sparkExecutorMemory}
|
||||||
|
--executor-cores=${sparkExecutorCores}
|
||||||
|
--driver-memory=${sparkDriverMemory}
|
||||||
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
|
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||||
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
|
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||||
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
|
</spark-opts>
|
||||||
|
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||||
|
<arg>--sourcePath</arg><arg>${crossrefDumpPath}/files</arg>
|
||||||
|
<arg>--targetPath</arg><arg>${crossrefDumpPath}/crossref_unpack/</arg>
|
||||||
|
|
||||||
|
</spark>
|
||||||
<ok to="GenerateCrossrefDataset"/>
|
<ok to="GenerateCrossrefDataset"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
</action>
|
</action>
|
||||||
|
@ -154,9 +180,9 @@
|
||||||
<class>eu.dnetlib.doiboost.crossref.GenerateCrossrefDataset</class>
|
<class>eu.dnetlib.doiboost.crossref.GenerateCrossrefDataset</class>
|
||||||
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
<jar>dhp-doiboost-${projectVersion}.jar</jar>
|
||||||
<spark-opts>
|
<spark-opts>
|
||||||
--executor-memory=${sparkExecutorMemory}
|
--executor-memory=7G
|
||||||
--executor-cores=2
|
--executor-cores=2
|
||||||
--driver-memory=${sparkDriverMemory}
|
--driver-memory=7G
|
||||||
--conf spark.sql.shuffle.partitions=3840
|
--conf spark.sql.shuffle.partitions=3840
|
||||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||||
|
@ -164,7 +190,7 @@
|
||||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||||
</spark-opts>
|
</spark-opts>
|
||||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||||
<arg>--sourcePath</arg><arg>${crossrefDumpPath}/files/</arg>
|
<arg>--sourcePath</arg><arg>${crossrefDumpPath}/crossref_unpack/</arg>
|
||||||
<arg>--targetPath</arg><arg>${inputPathCrossref}/crossref_ds</arg>
|
<arg>--targetPath</arg><arg>${inputPathCrossref}/crossref_ds</arg>
|
||||||
|
|
||||||
</spark>
|
</spark>
|
||||||
|
@ -174,7 +200,8 @@
|
||||||
|
|
||||||
<action name="removeFiles">
|
<action name="removeFiles">
|
||||||
<fs>
|
<fs>
|
||||||
<delete path="${crossrefDumpPath}/files"/>
|
<!-- <delete path="${crossrefDumpPath}/files"/>-->
|
||||||
|
<delete path="${crossrefDumpPath}/crossref_unpack/"/>
|
||||||
</fs>
|
</fs>
|
||||||
<ok to="ResetMagWorkingPath"/>
|
<ok to="ResetMagWorkingPath"/>
|
||||||
<error to="Kill"/>
|
<error to="Kill"/>
|
||||||
|
|
|
@ -395,4 +395,71 @@ class CrossrefMappingTest {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testSetDateOfAcceptanceCrossRef2Oaf(): Unit = {
|
||||||
|
|
||||||
|
val json = Source.fromInputStream(getClass.getResourceAsStream("dump_file.json")).mkString
|
||||||
|
assertNotNull(json)
|
||||||
|
|
||||||
|
assertFalse(json.isEmpty);
|
||||||
|
|
||||||
|
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
|
||||||
|
|
||||||
|
assertTrue(resultList.nonEmpty)
|
||||||
|
|
||||||
|
val items = resultList.filter(p => p.isInstanceOf[Publication])
|
||||||
|
|
||||||
|
assert(items.nonEmpty)
|
||||||
|
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)
|
||||||
|
//
|
||||||
|
// })
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1 @@
|
||||||
|
{"indexed":{"date-parts":[[2020,4,10]],"date-time":"2020-04-10T00:38:19Z","timestamp":1586479099385},"reference-count":0,"publisher":"American Medical Association (AMA)","issue":"4","content-domain":{"domain":[],"crossmark-restriction":false},"short-container-title":["Archives of Internal Medicine"],"published-print":{"date-parts":[[2006,2,27]]},"DOI":"10.1001/.389","type":"journal-article","created":{"date-parts":[[2006,2,27]],"date-time":"2006-02-27T21:28:23Z","timestamp":1141075703000},"page":"389-390","source":"Crossref","is-referenced-by-count":0,"title":["Decision Making at the Fringe of Evidence: Take What You Can Get"],"prefix":"10.1001","volume":"166","author":[{"given":"N. F.","family":"Col","affiliation":[]}],"member":"10","container-title":["Archives of Internal Medicine"],"original-title":[],"deposited":{"date-parts":[[2007,2,13]],"date-time":"2007-02-13T20:56:13Z","timestamp":1171400173000},"score":null,"subtitle":[],"short-title":[],"issued":{"date-parts":[[2006,2,27]]},"references-count":0,"URL":"http://dx.doi.org/10.1001/.389","relation":{},"ISSN":["0003-9926"],"issn-type":[{"value":"0003-9926","type":"print"}],"subject":["Internal Medicine"]}
|
Loading…
Reference in New Issue