Merge branch 'beta' of https://code-repo.d4science.org/D-Net/dnet-hadoop into beta
This commit is contained in:
commit
016337a0f9
|
@ -13,6 +13,8 @@ import java.util.stream.Collectors;
|
|||
import java.util.stream.Stream;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
|
||||
import com.github.sisyphsu.dateparser.DateParserUtils;
|
||||
import com.google.common.collect.Lists;
|
||||
|
@ -23,8 +25,6 @@ import eu.dnetlib.dhp.schema.common.ModelConstants;
|
|||
import eu.dnetlib.dhp.schema.common.ModelSupport;
|
||||
import eu.dnetlib.dhp.schema.oaf.*;
|
||||
import me.xuender.unidecode.Unidecode;
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
|
||||
public class GraphCleaningFunctions extends CleaningFunctions {
|
||||
|
||||
|
|
|
@ -27,7 +27,8 @@ object SparkCreateBaselineDataFrame {
|
|||
def requestBaseLineUpdatePage(maxFile: String): List[(String, String)] = {
|
||||
val data = requestPage("https://ftp.ncbi.nlm.nih.gov/pubmed/updatefiles/")
|
||||
|
||||
val result = data.linesWithSeparators.map(l =>l.stripLineEnd)
|
||||
val result = data.linesWithSeparators
|
||||
.map(l => l.stripLineEnd)
|
||||
.filter(l => l.startsWith("<a href="))
|
||||
.map { l =>
|
||||
val end = l.lastIndexOf("\">")
|
||||
|
|
|
@ -63,7 +63,9 @@ class BioScholixTest extends AbstractVocabularyTest {
|
|||
val records: String = Source
|
||||
.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/bio/pubmed_dump"))
|
||||
.mkString
|
||||
val r: List[Oaf] = records.linesWithSeparators.map(l =>l.stripLineEnd).toList
|
||||
val r: List[Oaf] = records.linesWithSeparators
|
||||
.map(l => l.stripLineEnd)
|
||||
.toList
|
||||
.map(s => mapper.readValue(s, classOf[PMArticle]))
|
||||
.map(a => PubMedToOaf.convert(a, vocabularies))
|
||||
assertEquals(10, r.size)
|
||||
|
@ -173,9 +175,10 @@ class BioScholixTest extends AbstractVocabularyTest {
|
|||
val records: String = Source
|
||||
.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/bio/pdb_dump"))
|
||||
.mkString
|
||||
records.linesWithSeparators.map(l =>l.stripLineEnd).foreach(s => assertTrue(s.nonEmpty))
|
||||
records.linesWithSeparators.map(l => l.stripLineEnd).foreach(s => assertTrue(s.nonEmpty))
|
||||
|
||||
val result: List[Oaf] = records.linesWithSeparators.map(l =>l.stripLineEnd).toList.flatMap(o => BioDBToOAF.pdbTOOaf(o))
|
||||
val result: List[Oaf] =
|
||||
records.linesWithSeparators.map(l => l.stripLineEnd).toList.flatMap(o => BioDBToOAF.pdbTOOaf(o))
|
||||
|
||||
assertTrue(result.nonEmpty)
|
||||
result.foreach(r => assertNotNull(r))
|
||||
|
@ -194,9 +197,10 @@ class BioScholixTest extends AbstractVocabularyTest {
|
|||
val records: String = Source
|
||||
.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/bio/uniprot_dump"))
|
||||
.mkString
|
||||
records.linesWithSeparators.map(l =>l.stripLineEnd).foreach(s => assertTrue(s.nonEmpty))
|
||||
records.linesWithSeparators.map(l => l.stripLineEnd).foreach(s => assertTrue(s.nonEmpty))
|
||||
|
||||
val result: List[Oaf] = records.linesWithSeparators.map(l =>l.stripLineEnd).toList.flatMap(o => BioDBToOAF.uniprotToOAF(o))
|
||||
val result: List[Oaf] =
|
||||
records.linesWithSeparators.map(l => l.stripLineEnd).toList.flatMap(o => BioDBToOAF.uniprotToOAF(o))
|
||||
|
||||
assertTrue(result.nonEmpty)
|
||||
result.foreach(r => assertNotNull(r))
|
||||
|
@ -239,9 +243,10 @@ class BioScholixTest extends AbstractVocabularyTest {
|
|||
val records: String = Source
|
||||
.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/bio/crossref_links"))
|
||||
.mkString
|
||||
records.linesWithSeparators.map(l =>l.stripLineEnd).foreach(s => assertTrue(s.nonEmpty))
|
||||
records.linesWithSeparators.map(l => l.stripLineEnd).foreach(s => assertTrue(s.nonEmpty))
|
||||
|
||||
val result: List[Oaf] = records.linesWithSeparators.map(l =>l.stripLineEnd).map(s => BioDBToOAF.crossrefLinksToOaf(s)).toList
|
||||
val result: List[Oaf] =
|
||||
records.linesWithSeparators.map(l => l.stripLineEnd).map(s => BioDBToOAF.crossrefLinksToOaf(s)).toList
|
||||
|
||||
assertNotNull(result)
|
||||
assertTrue(result.nonEmpty)
|
||||
|
@ -276,14 +281,17 @@ class BioScholixTest extends AbstractVocabularyTest {
|
|||
getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/bio/scholix_resolved")
|
||||
)
|
||||
.mkString
|
||||
records.linesWithSeparators.map(l =>l.stripLineEnd).foreach(s => assertTrue(s.nonEmpty))
|
||||
records.linesWithSeparators.map(l => l.stripLineEnd).foreach(s => assertTrue(s.nonEmpty))
|
||||
|
||||
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
|
||||
|
||||
val l: List[ScholixResolved] = records.linesWithSeparators.map(l =>l.stripLineEnd).map { input =>
|
||||
lazy val json = parse(input)
|
||||
json.extract[ScholixResolved]
|
||||
}.toList
|
||||
val l: List[ScholixResolved] = records.linesWithSeparators
|
||||
.map(l => l.stripLineEnd)
|
||||
.map { input =>
|
||||
lazy val json = parse(input)
|
||||
json.extract[ScholixResolved]
|
||||
}
|
||||
.toList
|
||||
|
||||
val result: List[Oaf] = l.map(s => BioDBToOAF.scholixResolvedToOAF(s))
|
||||
|
||||
|
|
|
@ -37,12 +37,24 @@ public class SubscriptionUtils {
|
|||
}
|
||||
|
||||
public static boolean verifyDateRange(final long date, final String min, final String max) {
|
||||
|
||||
long from = 0;
|
||||
long to = Long.MAX_VALUE;
|
||||
|
||||
try {
|
||||
return date >= DateUtils.parseDate(min, "yyyy-MM-dd").getTime()
|
||||
&& date < DateUtils.parseDate(max, "yyyy-MM-dd").getTime() + ONE_DAY;
|
||||
from = min != null ? DateUtils.parseDate(min, "yyyy-MM-dd").getTime() : 0;
|
||||
} catch (final ParseException e) {
|
||||
return false;
|
||||
from = 0;
|
||||
}
|
||||
|
||||
try {
|
||||
to = max != null ? DateUtils.parseDate(max, "yyyy-MM-dd").getTime() + ONE_DAY : Long.MAX_VALUE;
|
||||
} catch (final ParseException e) {
|
||||
to = Long.MAX_VALUE;
|
||||
}
|
||||
|
||||
return date >= from && date < to;
|
||||
|
||||
}
|
||||
|
||||
public static boolean verifyExact(final String s1, final String s2) {
|
||||
|
|
|
@ -41,6 +41,18 @@ public class SubscriptionUtilsTest {
|
|||
|
||||
assertTrue(SubscriptionUtils.verifyDateRange(date, "2010-01-01", "2011-01-01"));
|
||||
assertFalse(SubscriptionUtils.verifyDateRange(date, "2020-01-01", "2021-01-01"));
|
||||
|
||||
assertTrue(SubscriptionUtils.verifyDateRange(date, "2010-01-01", "NULL"));
|
||||
assertTrue(SubscriptionUtils.verifyDateRange(date, "2010-01-01", null));
|
||||
assertTrue(SubscriptionUtils.verifyDateRange(date, "NULL", "2011-01-01"));
|
||||
assertTrue(SubscriptionUtils.verifyDateRange(date, null, "2011-01-01"));
|
||||
assertTrue(SubscriptionUtils.verifyDateRange(date, "NULL", "NULL"));
|
||||
assertTrue(SubscriptionUtils.verifyDateRange(date, null, null));
|
||||
|
||||
assertFalse(SubscriptionUtils.verifyDateRange(date, "2020-01-01", null));
|
||||
assertFalse(SubscriptionUtils.verifyDateRange(date, "2020-01-01", "NULL"));
|
||||
assertFalse(SubscriptionUtils.verifyDateRange(date, null, "2005-01-01"));
|
||||
assertFalse(SubscriptionUtils.verifyDateRange(date, "NULL", "2005-01-01"));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -309,6 +309,8 @@ case object Crossref2Oaf {
|
|||
result
|
||||
}
|
||||
|
||||
|
||||
|
||||
def generateAuhtor(given: String, family: String, orcid: String, index: Int): Author = {
|
||||
val a = new Author
|
||||
a.setName(given)
|
||||
|
@ -370,10 +372,57 @@ case object Crossref2Oaf {
|
|||
case dataset: Dataset => convertDataset(dataset)
|
||||
}
|
||||
|
||||
|
||||
val doisReference:List[String] = for {
|
||||
JObject(reference_json) <- json \ "reference"
|
||||
JField("DOI", JString(doi_json)) <- reference_json
|
||||
} yield doi_json
|
||||
|
||||
|
||||
|
||||
if (doisReference!= null && doisReference.nonEmpty) {
|
||||
val citation_relations:List[Relation] = generateCitationRelations(doisReference, result)
|
||||
resultList = resultList ::: citation_relations
|
||||
}
|
||||
resultList = resultList ::: List(result)
|
||||
resultList
|
||||
}
|
||||
|
||||
|
||||
|
||||
private def createCiteRelation(source:Result, targetPid:String, targetPidType:String) :List[Relation] = {
|
||||
|
||||
|
||||
val targetId = IdentifierFactory.idFromPid("50",targetPidType, targetPid, true)
|
||||
|
||||
val from = new Relation
|
||||
from.setSource(source.getId)
|
||||
from.setTarget(targetId)
|
||||
from.setRelType(ModelConstants.RESULT_RESULT)
|
||||
from.setRelClass(ModelConstants.CITES)
|
||||
from.setSubRelType(ModelConstants.CITATION)
|
||||
from.setCollectedfrom(source.getCollectedfrom)
|
||||
from.setDataInfo(source.getDataInfo)
|
||||
from.setLastupdatetimestamp(source.getLastupdatetimestamp)
|
||||
|
||||
|
||||
val to = new Relation
|
||||
to.setTarget(source.getId)
|
||||
to.setSource(targetId)
|
||||
to.setRelType(ModelConstants.RESULT_RESULT)
|
||||
to.setRelClass(ModelConstants.IS_CITED_BY)
|
||||
to.setSubRelType(ModelConstants.CITATION)
|
||||
to.setCollectedfrom(source.getCollectedfrom)
|
||||
to.setDataInfo(source.getDataInfo)
|
||||
to.setLastupdatetimestamp(source.getLastupdatetimestamp)
|
||||
|
||||
List(from,to)
|
||||
}
|
||||
|
||||
def generateCitationRelations(dois:List[String], result:Result):List[Relation] = {
|
||||
dois.flatMap(d => createCiteRelation(result, d, "doi"))
|
||||
}
|
||||
|
||||
def mappingFunderToRelations(
|
||||
funders: List[mappingFunder],
|
||||
sourceId: String,
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -1,9 +1,14 @@
|
|||
package eu.dnetlib.dhp.doiboost.crossref
|
||||
|
||||
import eu.dnetlib.dhp.schema.common.ModelConstants
|
||||
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.json4s
|
||||
import org.json4s.JsonAST.{JField, JObject, JString}
|
||||
import org.json4s.{DefaultFormats, JValue}
|
||||
import org.json4s.jackson.JsonMethods
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.slf4j.{Logger, LoggerFactory}
|
||||
|
@ -31,13 +36,13 @@ class CrossrefMappingTest {
|
|||
.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/funder_doi"))
|
||||
.mkString
|
||||
|
||||
for (line <- funder_doi.linesWithSeparators.map(l =>l.stripLineEnd)) {
|
||||
for (line <- funder_doi.linesWithSeparators.map(l => l.stripLineEnd)) {
|
||||
val json = template.replace("%s", line)
|
||||
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
|
||||
assertTrue(resultList.nonEmpty)
|
||||
checkRelation(resultList)
|
||||
}
|
||||
for (line <- funder_name.linesWithSeparators.map(l =>l.stripLineEnd)) {
|
||||
for (line <- funder_name.linesWithSeparators.map(l => l.stripLineEnd)) {
|
||||
val json = template.replace("%s", line)
|
||||
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
|
||||
assertTrue(resultList.nonEmpty)
|
||||
|
@ -109,6 +114,44 @@ class CrossrefMappingTest {
|
|||
|
||||
}
|
||||
|
||||
|
||||
private def parseJson(input:String):JValue = {
|
||||
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
|
||||
lazy val json: json4s.JValue = JsonMethods.parse(input)
|
||||
|
||||
json
|
||||
}
|
||||
|
||||
@Test
|
||||
def testCitationRelations():Unit = {
|
||||
val json = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/doiboost/crossref/publication_license_embargo.json")).mkString
|
||||
|
||||
|
||||
assertNotNull(json)
|
||||
assertFalse(json.isEmpty)
|
||||
|
||||
val result:List[Oaf] = Crossref2Oaf.convert(json)
|
||||
|
||||
assertTrue(result.nonEmpty)
|
||||
|
||||
|
||||
val j = parseJson(json)
|
||||
|
||||
val doisReference: List[String] = for {
|
||||
JObject(reference_json) <- j \ "reference"
|
||||
JField("DOI", JString(doi_json)) <- reference_json
|
||||
} yield doi_json
|
||||
|
||||
|
||||
|
||||
val relationList:List[Relation] = result.filter(s => s.isInstanceOf[Relation]).map(r=> r.asInstanceOf[Relation]).filter(r => r.getSubRelType.equalsIgnoreCase(ModelConstants.CITATION))
|
||||
|
||||
assertNotNull(relationList)
|
||||
assertFalse(relationList.isEmpty)
|
||||
|
||||
assertEquals(doisReference.size*2, relationList.size)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testEmptyTitle(): Unit = {
|
||||
val json = Source
|
||||
|
|
|
@ -25,9 +25,11 @@ class MappingORCIDToOAFTest {
|
|||
.mkString
|
||||
assertNotNull(json)
|
||||
assertFalse(json.isEmpty)
|
||||
json.linesWithSeparators.map(l =>l.stripLineEnd).foreach(s => {
|
||||
assertNotNull(ORCIDToOAF.extractValueFromInputString(s))
|
||||
})
|
||||
json.linesWithSeparators
|
||||
.map(l => l.stripLineEnd)
|
||||
.foreach(s => {
|
||||
assertNotNull(ORCIDToOAF.extractValueFromInputString(s))
|
||||
})
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -22,7 +22,7 @@ class UnpayWallMappingTest {
|
|||
.mkString
|
||||
|
||||
var i: Int = 0
|
||||
for (line <- Ilist.linesWithSeparators.map(l =>l.stripLineEnd)) {
|
||||
for (line <- Ilist.linesWithSeparators.map(l => l.stripLineEnd)) {
|
||||
val p = UnpayWallToOAF.convertToOAF(line)
|
||||
|
||||
if (p != null) {
|
||||
|
@ -43,7 +43,7 @@ class UnpayWallMappingTest {
|
|||
i = i + 1
|
||||
}
|
||||
|
||||
val l = Ilist.linesWithSeparators.map(l =>l.stripLineEnd).next()
|
||||
val l = Ilist.linesWithSeparators.map(l => l.stripLineEnd).next()
|
||||
|
||||
val item = UnpayWallToOAF.convertToOAF(l)
|
||||
|
||||
|
|
|
@ -5,7 +5,6 @@ import java.io.IOException;
|
|||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.Dataset;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
@ -27,6 +26,7 @@ import org.slf4j.LoggerFactory;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import eu.dnetlib.dhp.oa.graph.clean.country.CleanCountrySparkJob;
|
||||
import eu.dnetlib.dhp.schema.oaf.Dataset;
|
||||
import eu.dnetlib.dhp.schema.oaf.Publication;
|
||||
|
||||
public class CleanCountryTest {
|
||||
|
@ -151,41 +151,40 @@ public class CleanCountryTest {
|
|||
@Test
|
||||
public void testDatasetClean() throws Exception {
|
||||
final String sourcePath = getClass()
|
||||
.getResource("/eu/dnetlib/dhp/oa/graph/clean/dataset_clean_country.json")
|
||||
.getPath();
|
||||
.getResource("/eu/dnetlib/dhp/oa/graph/clean/dataset_clean_country.json")
|
||||
.getPath();
|
||||
|
||||
spark
|
||||
.read()
|
||||
.textFile(sourcePath)
|
||||
.map(
|
||||
(MapFunction<String, Dataset>) r -> OBJECT_MAPPER.readValue(r, Dataset.class),
|
||||
Encoders.bean(Dataset.class))
|
||||
.write()
|
||||
.json(workingDir.toString() + "/dataset");
|
||||
.read()
|
||||
.textFile(sourcePath)
|
||||
.map(
|
||||
(MapFunction<String, Dataset>) r -> OBJECT_MAPPER.readValue(r, Dataset.class),
|
||||
Encoders.bean(Dataset.class))
|
||||
.write()
|
||||
.json(workingDir.toString() + "/dataset");
|
||||
|
||||
CleanCountrySparkJob.main(new String[] {
|
||||
"--isSparkSessionManaged", Boolean.FALSE.toString(),
|
||||
"--inputPath", workingDir.toString() + "/dataset",
|
||||
"-graphTableClassName", Dataset.class.getCanonicalName(),
|
||||
"-workingDir", workingDir.toString() + "/working",
|
||||
"-country", "NL",
|
||||
"-verifyParam", "10.17632",
|
||||
"-collectedfrom", "NARCIS",
|
||||
"-hostedBy", getClass()
|
||||
"--isSparkSessionManaged", Boolean.FALSE.toString(),
|
||||
"--inputPath", workingDir.toString() + "/dataset",
|
||||
"-graphTableClassName", Dataset.class.getCanonicalName(),
|
||||
"-workingDir", workingDir.toString() + "/working",
|
||||
"-country", "NL",
|
||||
"-verifyParam", "10.17632",
|
||||
"-collectedfrom", "NARCIS",
|
||||
"-hostedBy", getClass()
|
||||
.getResource("/eu/dnetlib/dhp/oa/graph/clean/hostedBy")
|
||||
.getPath()
|
||||
});
|
||||
|
||||
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
|
||||
JavaRDD<Dataset> tmp = sc
|
||||
.textFile(workingDir.toString() + "/dataset")
|
||||
.map(item -> OBJECT_MAPPER.readValue(item, Dataset.class));
|
||||
.textFile(workingDir.toString() + "/dataset")
|
||||
.map(item -> OBJECT_MAPPER.readValue(item, Dataset.class));
|
||||
|
||||
Assertions.assertEquals(1, tmp.count());
|
||||
|
||||
Assertions.assertEquals(0, tmp.first().getCountry().size());
|
||||
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -53,7 +53,8 @@ class ResolveEntitiesTest extends Serializable {
|
|||
def generateUpdates(spark: SparkSession): Unit = {
|
||||
val template = Source.fromInputStream(this.getClass.getResourceAsStream("updates")).mkString
|
||||
|
||||
val pids: List[String] = template.linesWithSeparators.map(l =>l.stripLineEnd)
|
||||
val pids: List[String] = template.linesWithSeparators
|
||||
.map(l => l.stripLineEnd)
|
||||
.map { id =>
|
||||
val r = new Result
|
||||
r.setId(id.toLowerCase.trim)
|
||||
|
@ -127,7 +128,7 @@ class ResolveEntitiesTest extends Serializable {
|
|||
entities.foreach { e =>
|
||||
val template = Source.fromInputStream(this.getClass.getResourceAsStream(s"$e")).mkString
|
||||
spark
|
||||
.createDataset(spark.sparkContext.parallelize(template.linesWithSeparators.map(l =>l.stripLineEnd).toList))
|
||||
.createDataset(spark.sparkContext.parallelize(template.linesWithSeparators.map(l => l.stripLineEnd).toList))
|
||||
.as[String]
|
||||
.write
|
||||
.option("compression", "gzip")
|
||||
|
@ -264,7 +265,8 @@ class ResolveEntitiesTest extends Serializable {
|
|||
Source
|
||||
.fromInputStream(this.getClass.getResourceAsStream(s"publication"))
|
||||
.mkString
|
||||
.linesWithSeparators.map(l =>l.stripLineEnd)
|
||||
.linesWithSeparators
|
||||
.map(l => l.stripLineEnd)
|
||||
.next(),
|
||||
classOf[Publication]
|
||||
)
|
||||
|
|
|
@ -47,7 +47,7 @@ class ScholixGraphTest extends AbstractVocabularyTest {
|
|||
val inputRelations = Source
|
||||
.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/oaf_to_summary"))
|
||||
.mkString
|
||||
val items = inputRelations.linesWithSeparators.map(l =>l.stripLineEnd).toList
|
||||
val items = inputRelations.linesWithSeparators.map(l => l.stripLineEnd).toList
|
||||
assertNotNull(items)
|
||||
items.foreach(i => assertTrue(i.nonEmpty))
|
||||
val result =
|
||||
|
@ -69,7 +69,8 @@ class ScholixGraphTest extends AbstractVocabularyTest {
|
|||
getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/merge_result_scholix")
|
||||
)
|
||||
.mkString
|
||||
val result: List[(Relation, ScholixSummary)] = inputRelations.linesWithSeparators.map(l =>l.stripLineEnd)
|
||||
val result: List[(Relation, ScholixSummary)] = inputRelations.linesWithSeparators
|
||||
.map(l => l.stripLineEnd)
|
||||
.sliding(2)
|
||||
.map(s => (s.head, s(1)))
|
||||
.map(p => (mapper.readValue(p._1, classOf[Relation]), mapper.readValue(p._2, classOf[ScholixSummary])))
|
||||
|
|
Loading…
Reference in New Issue