>, P>) value -> {
Optional p = Optional.ofNullable(value._1()).map(Tuple2::_2);
Optional b = Optional.ofNullable(value._2()).map(Tuple2::_2);
+
+ if (p.orElse((P) b.orElse((B) DATASOURCE)) instanceof Datasource) {
+ return mergeDatasource(p, b);
+ }
switch (priority) {
default:
case "BETA":
@@ -119,6 +133,36 @@ public class MergeGraphSparkJob {
.json(outputPath);
}
+ /**
+ * Datasources involved in the merge operation doesn't obey to the infra precedence policy, but relies on a custom
+ * behaviour that, given two datasources from beta and prod returns the one from prod with the highest
+ * compatibility among the two.
+ *
+ * @param p datasource from PROD
+ * @param b datasource from BETA
+ * @param
Datasource class type from PROD
+ * @param Datasource class type from BETA
+ * @return the datasource from PROD with the highest compatibility level.
+ */
+ protected static
P mergeDatasource(Optional
p, Optional b) {
+ if (p.isPresent() & !b.isPresent()) {
+ return p.get();
+ }
+ if (b.isPresent() & !p.isPresent()) {
+ return (P) b.get();
+ }
+ if (!b.isPresent() & !p.isPresent()) {
+ return null; // unlikely, at least one should be produced by the join operation
+ }
+
+ Datasource dp = (Datasource) p.get();
+ Datasource db = (Datasource) b.get();
+
+ List list = Arrays.asList(dp.getOpenairecompatibility(), db.getOpenairecompatibility());
+ dp.setOpenairecompatibility(Collections.min(list, new DatasourceCompatibilityComparator()));
+ return (P) dp;
+ }
+
private static P mergeWithPriorityToPROD(Optional
p, Optional b) {
if (b.isPresent() & !p.isPresent()) {
return (P) b.get();
diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java
index 1e7b56ee9..adf7b92be 100644
--- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java
+++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java
@@ -38,13 +38,11 @@ import java.io.IOException;
import java.sql.Array;
import java.sql.ResultSet;
import java.sql.SQLException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Date;
-import java.util.List;
+import java.util.*;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.function.Predicate;
+import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
@@ -197,7 +195,14 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i
final Datasource ds = new Datasource();
ds.setId(createOpenaireId(10, rs.getString("datasourceid"), true));
- ds.setOriginalId(Arrays.asList((String[]) rs.getArray("identities").getArray()));
+ ds
+ .setOriginalId(
+ Arrays
+ .asList(
+ (String[]) rs.getArray("identities").getArray())
+ .stream()
+ .filter(StringUtils::isNotBlank)
+ .collect(Collectors.toList()));
ds
.setCollectedfrom(
listKeyValues(
@@ -243,7 +248,13 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i
ds.setCertificates(field(rs.getString("certificates"), info));
ds.setPolicies(new ArrayList<>()); // The sql query returns an empty array
ds
- .setJournal(prepareJournal(rs.getString("officialname"), rs.getString("journal"), info)); // Journal
+ .setJournal(
+ journal(
+ rs.getString("officialname"),
+ rs.getString("issnPrinted"),
+ rs.getString("issnOnline"),
+ rs.getString("issnLinking"),
+ info)); // Journal
ds.setDataInfo(info);
ds.setLastupdatetimestamp(lastUpdateTimestamp);
@@ -567,21 +578,15 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i
return res;
}
- private Journal prepareJournal(final String name, final String sj, final DataInfo info) {
- if (StringUtils.isNotBlank(sj)) {
- final String[] arr = sj.split("@@@");
- if (arr.length == 3) {
- final String issn = StringUtils.isNotBlank(arr[0]) ? arr[0].trim() : null;
- final String eissn = StringUtils.isNotBlank(arr[1]) ? arr[1].trim() : null;
+ private Journal prepareJournal(final ResultSet rs, final DataInfo info) throws SQLException {
+ if (Objects.isNull(rs)) {
+ return null;
+ } else {
- final String lissn = StringUtils.isNotBlank(arr[2]) ? arr[2].trim() : null;
-
- if (issn != null || eissn != null || lissn != null) {
- return journal(name, issn, eissn, lissn, null, null, null, null, null, null, null, info);
- }
- }
+ return journal(
+ rs.getString("officialname"), rs.getString("issnPrinted"), rs.getString("issnOnline"),
+ rs.getString("issnLinking"), info);
}
- return null;
}
@Override
diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/OafMapperUtils.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/OafMapperUtils.java
index 63db13b8f..84b29e3d4 100644
--- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/OafMapperUtils.java
+++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/OafMapperUtils.java
@@ -153,6 +153,27 @@ public class OafMapperUtils {
return p;
}
+ public static Journal journal(
+ final String name,
+ final String issnPrinted,
+ final String issnOnline,
+ final String issnLinking,
+ final DataInfo dataInfo) {
+ return journal(
+ name,
+ issnPrinted,
+ issnOnline,
+ issnLinking,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ null,
+ dataInfo);
+ }
+
public static Journal journal(
final String name,
final String issnPrinted,
diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/EBIAggregator.scala b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/EBIAggregator.scala
index d1bf39475..ee2dbadfd 100644
--- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/EBIAggregator.scala
+++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/sx/ebi/EBIAggregator.scala
@@ -1,4 +1,5 @@
package eu.dnetlib.dhp.sx.ebi
+import eu.dnetlib.dhp.oa.merge.AuthorMerger
import eu.dnetlib.dhp.schema.oaf.{Publication, Relation, Dataset => OafDataset}
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIUnknown}
import org.apache.spark.sql.{Encoder, Encoders}
@@ -14,6 +15,7 @@ object EBIAggregator {
override def reduce(b: OafDataset, a: (String, OafDataset)): OafDataset = {
b.mergeFrom(a._2)
+ b.setAuthor(AuthorMerger.mergeAuthor(a._2.getAuthor, b.getAuthor))
if (b.getId == null)
b.setId(a._2.getId)
b
@@ -22,6 +24,7 @@ object EBIAggregator {
override def merge(wx: OafDataset, wy: OafDataset): OafDataset = {
wx.mergeFrom(wy)
+ wx.setAuthor(AuthorMerger.mergeAuthor(wy.getAuthor, wx.getAuthor))
if(wx.getId == null && wy.getId.nonEmpty)
wx.setId(wy.getId)
wx
@@ -35,8 +38,6 @@ object EBIAggregator {
Encoders.kryo(classOf[OafDataset])
}
-
-
def getDLIUnknownAggregator(): Aggregator[(String, DLIUnknown), DLIUnknown, DLIUnknown] = new Aggregator[(String, DLIUnknown), DLIUnknown, DLIUnknown]{
override def zero: DLIUnknown = new DLIUnknown()
@@ -69,6 +70,7 @@ object EBIAggregator {
override def reduce(b: DLIDataset, a: (String, DLIDataset)): DLIDataset = {
b.mergeFrom(a._2)
+ b.setAuthor(AuthorMerger.mergeAuthor(a._2.getAuthor, b.getAuthor))
if (b.getId == null)
b.setId(a._2.getId)
b
@@ -76,6 +78,7 @@ object EBIAggregator {
override def merge(wx: DLIDataset, wy: DLIDataset): DLIDataset = {
wx.mergeFrom(wy)
+ wx.setAuthor(AuthorMerger.mergeAuthor(wy.getAuthor, wx.getAuthor))
if(wx.getId == null && wy.getId.nonEmpty)
wx.setId(wy.getId)
wx
@@ -96,6 +99,8 @@ object EBIAggregator {
override def reduce(b: DLIPublication, a: (String, DLIPublication)): DLIPublication = {
b.mergeFrom(a._2)
+ b.setAuthor(AuthorMerger.mergeAuthor(a._2.getAuthor, b.getAuthor))
+
if (b.getId == null)
b.setId(a._2.getId)
b
@@ -104,6 +109,7 @@ object EBIAggregator {
override def merge(wx: DLIPublication, wy: DLIPublication): DLIPublication = {
wx.mergeFrom(wy)
+ wx.setAuthor(AuthorMerger.mergeAuthor(wy.getAuthor, wx.getAuthor))
if(wx.getId == null && wy.getId.nonEmpty)
wx.setId(wy.getId)
wx
@@ -124,6 +130,7 @@ object EBIAggregator {
override def reduce(b: Publication, a: (String, Publication)): Publication = {
b.mergeFrom(a._2)
+ b.setAuthor(AuthorMerger.mergeAuthor(a._2.getAuthor, b.getAuthor))
if (b.getId == null)
b.setId(a._2.getId)
b
@@ -132,6 +139,7 @@ object EBIAggregator {
override def merge(wx: Publication, wy: Publication): Publication = {
wx.mergeFrom(wy)
+ wx.setAuthor(AuthorMerger.mergeAuthor(wy.getAuthor, wx.getAuthor))
if(wx.getId == null && wy.getId.nonEmpty)
wx.setId(wy.getId)
wx
@@ -145,7 +153,6 @@ object EBIAggregator {
Encoders.kryo(classOf[Publication])
}
-
def getRelationAggregator(): Aggregator[(String, Relation), Relation, Relation] = new Aggregator[(String, Relation), Relation, Relation]{
override def zero: Relation = new Relation()
@@ -166,10 +173,4 @@ object EBIAggregator {
override def outputEncoder: Encoder[Relation] =
Encoders.kryo(classOf[Relation])
}
-
-
-
-
-
-
}
diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/lib/scripts/postprocessing.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/lib/scripts/postprocessing.sql
index 778e3afd2..ea483a4a7 100644
--- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/lib/scripts/postprocessing.sql
+++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/hive/oozie_app/lib/scripts/postprocessing.sql
@@ -1,10 +1,10 @@
DROP VIEW IF EXISTS ${hiveDbName}.result;
CREATE VIEW IF NOT EXISTS ${hiveDbName}.result as
- select id, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance from ${hiveDbName}.publication p
+ select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance from ${hiveDbName}.publication p
union all
- select id, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance from ${hiveDbName}.dataset d
+ select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance from ${hiveDbName}.dataset d
union all
- select id, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance from ${hiveDbName}.software s
+ select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance from ${hiveDbName}.software s
union all
- select id, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance from ${hiveDbName}.otherresearchproduct o;
+ select id, originalid, dateofcollection, title, publisher, bestaccessright, datainfo, collectedfrom, pid, author, resulttype, language, country, subject, description, dateofacceptance, relevantdate, embargoenddate, resourcetype, context, externalreference, instance from ${hiveDbName}.otherresearchproduct o;
diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryDatasources.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryDatasources.sql
index 7ca672835..f0a4161ab 100644
--- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryDatasources.sql
+++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryDatasources.sql
@@ -84,8 +84,10 @@ SELECT
dc.id AS collectedfromid,
dc.officialname AS collectedfromname,
d.typology||'@@@dnet:datasource_typologies' AS datasourcetype,
- 'sysimport:crosswalk:entityregistry@@@dnet:provenance_actions' AS provenanceaction,
- concat_ws(' @@@ ', d.issn, d.eissn, d.lissn) AS journal
+ 'sysimport:crosswalk:entityregistry@@@dnet:provenance_actions' AS provenanceaction,
+ d.issn AS issnPrinted,
+ d.eissn AS issnOnline,
+ d.lissn AS issnLinking
FROM dsm_datasources d
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJobTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJobTest.java
new file mode 100644
index 000000000..28e8e5abc
--- /dev/null
+++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphSparkJobTest.java
@@ -0,0 +1,84 @@
+
+package eu.dnetlib.dhp.oa.graph.merge;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.io.IOException;
+import java.util.Optional;
+
+import org.apache.commons.io.IOUtils;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import eu.dnetlib.dhp.schema.oaf.Datasource;
+
+public class MergeGraphSparkJobTest {
+
+ private ObjectMapper mapper;
+
+ @BeforeEach
+ public void setUp() {
+ mapper = new ObjectMapper().configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+ }
+
+ @Test
+ public void testMergeDatasources() throws IOException {
+ assertEquals(
+ "openaire-cris_1.1",
+ MergeGraphSparkJob
+ .mergeDatasource(
+ d("datasource_cris.json"),
+ d("datasource_UNKNOWN.json"))
+ .getOpenairecompatibility()
+ .getClassid());
+ assertEquals(
+ "openaire-cris_1.1",
+ MergeGraphSparkJob
+ .mergeDatasource(
+ d("datasource_UNKNOWN.json"),
+ d("datasource_cris.json"))
+ .getOpenairecompatibility()
+ .getClassid());
+ assertEquals(
+ "driver-openaire2.0",
+ MergeGraphSparkJob
+ .mergeDatasource(
+ d("datasource_native.json"),
+ d("datasource_driver-openaire2.0.json"))
+ .getOpenairecompatibility()
+ .getClassid());
+ assertEquals(
+ "driver-openaire2.0",
+ MergeGraphSparkJob
+ .mergeDatasource(
+ d("datasource_driver-openaire2.0.json"),
+ d("datasource_native.json"))
+ .getOpenairecompatibility()
+ .getClassid());
+ assertEquals(
+ "openaire4.0",
+ MergeGraphSparkJob
+ .mergeDatasource(
+ d("datasource_notCompatible.json"),
+ d("datasource_openaire4.0.json"))
+ .getOpenairecompatibility()
+ .getClassid());
+ assertEquals(
+ "notCompatible",
+ MergeGraphSparkJob
+ .mergeDatasource(
+ d("datasource_notCompatible.json"),
+ d("datasource_UNKNOWN.json"))
+ .getOpenairecompatibility()
+ .getClassid());
+ }
+
+ private Optional d(String file) throws IOException {
+ String json = IOUtils.toString(getClass().getResourceAsStream(file));
+ return Optional.of(mapper.readValue(json, Datasource.class));
+ }
+
+}
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java
index 2c10f8f58..af9fadb5a 100644
--- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java
+++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java
@@ -304,9 +304,40 @@ public class MappersTest {
assertValidId(d.getCollectedfrom().get(0).getKey());
assertTrue(StringUtils.isNotBlank(d.getTitle().get(0).getValue()));
assertEquals(1, d.getAuthor().size());
- assertEquals(0, d.getSubject().size());
+ assertEquals(1, d.getSubject().size());
assertEquals(1, d.getInstance().size());
assertEquals(1, d.getPid().size());
+ assertNotNull(d.getInstance().get(0).getUrl());
+ }
+
+ @Test
+ void testClaimFromCrossref() throws IOException {
+ final String xml = IOUtils.toString(getClass().getResourceAsStream("oaf_claim_crossref.xml"));
+ final List list = new OafToOafMapper(vocs, false).processMdRecord(xml);
+
+ System.out.println("***************");
+ System.out.println(new ObjectMapper().writeValueAsString(list));
+ System.out.println("***************");
+
+ final Publication p = (Publication) list.get(0);
+ assertValidId(p.getId());
+ assertValidId(p.getCollectedfrom().get(0).getKey());
+ System.out.println(p.getTitle().get(0).getValue());
+ assertTrue(StringUtils.isNotBlank(p.getTitle().get(0).getValue()));
+ }
+
+ @Test
+ void testODFRecord() throws IOException {
+ final String xml = IOUtils.toString(getClass().getResourceAsStream("odf_record.xml"));
+ List list = new OdfToOafMapper(vocs, false).processMdRecord(xml);
+ System.out.println("***************");
+ System.out.println(new ObjectMapper().writeValueAsString(list));
+ System.out.println("***************");
+ final Dataset p = (Dataset) list.get(0);
+ assertValidId(p.getId());
+ assertValidId(p.getCollectedfrom().get(0).getKey());
+ System.out.println(p.getTitle().get(0).getValue());
+ assertTrue(StringUtils.isNotBlank(p.getTitle().get(0).getValue()));
}
private void assertValidId(final String id) {
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplicationTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplicationTest.java
index 011cc18e6..f663d6095 100644
--- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplicationTest.java
+++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplicationTest.java
@@ -80,9 +80,9 @@ public class MigrateDbEntitiesApplicationTest {
assertEquals(getValueAsString("namespaceprefix", fields), ds.getNamespaceprefix().getValue());
assertEquals(getValueAsString("collectedfromname", fields), ds.getCollectedfrom().get(0).getValue());
assertEquals(getValueAsString("officialname", fields), ds.getJournal().getName());
- assertEquals("2579-5449", ds.getJournal().getIssnPrinted());
- assertEquals("2597-6540", ds.getJournal().getIssnOnline());
- assertEquals(null, ds.getJournal().getIssnLinking());
+ assertEquals(getValueAsString("issnPrinted", fields), ds.getJournal().getIssnPrinted());
+ assertEquals(getValueAsString("issnOnline", fields), ds.getJournal().getIssnOnline());
+ assertEquals(getValueAsString("issnLinking", fields), ds.getJournal().getIssnLinking());
}
@Test
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerAggregationTest.scala b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerAggregationTest.scala
new file mode 100644
index 000000000..4d83057f2
--- /dev/null
+++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerAggregationTest.scala
@@ -0,0 +1,54 @@
+package eu.dnetlib.dhp.sx.graph
+
+import com.fasterxml.jackson.databind.{ObjectMapper, SerializationFeature}
+import eu.dnetlib.dhp.schema.scholexplorer.DLIPublication
+import eu.dnetlib.dhp.sx.ebi.EBIAggregator
+import org.apache.spark.sql.{Dataset, Encoder, Encoders, SparkSession}
+import org.junit.jupiter.api.Assertions._
+import org.junit.jupiter.api.Test
+
+import scala.io.Source
+
+class SparkScholexplorerAggregationTest {
+
+
+ @Test
+ def testFunderRelationshipsMapping(): Unit = {
+ val publications = Source.fromInputStream(getClass.getResourceAsStream("publication.json")).mkString
+
+ var s: List[DLIPublication] = List[DLIPublication]()
+
+ val m: ObjectMapper = new ObjectMapper()
+
+ m.enable(SerializationFeature.INDENT_OUTPUT)
+
+ for (line <- publications.lines) {
+ s = m.readValue(line, classOf[DLIPublication]) :: s
+
+
+ }
+
+
+ implicit val pubEncoder: Encoder[DLIPublication] = Encoders.kryo[DLIPublication]
+ val spark: SparkSession = SparkSession.builder().appName("Test").master("local[*]").getOrCreate()
+
+
+ val ds: Dataset[DLIPublication] = spark.createDataset(spark.sparkContext.parallelize(s)).as[DLIPublication]
+
+ val unique = ds.map(d => (d.getId, d))(Encoders.tuple(Encoders.STRING, pubEncoder))
+ .groupByKey(_._1)(Encoders.STRING)
+ .agg(EBIAggregator.getDLIPublicationAggregator().toColumn)
+ .map(p => p._2)
+
+ val uniquePubs: DLIPublication = unique.first()
+
+ s.foreach(pp => assertFalse(pp.getAuthor.isEmpty))
+
+
+ assertNotNull(uniquePubs.getAuthor)
+ assertFalse(uniquePubs.getAuthor.isEmpty)
+
+
+ }
+
+}
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerGraphImporterTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerGraphImporterTest.java
index ed3b6efdc..ce00466df 100644
--- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerGraphImporterTest.java
+++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/sx/graph/SparkScholexplorerGraphImporterTest.java
@@ -2,4 +2,5 @@
package eu.dnetlib.dhp.sx.graph;
public class SparkScholexplorerGraphImporterTest {
+
}
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_UNKNOWN.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_UNKNOWN.json
new file mode 100644
index 000000000..a01085c8f
--- /dev/null
+++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_UNKNOWN.json
@@ -0,0 +1 @@
+{ "id": "10|274269ac6f3b::2a2e2793b500f3f7b47ef24b1a9277b7", "openairecompatibility": { "classid": "UNKNOWN" }}
\ No newline at end of file
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_cris.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_cris.json
new file mode 100644
index 000000000..6f2b7aa7d
--- /dev/null
+++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_cris.json
@@ -0,0 +1 @@
+{ "id": "10|274269ac6f3b::2a2e2793b500f3f7b47ef24b1a9277b7", "openairecompatibility": { "classid": "openaire-cris_1.1" }}
\ No newline at end of file
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_driver-openaire2.0.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_driver-openaire2.0.json
new file mode 100644
index 000000000..d2e375f55
--- /dev/null
+++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_driver-openaire2.0.json
@@ -0,0 +1 @@
+{ "id": "10|274269ac6f3b::2a2e2793b500f3f7b47ef24b1a9277b7", "openairecompatibility": { "classid": "driver-openaire2.0" }}
\ No newline at end of file
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_hostedby.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_hostedby.json
new file mode 100644
index 000000000..03db887f5
--- /dev/null
+++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_hostedby.json
@@ -0,0 +1 @@
+{ "id": "10|274269ac6f3b::2a2e2793b500f3f7b47ef24b1a9277b7", "openairecompatibility": { "classid": "hostedBy" }}
\ No newline at end of file
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_native.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_native.json
new file mode 100644
index 000000000..7831a3fc3
--- /dev/null
+++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_native.json
@@ -0,0 +1 @@
+{ "id": "10|274269ac6f3b::2a2e2793b500f3f7b47ef24b1a9277b7", "openairecompatibility": { "classid": "native" }}
\ No newline at end of file
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_notCompatible.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_notCompatible.json
new file mode 100644
index 000000000..8dabe5d2c
--- /dev/null
+++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_notCompatible.json
@@ -0,0 +1 @@
+{ "id": "10|274269ac6f3b::2a2e2793b500f3f7b47ef24b1a9277b7", "openairecompatibility": { "classid": "notCompatible" }}
\ No newline at end of file
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_openaire2.0.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_openaire2.0.json
new file mode 100644
index 000000000..e2db47943
--- /dev/null
+++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_openaire2.0.json
@@ -0,0 +1 @@
+{ "id": "10|274269ac6f3b::2a2e2793b500f3f7b47ef24b1a9277b7", "openairecompatibility": { "classid": "openaire2.0" }}
\ No newline at end of file
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_openaire2.0_data.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_openaire2.0_data.json
new file mode 100644
index 000000000..b8480faf0
--- /dev/null
+++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_openaire2.0_data.json
@@ -0,0 +1 @@
+{ "id": "10|274269ac6f3b::2a2e2793b500f3f7b47ef24b1a9277b7", "openairecompatibility": { "classid": "openaire2.0_data" }}
\ No newline at end of file
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_openaire3.0.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_openaire3.0.json
new file mode 100644
index 000000000..43bb0a7a4
--- /dev/null
+++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_openaire3.0.json
@@ -0,0 +1 @@
+{ "id": "10|274269ac6f3b::2a2e2793b500f3f7b47ef24b1a9277b7", "openairecompatibility": { "classid": "openaire3.0" }}
\ No newline at end of file
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_openaire4.0.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_openaire4.0.json
new file mode 100644
index 000000000..7cdba6a4e
--- /dev/null
+++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/merge/datasource_openaire4.0.json
@@ -0,0 +1 @@
+{ "id": "10|274269ac6f3b::2a2e2793b500f3f7b47ef24b1a9277b7", "openairecompatibility": { "classid": "openaire4.0" }}
\ No newline at end of file
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/datasources_resultset_entry.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/datasources_resultset_entry.json
index 0f1da7095..8f8aed3a0 100644
--- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/datasources_resultset_entry.json
+++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/datasources_resultset_entry.json
@@ -228,8 +228,18 @@
"value": "sysimport:crosswalk:entityregistry@@@dnet:provenance_actions"
},
{
- "field": "journal",
+ "field": "issnPrinted",
"type": "string",
- "value": "2579-5449 @@@ 2597-6540 @@@ "
+ "value": "2579-5449"
+ },
+ {
+ "field": "issnOnline",
+ "type": "string",
+ "value": "2579-5448"
+ },
+ {
+ "field": "issnLinking",
+ "type": "string",
+ "value": "2579-5447"
}
]
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_claim_crossref.xml b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_claim_crossref.xml
new file mode 100644
index 000000000..8f69a5e2d
--- /dev/null
+++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_claim_crossref.xml
@@ -0,0 +1,68 @@
+
+
+
+ userclaim___::7f0f7807f17db50e5c2b5c452ccaf06d
+ userclaim___::7f0f7807f17db50e5c2b5c452ccaf06d
+ 2020-08-06T07:04:09.62Z
+
+
+
+
+
+ 2020-08-06T07:20:57.911Z
+ openaire____
+
+
+ A case report of serious haemolysis in a glucose-6-phosphate dehydrogenase-deficient COVID-19 patient receiving hydroxychloroquine
+ Maillart, E.
+ Leemans, S.
+ Van Noten, H.
+ Vandergraesen, T.
+ Mahadeb, B.
+ Salaouatchi, M. T.
+ De Bels, D.
+ Clevenbergh, P.
+
+ http://dx.doi.org/10.1080/23744235.2020.1774644
+
+ Informa UK Limited
+ Crossref
+ Infectious Diseases
+ Microbiology (medical)
+ General Immunology and Microbiology
+ Infectious Diseases
+ General Medicine
+ journal-article
+ 0001
+ 2020-06-04
+
+ UNKNOWN
+
+
+ 10.1080/23744235.2020.1774644
+ Infectious Diseases
+
+
+
+
+ file%3A%2F%2F%2Fsrv%2Fclaims%2Frecords%2Fpublication%2Fcrossref
+
+
+
+
+
+
+ false
+ false
+ 0.9
+
+
+
+
+
\ No newline at end of file
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_nakala.xml b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_nakala.xml
index 105d0c413..7796e4c37 100644
--- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_nakala.xml
+++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_nakala.xml
@@ -1,77 +1,75 @@
+ xmlns:oaf="http://namespace.openaire.eu/oaf"
+ xmlns:oai="http://www.openarchives.org/OAI/2.0/" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
- r3f5b9831893::cca7367159bc3ff90cd2f75bf9dc21c4
- oai:nakala.fr:hdl_11280_847e01df
- 2020-08-01T00:16:24.742Z
+ xmlns:dri="http://www.driver-repository.eu/namespace/dri" xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance">
+ r3f5b9831893::01a497c6c6b44289c52dcdf22b6c0fc0
+ oai:nakala.fr:hdl_11280_50f302c6
+ 2020-10-03T06:06:52.228Z
r3f5b9831893
- oai:nakala.fr:hdl_11280_847e01df
- 2020-06-08T01:01:38Z
- hdl_11280_2b09fc10
- hdl_11280_c1bc48d0
- hdl_11280_57c8db3a
- 2020-08-01T00:31:35.625Z
+ oai:nakala.fr:hdl_11280_50f302c6
+ 2020-09-19T23:56:08Z
+ hdl_11280_96355742
+ hdl_11280_26914437
+ hdl_11280_86561837
+ 2020-10-19T15:39:52.151Z
- 277
- http://hdl.handle.net/11280/847e01df
-
- http://hdl.handle.net/http://hdl.handle.net/11280/847e01df
-
- http://nakala.fr/data/11280/847e01df
-
+ http://nakala.fr/data/11280/50f302c6
+ 11280/50f302c6
+
- DHAAP
+ Desbrosse, Xavier
-
- CVP_Notice277-1 place du Docteur Antoine Béclère _PHO02.jpg
+
+ Les rues Stalingrad en France (1945-2013)
-
- Hôpital Saint-Antoine. Fragment de dalle funéraire trouvée en décembre 1932. Paris (XIIème arr.). Photographie d'Albert Citerne (1876-1970). Plaque de verre, 1932. Département Histoire de l'Architecture et Archéologie de Paris.
- Nfa_1146
- Hôpital Saint-Antoine. Fragment de dalle funéraire trouvée en décembre 1932. Paris (XIIème arr.). Photographie d'Albert Citerne (1876-1970). Plaque de verre, 1932. Département Histoire de l'Architecture et Archéologie de Paris.
+
+ Rues – Noms -- France
+
+
+ Cette carte appartient à la collection « Guerre froide vue d’en bas » élaborée dans le cadre de l’enquête 2009-2013 du réseau des correspondants départementaux de l’IHTP « La Guerre froide vue d’en bas : 1947-1967 », enquête conduite sous la direction de Philippe Buton Professeur d’Histoire contemporaine à l’Université de Reims, d’Olivier Büttner Ingénieur de Recherche IHTP-CNRS et de Michel Hastings, Professeur de Science politique à l’Institut d’Etudes Politiques de Lille.
- Nakala by Huma-Num
-
+ IHTP-CNRS
+
- DHAAP, Pôle Archéologique
+ (CNRS), Institut d'Histoire du Temps Présent (IHTP) - Centre National de la Recherche Scientifique
-
- 1932
+
+ 2013
- StillImage
-
-
-
+ Carte
+
+
+ France
+
+
- http://hdl.handle.net/11280/847e01df
+ 11280/50f302c6
+
0025
- OPEN
+ UNKNOWN
und
+ xmlns:dri="http://www.driver-repository.eu/namespace/dri" xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance">
-
- https%3A%2F%2Fwww.nakala.fr%2Foai_oa%2F11280%2F8892ab4b
- oai:nakala.fr:hdl_11280_847e01df
- 2020-06-08T01:01:38Z
+
+ https%3A%2F%2Fwww.nakala.fr%2Foai_oa%2F11280%2F92c4d30b
+ oai:nakala.fr:hdl_11280_50f302c6
+ 2020-09-19T23:56:08Z
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_record.xml b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_record.xml
new file mode 100644
index 000000000..e2d51a43d
--- /dev/null
+++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/odf_record.xml
@@ -0,0 +1,102 @@
+
+
+
+ r3a507cdacc5::03b31980d9bb3c4609e6005c4a3baba6
+ oai:lindat.mff.cuni.cz:11372/LRT-1844
+ 2020-09-04T14:36:48.411Z
+ r3a507cdacc5
+ oai:lindat.mff.cuni.cz:11372/LRT-1844
+ 2016-12-07T11:10:30Z
+ hdl_11858_00-097C-0000-0007-710A-A
+ hdl_11858_00-097C-0000-0007-710B-8
+ openaire_data
+ 2020-09-04T14:39:16.458Z
+
+
+
+ 11372/LRT-1844
+
+ http://hdl.handle.net/11372/LRT-1844
+
+
+
+ Hercig, Tomáš
+
+
+ Brychcín, Tomáš
+
+
+ Svoboda, Lukáš
+
+
+ Konkol, Michal
+
+
+ Steinberger, Josef
+
+
+
+ Restaurant Reviews CZ ABSA corpus v2
+
+ University of West Bohemia, Department of Computer Science and Engineering
+ 2016
+
+
+ European Commission
+ info:eu-repo/grantAgreement/EC/FP7/630786
+
+
+
+ 2016
+ 2016-12-07T11:10:30Z
+ 2016-12-07T11:10:30Z
+
+ corpus
+
+
+
+
+
+ Restaurant Reviews CZ ABSA - 2.15k reviews with their related target and category
+
+ The work done is described in the paper: https://doi.org/10.13053/CyS-20-3-2469
+
+
+ 11372/LRT-1844
+ 2016-12-07
+ 0021
+ 2016-01-01
+ OPEN
+ http://creativecommons.org/licenses/by-nc-sa/4.0/
+ und
+ corda_______::630786
+
+
+
+
+
+
+ https%3A%2F%2Flindat.mff.cuni.cz%2Frepository%2Foai%2Fopenaire_data
+ oai:lindat.mff.cuni.cz:11372/LRT-1844
+ 2016-12-07T11:10:30Z
+
+
+
+
+ false
+ false
+ 0.9
+
+
+
+
+
diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/sx/graph/publication.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/sx/graph/publication.json
new file mode 100644
index 000000000..539dd5e62
--- /dev/null
+++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/sx/graph/publication.json
@@ -0,0 +1,10 @@
+{"collectedfrom":[{"key":"dli_________::datacite","value":"Datasets in Datacite","dataInfo":null}],"dataInfo":{"invisible":false,"inferred":null,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":null},"lastupdatetimestamp":null,"id":"50|1307198540d2264d839dfd8c9a19f4a7","originalId":["10.3390/w11050916"],"pid":[{"value":"10.3390/w11050916","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":null}],"dateofcollection":"2018-10-28T00:39:04.337Z","dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"author":[{"fullname":"Cao, Qing","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao, Zhenchun","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Yuan, Feifei","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Berndtsson, Ronny","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Xu, Shijie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Gao, Huibin","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao, Jie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"publication","schemename":"publication"},"language":null,"country":null,"subject":[],"title":[{"value":"On the Predictability of Daily Rainfall during Rainy Season over the Huaihe River Basin","qualifier":{"classid":"main title","classname":null,"schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":null}],"relevantdate":[{"value":"2019-05-01","qualifier":{"classid":"date","classname":"date","schemeid":"dnet::date","schemename":"dnet::date"},"dataInfo":null}],"description":[{"value":"In terms of climate change and precipitation, there is large interest in how large-scale climatic features affect regional rainfall amount and rainfall occurrence. Large-scale climate elements need to be downscaled to the regional level for hydrologic applications. Here, a new Nonhomogeneous Hidden Markov Model (NHMM) called the Bayesian-NHMM is presented for downscaling and predicting of multisite daily rainfall during rainy season over the Huaihe River Basin (HRB). The Bayesian-NHMM provides a Bayesian method for parameters estimation. The model avoids the risk to have no solutions for parameter estimation, which often occurs in the traditional NHMM that uses point estimates of parameters. The Bayesian-NHMM accurately captures seasonality and interannual variability of rainfall amount and wet days during the rainy season. The model establishes a link between large-scale meteorological characteristics and local precipitation patterns. It also provides a more stable and efficient method to estimate parameters...","dataInfo":null}],"dateofacceptance":null,"publisher":{"value":"MDPI AG","dataInfo":null},"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":[{"license":null,"accessright":null,"instancetype":null,"hostedby":{"key":"openaire____::1256f046-bf1f-4afc-8b47-d0b147148b18","value":"Unknown Repository","dataInfo":null},"url":["10.3390/w11050916"],"distributionlocation":null,"collectedfrom":null,"dateofacceptance":null,"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"journal":null,"originalObjIdentifier":"datacite____::100bb045f34ea2da81433d0b9ae3afa1","dlicollectedfrom":[{"id":"dli_________::datacite","name":"Datasets in Datacite","completionStatus":"complete","collectionMode":null}],"completionStatus":"complete"}
+{"collectedfrom":[{"key":"dli_________::datacite","value":"Datasets in Datacite","dataInfo":null}],"dataInfo":{"invisible":false,"inferred":null,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":null},"lastupdatetimestamp":null,"id":"50|1307198540d2264d839dfd8c9a19f4a7","originalId":["10.3390/w11050916"],"pid":[{"value":"10.3390/w11050916","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":null}],"dateofcollection":"2018-10-28T00:39:04.337Z","dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"author":[{"fullname":"Cao, Qing","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao, Zhenchun","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Yuan, Feifei","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Berndtsson, Ronny","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Xu, Shijie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Gao, Huibin","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao, Jie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"publication","schemename":"publication"},"language":null,"country":null,"subject":[],"title":[{"value":"On the Predictability of Daily Rainfall during Rainy Season over the Huaihe River Basin","qualifier":{"classid":"main title","classname":null,"schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":null}],"relevantdate":[{"value":"2019-05-01","qualifier":{"classid":"date","classname":"date","schemeid":"dnet::date","schemename":"dnet::date"},"dataInfo":null}],"description":[{"value":"In terms of climate change and precipitation, there is large interest in how large-scale climatic features affect regional rainfall amount and rainfall occurrence. Large-scale climate elements need to be downscaled to the regional level for hydrologic applications. Here, a new Nonhomogeneous Hidden Markov Model (NHMM) called the Bayesian-NHMM is presented for downscaling and predicting of multisite daily rainfall during rainy season over the Huaihe River Basin (HRB). The Bayesian-NHMM provides a Bayesian method for parameters estimation. The model avoids the risk to have no solutions for parameter estimation, which often occurs in the traditional NHMM that uses point estimates of parameters. The Bayesian-NHMM accurately captures seasonality and interannual variability of rainfall amount and wet days during the rainy season. The model establishes a link between large-scale meteorological characteristics and local precipitation patterns. It also provides a more stable and efficient method to estimate parameters in the model. These results suggest that prediction of daily precipitation could be improved by the suggested new Bayesian-NHMM method, which can be helpful for water resources management and research on climate change.","dataInfo":null}],"dateofacceptance":null,"publisher":{"value":"MDPI AG","dataInfo":null},"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":[{"license":null,"accessright":null,"instancetype":null,"hostedby":{"key":"openaire____::1256f046-bf1f-4afc-8b47-d0b147148b18","value":"Unknown Repository","dataInfo":null},"url":["10.3390/w11050916"],"distributionlocation":null,"collectedfrom":null,"dateofacceptance":null,"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"journal":null,"originalObjIdentifier":"datacite____::100bb045f34ea2da81433d0b9ae3afa1","dlicollectedfrom":[{"id":"dli_________::datacite","name":"Datasets in Datacite","completionStatus":"complete","collectionMode":null}],"completionStatus":"complete"}
+{"collectedfrom":[{"key":"dli_________::datacite","value":"Datasets in Datacite","dataInfo":null}],"dataInfo":{"invisible":false,"inferred":null,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":null},"lastupdatetimestamp":null,"id":"50|1307198540d2264d839dfd8c9a19f4a7","originalId":["10.3390/w11050916"],"pid":[{"value":"10.3390/w11050916","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":null}],"dateofcollection":"2018-10-28T00:39:04.337Z","dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"author":[{"fullname":"Cao, Qing","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao, Zhenchun","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Yuan, Feifei","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Berndtsson, Ronny","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Xu, Shijie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Gao, Huibin","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao, Jie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"publication","schemename":"publication"},"language":null,"country":null,"subject":[],"title":[{"value":"On the Predictability of Daily Rainfall during Rainy Season over the Huaihe River Basin","qualifier":{"classid":"main title","classname":null,"schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":null}],"relevantdate":[{"value":"2019-05-01","qualifier":{"classid":"date","classname":"date","schemeid":"dnet::date","schemename":"dnet::date"},"dataInfo":null}],"description":[{"value":"In terms of climate change and precipitation, there is large interest in how large-scale climatic features affect regional rainfall amount and rainfall occurrence. Large-scale climate elements need to be downscaled to the regional level for hydrologic applications. Here, a new Nonhomogeneous Hidden Markov Model (NHMM) called the Bayesian-NHMM is presented for downscaling and predicting of multisite daily rainfall during rainy season over the Huaihe River Basin (HRB). The Bayesian-NHMM provides a Bayesian method for parameters estimation. The model avoids the risk to have no solutions for parameter estimation, which often occurs in the traditional NHMM that uses point estimates of parameters. The Bayesian-NHMM accurately captures seasonality and interannual variability of rainfall amount and wet days during the rainy season. The model establishes a link between large-scale meteorological characteristics and local precipitation patterns. It also provides a more stable and efficient method to estimate parameters in the model. These results suggest that prediction of daily precipitation could be improved by the suggested new Bayesian-NHMM method, which can be helpful for water resources management and research on climate change.","dataInfo":null}],"dateofacceptance":null,"publisher":{"value":"MDPI AG","dataInfo":null},"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":[{"license":null,"accessright":null,"instancetype":null,"hostedby":{"key":"openaire____::1256f046-bf1f-4afc-8b47-d0b147148b18","value":"Unknown Repository","dataInfo":null},"url":["10.3390/w11050916"],"distributionlocation":null,"collectedfrom":null,"dateofacceptance":null,"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"journal":null,"originalObjIdentifier":"datacite____::100bb045f34ea2da81433d0b9ae3afa1","dlicollectedfrom":[{"id":"dli_________::datacite","name":"Datasets in Datacite","completionStatus":"complete","collectionMode":null}],"completionStatus":"complete"}
+{"collectedfrom":[{"key":"dli_________::datacite","value":"Datasets in Datacite","dataInfo":null}],"dataInfo":{"invisible":false,"inferred":null,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":null},"lastupdatetimestamp":null,"id":"50|1307198540d2264d839dfd8c9a19f4a7","originalId":["10.3390/w11050916"],"pid":[{"value":"10.3390/w11050916","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":null}],"dateofcollection":"2018-10-28T00:39:04.337Z","dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"author":[{"fullname":"Cao, Qing","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao, Zhenchun","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Yuan, Feifei","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Berndtsson, Ronny","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Xu, Shijie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Gao, Huibin","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao, Jie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"publication","schemename":"publication"},"language":null,"country":null,"subject":[],"title":[{"value":"On the Predictability of Daily Rainfall during Rainy Season over the Huaihe River Basin","qualifier":{"classid":"main title","classname":null,"schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":null}],"relevantdate":[{"value":"2019-05-01","qualifier":{"classid":"date","classname":"date","schemeid":"dnet::date","schemename":"dnet::date"},"dataInfo":null}],"description":[{"value":"In terms of climate change and precipitation, there is large interest in how large-scale climatic features affect regional rainfall amount and rainfall occurrence. Large-scale climate elements need to be downscaled to the regional level for hydrologic applications. Here, a new Nonhomogeneous Hidden Markov Model (NHMM) called the Bayesian-NHMM is presented for downscaling and predicting of multisite daily rainfall during rainy season over the Huaihe River Basin (HRB). The Bayesian-NHMM provides a Bayesian method for parameters estimation. The model avoids the risk to have no solutions for parameter estimation, which often occurs in the traditional NHMM that uses point estimates of parameters. The Bayesian-NHMM accurately captures seasonality and interannual variability of rainfall amount and wet days during the rainy season. The model establishes a link between large-scale meteorological characteristics and local precipitation patterns. It also provides a more stable and efficient method to estimate parameters in the model. These results suggest that prediction of daily precipitation could be improved by the suggested new Bayesian-NHMM method, which can be helpful for water resources management and research on climate change.","dataInfo":null}],"dateofacceptance":null,"publisher":{"value":"MDPI AG","dataInfo":null},"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":[{"license":null,"accessright":null,"instancetype":null,"hostedby":{"key":"openaire____::1256f046-bf1f-4afc-8b47-d0b147148b18","value":"Unknown Repository","dataInfo":null},"url":["10.3390/w11050916"],"distributionlocation":null,"collectedfrom":null,"dateofacceptance":null,"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"journal":null,"originalObjIdentifier":"datacite____::100bb045f34ea2da81433d0b9ae3afa1","dlicollectedfrom":[{"id":"dli_________::datacite","name":"Datasets in Datacite","completionStatus":"complete","collectionMode":null}],"completionStatus":"complete"}
+{"collectedfrom":[{"key":"dli_________::datacite","value":"Datasets in Datacite","dataInfo":null}],"dataInfo":{"invisible":false,"inferred":null,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":null},"lastupdatetimestamp":null,"id":"50|1307198540d2264d839dfd8c9a19f4a7","originalId":["10.3390/w11050916"],"pid":[{"value":"10.3390/w11050916","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":null}],"dateofcollection":"2018-10-28T00:39:04.337Z","dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"author":[{"fullname":"Cao, Qing","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao, Zhenchun","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Yuan, Feifei","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Berndtsson, Ronny","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Xu, Shijie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Gao, Huibin","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao, Jie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"publication","schemename":"publication"},"language":null,"country":null,"subject":[],"title":[{"value":"On the Predictability of Daily Rainfall during Rainy Season over the Huaihe River Basin","qualifier":{"classid":"main title","classname":null,"schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":null}],"relevantdate":[{"value":"2019-05-01","qualifier":{"classid":"date","classname":"date","schemeid":"dnet::date","schemename":"dnet::date"},"dataInfo":null}],"description":[{"value":"In terms of climate change and precipitation, there is large interest in how large-scale climatic features affect regional rainfall amount and rainfall occurrence. Large-scale climate elements need to be downscaled to the regional level for hydrologic applications. Here, a new Nonhomogeneous Hidden Markov Model (NHMM) called the Bayesian-NHMM is presented for downscaling and predicting of multisite daily rainfall during rainy season over the Huaihe River Basin (HRB). The Bayesian-NHMM provides a Bayesian method for parameters estimation. The model avoids the risk to have no solutions for parameter estimation, which often occurs in the traditional NHMM that uses point estimates of parameters. The Bayesian-NHMM accurately captures seasonality and interannual variability of rainfall amount and wet days during the rainy season. The model establishes a link between large-scale meteorological characteristics and local precipitation patterns. It also provides a more stable and efficient method to estimate parameters in the model. These results suggest that prediction of daily precipitation could be improved by the suggested new Bayesian-NHMM method, which can be helpful for water resources management and research on climate change.","dataInfo":null}],"dateofacceptance":null,"publisher":{"value":"MDPI AG","dataInfo":null},"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":[{"license":null,"accessright":null,"instancetype":null,"hostedby":{"key":"openaire____::1256f046-bf1f-4afc-8b47-d0b147148b18","value":"Unknown Repository","dataInfo":null},"url":["10.3390/w11050916"],"distributionlocation":null,"collectedfrom":null,"dateofacceptance":null,"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"journal":null,"originalObjIdentifier":"datacite____::100bb045f34ea2da81433d0b9ae3afa1","dlicollectedfrom":[{"id":"dli_________::datacite","name":"Datasets in Datacite","completionStatus":"complete","collectionMode":null}],"completionStatus":"complete"}
+{"collectedfrom":[{"key":"dli_________::crossref","value":"Crossref","dataInfo":null}],"dataInfo":{"invisible":false,"inferred":null,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":null},"lastupdatetimestamp":null,"id":"50|1307198540d2264d839dfd8c9a19f4a7","originalId":["1307198540d2264d839dfd8c9a19f4a7"],"pid":[{"value":"10.3390/w11050916","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":null}],"dateofcollection":"2020-10-04T14:16:06.105Z","dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"author":[{"fullname":"Cao Qing","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao Zhenchun","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Yuan Feifei","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Berndtsson Ronny","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Xu Shijie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Gao Huibin","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao Jie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"publication","schemename":"publication"},"language":null,"country":null,"subject":[],"title":[{"value":"On the Predictability of Daily Rainfall during Rainy Season over the Huaihe River Basin","qualifier":{"classid":"main title","classname":null,"schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":null}],"relevantdate":[{"value":"2019-05-02T07:15:22Z","qualifier":{"classid":"date","classname":"date","schemeid":"dnet::date","schemename":"dnet::date"},"dataInfo":null}],"description":[{"value":null,"dataInfo":null}],"dateofacceptance":null,"publisher":{"value":"MDPI AG","dataInfo":null},"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":[],"journal":null,"originalObjIdentifier":"dli_resolver::1307198540d2264d839dfd8c9a19f4a7","dlicollectedfrom":[{"id":"dli_________::crossref","name":"Crossref","completionStatus":"complete","collectionMode":"resolved"}],"completionStatus":"complete"}
+{"collectedfrom":[{"key":"dli_________::crossref","value":"Crossref","dataInfo":null}],"dataInfo":{"invisible":false,"inferred":null,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":null},"lastupdatetimestamp":null,"id":"50|1307198540d2264d839dfd8c9a19f4a7","originalId":["1307198540d2264d839dfd8c9a19f4a7"],"pid":[{"value":"10.3390/w11050916","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":null}],"dateofcollection":"2020-09-27T11:39:38.835Z","dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"author":[{"fullname":"Cao Qing","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao Zhenchun","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Yuan Feifei","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Berndtsson Ronny","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Xu Shijie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Gao Huibin","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao Jie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"publication","schemename":"publication"},"language":null,"country":null,"subject":[],"title":[{"value":"On the Predictability of Daily Rainfall during Rainy Season over the Huaihe River Basin","qualifier":{"classid":"main title","classname":null,"schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":null}],"relevantdate":[{"value":"2019-05-02T07:15:22Z","qualifier":{"classid":"date","classname":"date","schemeid":"dnet::date","schemename":"dnet::date"},"dataInfo":null}],"description":[{"value":null,"dataInfo":null}],"dateofacceptance":null,"publisher":{"value":"MDPI AG","dataInfo":null},"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":[],"journal":null,"originalObjIdentifier":"dli_resolver::1307198540d2264d839dfd8c9a19f4a7","dlicollectedfrom":[{"id":"dli_________::crossref","name":"Crossref","completionStatus":"complete","collectionMode":"resolved"}],"completionStatus":"complete"}
+{"collectedfrom":[{"key":"dli_________::crossref","value":"Crossref","dataInfo":null}],"dataInfo":{"invisible":false,"inferred":null,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":null},"lastupdatetimestamp":null,"id":"50|1307198540d2264d839dfd8c9a19f4a7","originalId":["1307198540d2264d839dfd8c9a19f4a7"],"pid":[{"value":"10.3390/w11050916","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":null}],"dateofcollection":"2020-08-30T11:48:49.809Z","dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"author":[{"fullname":"Cao Qing","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao Zhenchun","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Yuan Feifei","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Berndtsson Ronny","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Xu Shijie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Gao Huibin","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao Jie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"publication","schemename":"publication"},"language":null,"country":null,"subject":[],"title":[{"value":"On the Predictability of Daily Rainfall during Rainy Season over the Huaihe River Basin","qualifier":{"classid":"main title","classname":null,"schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":null}],"relevantdate":[{"value":"2019-05-02T07:15:22Z","qualifier":{"classid":"date","classname":"date","schemeid":"dnet::date","schemename":"dnet::date"},"dataInfo":null}],"description":[{"value":null,"dataInfo":null}],"dateofacceptance":null,"publisher":{"value":"MDPI AG","dataInfo":null},"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":[],"journal":null,"originalObjIdentifier":"dli_resolver::1307198540d2264d839dfd8c9a19f4a7","dlicollectedfrom":[{"id":"dli_________::crossref","name":"Crossref","completionStatus":"complete","collectionMode":"resolved"}],"completionStatus":"complete"}
+{"collectedfrom":[{"key":"dli_________::crossref","value":"Crossref","dataInfo":null}],"dataInfo":{"invisible":false,"inferred":null,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":null},"lastupdatetimestamp":null,"id":"50|1307198540d2264d839dfd8c9a19f4a7","originalId":["1307198540d2264d839dfd8c9a19f4a7"],"pid":[{"value":"10.3390/w11050916","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":null}],"dateofcollection":"2020-08-14T14:25:55.176Z","dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"author":[{"fullname":"Cao Qing","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao Zhenchun","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Yuan Feifei","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Berndtsson Ronny","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Xu Shijie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Gao Huibin","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao Jie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"publication","schemename":"publication"},"language":null,"country":null,"subject":[],"title":[{"value":"On the Predictability of Daily Rainfall during Rainy Season over the Huaihe River Basin","qualifier":{"classid":"main title","classname":null,"schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":null}],"relevantdate":[{"value":"2019-05-02T07:15:22Z","qualifier":{"classid":"date","classname":"date","schemeid":"dnet::date","schemename":"dnet::date"},"dataInfo":null}],"description":[{"value":null,"dataInfo":null}],"dateofacceptance":null,"publisher":{"value":"MDPI AG","dataInfo":null},"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":[],"journal":null,"originalObjIdentifier":"dli_resolver::1307198540d2264d839dfd8c9a19f4a7","dlicollectedfrom":[{"id":"dli_________::crossref","name":"Crossref","completionStatus":"complete","collectionMode":"resolved"}],"completionStatus":"complete"}
+{"collectedfrom":[{"key":"dli_________::crossref","value":"Crossref","dataInfo":null}],"dataInfo":{"invisible":false,"inferred":null,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":null},"lastupdatetimestamp":null,"id":"50|1307198540d2264d839dfd8c9a19f4a7","originalId":["1307198540d2264d839dfd8c9a19f4a7"],"pid":[{"value":"10.3390/w11050916","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":null}],"dateofcollection":"2020-08-09T11:35:23.526Z","dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"author":[{"fullname":"Cao Qing","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao Zhenchun","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Yuan Feifei","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Berndtsson Ronny","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Xu Shijie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Gao Huibin","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null},{"fullname":"Hao Jie","name":null,"surname":null,"rank":null,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"publication","schemename":"publication"},"language":null,"country":null,"subject":[],"title":[{"value":"On the Predictability of Daily Rainfall during Rainy Season over the Huaihe River Basin","qualifier":{"classid":"main title","classname":null,"schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":null}],"relevantdate":[{"value":"2019-05-02T07:15:22Z","qualifier":{"classid":"date","classname":"date","schemeid":"dnet::date","schemename":"dnet::date"},"dataInfo":null}],"description":[{"value":null,"dataInfo":null}],"dateofacceptance":null,"publisher":{"value":"MDPI AG","dataInfo":null},"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":[],"journal":null,"originalObjIdentifier":"dli_resolver::1307198540d2264d839dfd8c9a19f4a7","dlicollectedfrom":[{"id":"dli_________::crossref","name":"Crossref","completionStatus":"complete","collectionMode":"resolved"}],"completionStatus":"complete"}
\ No newline at end of file
diff --git a/dhp-workflows/dhp-graph-provision-scholexplorer/pom.xml b/dhp-workflows/dhp-graph-provision-scholexplorer/pom.xml
index 05ca7d4ce..b287e9c88 100644
--- a/dhp-workflows/dhp-graph-provision-scholexplorer/pom.xml
+++ b/dhp-workflows/dhp-graph-provision-scholexplorer/pom.xml
@@ -62,6 +62,10 @@
dhp-schemas
${project.version}