[Country Propagation] changed the logic to get the collectedfrom at the result level. To fix issue when no instance is created for a result that should have the country associated. Change the code to use spark instead of hive to prepare the data needed for the propagation step. Added new tests for the intermediate steps and new verification for the propagation itself

pull/203/head
Miriam Baglioni 2 years ago
parent 3970651ee1
commit 2b643059fa

@ -22,4 +22,11 @@ public class CountrySbs implements Serializable {
public void setClassname(String classname) {
this.classname = classname;
}
public static CountrySbs newInstance(String classid, String classname) {
CountrySbs csbs = new CountrySbs();
csbs.classid = classid;
csbs.classname = classname;
return csbs;
}
}

@ -22,4 +22,11 @@ public class DatasourceCountry implements Serializable {
public void setCountry(CountrySbs country) {
this.country = country;
}
public static DatasourceCountry newInstance(String dataSourceId, CountrySbs country) {
DatasourceCountry dsc = new DatasourceCountry();
dsc.dataSourceId = dataSourceId;
dsc.country = country;
return dsc;
}
}

@ -0,0 +1,32 @@
package eu.dnetlib.dhp.countrypropagation;
import java.io.Serializable;
public class EntityEntityRel implements Serializable {
private String entity1Id;
private String entity2Id;
public static EntityEntityRel newInstance(String source, String target) {
EntityEntityRel dso = new EntityEntityRel();
dso.entity1Id = source;
dso.entity2Id = target;
return dso;
}
public String getEntity1Id() {
return entity1Id;
}
public void setEntity1Id(String entity1Id) {
this.entity1Id = entity1Id;
}
public String getEntity2Id() {
return entity2Id;
}
public void setEntity2Id(String entity2Id) {
this.entity2Id = entity2Id;
}
}

@ -2,14 +2,17 @@
package eu.dnetlib.dhp.countrypropagation;
import static eu.dnetlib.dhp.PropagationConstant.*;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.ForeachFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
@ -21,7 +24,9 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.Datasource;
import eu.dnetlib.dhp.schema.oaf.Organization;
import eu.dnetlib.dhp.schema.oaf.Qualifier;
import eu.dnetlib.dhp.schema.oaf.Relation;
import scala.Tuple2;
/**
* For the association of the country to the datasource The association is computed only for datasource of specific type
@ -54,9 +59,8 @@ public class PrepareDatasourceCountryAssociation {
log.info("outputPath {}: ", outputPath);
SparkConf conf = new SparkConf();
conf.set("hive.metastore.uris", parser.get("hive_metastore_uris"));
runWithSparkHiveSession(
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
@ -77,40 +81,46 @@ public class PrepareDatasourceCountryAssociation {
String inputPath,
String outputPath) {
final String whitelisted = whitelist
.stream()
.map(id -> " d.id = '" + id + "'")
.collect(Collectors.joining(" OR "));
final String allowed = allowedtypes
.stream()
.map(type -> " d.datasourcetype.classid = '" + type + "'")
.collect(Collectors.joining(" OR "));
Dataset<Datasource> datasource = readPath(spark, inputPath + "/datasource", Datasource.class);
Dataset<Relation> relation = readPath(spark, inputPath + "/relation", Relation.class);
Dataset<Organization> organization = readPath(spark, inputPath + "/organization", Organization.class);
datasource.createOrReplaceTempView("datasource");
relation.createOrReplaceTempView("relation");
organization.createOrReplaceTempView("organization");
String query = "SELECT source dataSourceId, " +
"named_struct('classid', country.classid, 'classname', country.classname) country " +
"FROM datasource d " +
"JOIN relation rel " +
"ON d.id = rel.source " +
"JOIN organization o " +
"ON o.id = rel.target " +
"WHERE rel.datainfo.deletedbyinference = false " +
"and lower(rel.relclass) = '" + ModelConstants.IS_PROVIDED_BY.toLowerCase() + "'" +
"and o.datainfo.deletedbyinference = false " +
"and length(o.country.classid) > 0 " +
"and (" + allowed + " or " + whitelisted + ")";
spark
.sql(query)
.as(Encoders.bean(DatasourceCountry.class))
// filtering of the datasource taking only the non deleted by inference and those with the allowed types or
// whose id is in whitelist
Dataset<Datasource> datasource = readPath(spark, inputPath + "/datasource", Datasource.class)
.filter(
(FilterFunction<Datasource>) ds -> !ds.getDataInfo().getDeletedbyinference() &&
(allowedtypes.contains(ds.getDatasourcetype().getClassid()) ||
whitelist.contains(ds.getId())));
// filtering of the relations taking the non deleted by inference and those with IsProvidedBy as relclass
Dataset<Relation> relation = readPath(spark, inputPath + "/relation", Relation.class)
.filter(
(FilterFunction<Relation>) rel -> rel.getRelClass().equalsIgnoreCase(ModelConstants.IS_PROVIDED_BY) &&
!rel.getDataInfo().getDeletedbyinference());
// filtering of the organization taking only the non deleted by inference and those with information about the
// country
Dataset<Organization> organization = readPath(spark, inputPath + "/organization", Organization.class)
.filter(
(FilterFunction<Organization>) o -> !o.getDataInfo().getDeletedbyinference() &&
o.getCountry().getClassid().length() > 0 &&
!o.getCountry().getClassid().equals(ModelConstants.UNKNOWN)) ;
// associated the datasource id with the id of the organization providing the datasource
Dataset<EntityEntityRel> dse = datasource
.joinWith(relation, datasource.col("id").equalTo(relation.col("source")))
.map(
(MapFunction<Tuple2<Datasource, Relation>, EntityEntityRel>) t2 -> EntityEntityRel
.newInstance(t2._2.getSource(), t2._2.getTarget()),
Encoders.bean(EntityEntityRel.class));
// joins with the information stored in the organization dataset to associate the country to the datasource id
dse
.joinWith(organization, dse.col("entity2Id").equalTo(organization.col("id")))
.map((MapFunction<Tuple2<EntityEntityRel, Organization>, DatasourceCountry>) t2 -> {
Qualifier country = t2._2.getCountry();
return DatasourceCountry
.newInstance(
t2._1.getEntity1Id(),
CountrySbs.newInstance(country.getClassid(), country.getClassname()));
}, Encoders.bean(DatasourceCountry.class))
.write()
.option("compression", "gzip")
.mode(SaveMode.Overwrite)

@ -3,14 +3,21 @@ package eu.dnetlib.dhp.countrypropagation;
import static eu.dnetlib.dhp.PropagationConstant.*;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.MapGroupsFunction;
import org.apache.spark.sql.*;
import org.apache.spark.sql.Dataset;
import org.slf4j.Logger;
@ -23,14 +30,6 @@ import scala.Tuple2;
public class PrepareResultCountrySet {
private static final Logger log = LoggerFactory.getLogger(PrepareResultCountrySet.class);
private static final String RESULT_COUNTRYSET_QUERY = "SELECT id resultId, collect_set(country) countrySet "
+ "FROM ( SELECT id, country "
+ "FROM datasource_country JOIN cfhb ON cf = dataSourceId "
+ "UNION ALL "
+ "SELECT id, country FROM datasource_country "
+ "JOIN cfhb ON hb = dataSourceId ) tmp "
+ "GROUP BY id";
public static void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils
.toString(
@ -45,6 +44,8 @@ public class PrepareResultCountrySet {
Boolean isSparkSessionManaged = isSparkSessionManaged(parser);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
String workingPath = parser.get("workingPath");
String inputPath = parser.get("sourcePath");
log.info("inputPath: {}", inputPath);
@ -60,9 +61,9 @@ public class PrepareResultCountrySet {
Class<? extends Result> resultClazz = (Class<? extends Result>) Class.forName(resultClassName);
SparkConf conf = new SparkConf();
conf.set("hive.metastore.uris", parser.get("hive_metastore_uris"));
//conf.set("hive.metastore.uris", parser.get("hive_metastore_uris"));
runWithSparkHiveSession(
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
@ -72,6 +73,7 @@ public class PrepareResultCountrySet {
inputPath,
outputPath,
datasourcecountrypath,
workingPath,
resultClazz);
});
}
@ -81,43 +83,63 @@ public class PrepareResultCountrySet {
String inputPath,
String outputPath,
String datasourcecountrypath,
String workingPath,
Class<R> resultClazz) {
Dataset<R> result = readPath(spark, inputPath, resultClazz);
result.createOrReplaceTempView("result");
createCfHbforResult(spark);
// selects all the results non deleted by inference and non invisible
Dataset<R> result = readPath(spark, inputPath, resultClazz)
.filter(
(FilterFunction<R>) r -> !r.getDataInfo().getDeletedbyinference() &&
!r.getDataInfo().getInvisible());
// of the results collects the distinct keys for collected from (at the level of the result) and hosted by
// and produces pairs resultId, key for each distinct key associated to the result
result.flatMap((FlatMapFunction<R, EntityEntityRel>) r -> {
Set<String> cfhb = r.getCollectedfrom().stream().map(cf -> cf.getKey()).collect(Collectors.toSet());
cfhb.addAll(r.getInstance().stream().map(i -> i.getHostedby().getKey()).collect(Collectors.toSet()));
return cfhb
.stream()
.map(value -> EntityEntityRel.newInstance(r.getId(), value))
.collect(Collectors.toList())
.iterator();
}, Encoders.bean(EntityEntityRel.class))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(workingPath + "/resultCfHb");
Dataset<DatasourceCountry> datasource_country = readPath(spark, datasourcecountrypath, DatasourceCountry.class);
datasource_country.createOrReplaceTempView("datasource_country");
spark
.sql(RESULT_COUNTRYSET_QUERY)
.as(Encoders.bean(ResultCountrySet.class))
.toJavaRDD()
.mapToPair(value -> new Tuple2<>(value.getResultId(), value))
.reduceByKey((a, b) -> {
ArrayList<CountrySbs> countryList = a.getCountrySet();
Set<String> countryCodes = countryList
.stream()
.map(CountrySbs::getClassid)
.collect(Collectors.toSet());
b
.getCountrySet()
.stream()
.forEach(c -> {
if (!countryCodes.contains(c.getClassid())) {
countryList.add(c);
countryCodes.add(c.getClassid());
}
Dataset<EntityEntityRel> cfhb = readPath(spark, workingPath + "/resultCfHb", EntityEntityRel.class);
datasource_country
.joinWith(
cfhb, cfhb
.col("entity2Id")
.equalTo(datasource_country.col("datasourceId")))
.groupByKey(
(MapFunction<Tuple2<DatasourceCountry, EntityEntityRel>, String>) t2 -> t2._2().getEntity1Id(),
Encoders.STRING())
.mapGroups(
(MapGroupsFunction<String, Tuple2<DatasourceCountry, EntityEntityRel>, ResultCountrySet>) (k, it) -> {
ResultCountrySet rcs = new ResultCountrySet();
rcs.setResultId(k);
Set<CountrySbs> set = new HashSet<>();
Set<String> countryCodes = new HashSet<>();
DatasourceCountry first = it.next()._1();
countryCodes.add(first.getCountry().getClassid());
set.add(first.getCountry());
it.forEachRemaining(t2 -> {
if (!countryCodes.contains(t2._1().getCountry().getClassid()))
set.add(t2._1().getCountry());
});
a.setCountrySet(countryList);
return a;
})
.map(couple -> OBJECT_MAPPER.writeValueAsString(couple._2()))
.saveAsTextFile(outputPath, GzipCodec.class);
rcs.setCountrySet(new ArrayList<>(set));
return rcs;
}, Encoders.bean(ResultCountrySet.class))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath);
}
}

@ -56,12 +56,6 @@ public class SparkCountryPropagationJob {
final String resultClassName = parser.get("resultTableName");
log.info("resultTableName: {}", resultClassName);
final Boolean saveGraph = Optional
.ofNullable(parser.get("saveGraph"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("saveGraph: {}", saveGraph);
Class<? extends Result> resultClazz = (Class<? extends Result>) Class.forName(resultClassName);
SparkConf conf = new SparkConf();
@ -75,8 +69,7 @@ public class SparkCountryPropagationJob {
sourcePath,
preparedInfoPath,
outputPath,
resultClazz,
saveGraph);
resultClazz);
});
}
@ -85,27 +78,26 @@ public class SparkCountryPropagationJob {
String sourcePath,
String preparedInfoPath,
String outputPath,
Class<R> resultClazz,
boolean saveGraph) {
if (saveGraph) {
log.info("Reading Graph table from: {}", sourcePath);
Dataset<R> res = readPath(spark, sourcePath, resultClazz);
log.info("Reading prepared info: {}", preparedInfoPath);
Dataset<ResultCountrySet> prepared = spark
.read()
.json(preparedInfoPath)
.as(Encoders.bean(ResultCountrySet.class));
res
.joinWith(prepared, res.col("id").equalTo(prepared.col("resultId")), "left_outer")
.map(getCountryMergeFn(), Encoders.bean(resultClazz))
.write()
.option("compression", "gzip")
.mode(SaveMode.Overwrite)
.json(outputPath);
}
Class<R> resultClazz) {
log.info("Reading Graph table from: {}", sourcePath);
Dataset<R> res = readPath(spark, sourcePath, resultClazz);
log.info("Reading prepared info: {}", preparedInfoPath);
Dataset<ResultCountrySet> prepared = spark
.read()
.json(preparedInfoPath)
.as(Encoders.bean(ResultCountrySet.class));
res
.joinWith(prepared, res.col("id").equalTo(prepared.col("resultId")), "left_outer")
.map(getCountryMergeFn(), Encoders.bean(resultClazz))
.write()
.option("compression", "gzip")
.mode(SaveMode.Overwrite)
.json(outputPath);
}
private static <R extends Result> MapFunction<Tuple2<R, ResultCountrySet>, R> getCountryMergeFn() {

@ -11,12 +11,6 @@
"paramDescription": "the hive metastore uris",
"paramRequired": false
},
{
"paramName":"sg",
"paramLongName":"saveGraph",
"paramDescription": "true if the new version of the graph must be saved",
"paramRequired": false
},
{
"paramName":"tn",
"paramLongName":"resultTableName",

@ -5,12 +5,6 @@
"paramDescription": "the path of the sequencial file to read",
"paramRequired": true
},
{
"paramName":"h",
"paramLongName":"hive_metastore_uris",
"paramDescription": "the hive metastore uris",
"paramRequired": true
},
{
"paramName": "out",
"paramLongName": "outputPath",

@ -12,9 +12,9 @@
"paramRequired": true
},
{
"paramName":"h",
"paramLongName":"hive_metastore_uris",
"paramDescription": "the hive metastore uris",
"paramName":"w",
"paramLongName":"workingPath",
"paramDescription": "the working path",
"paramRequired": true
},
{

@ -7,6 +7,7 @@ import java.nio.file.Path;
import java.util.ArrayList;
import java.util.List;
import eu.dnetlib.dhp.schema.oaf.Publication;
import org.apache.commons.io.FileUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
@ -69,17 +70,16 @@ public class CountryPropagationJobTest {
@Test
void testCountryPropagationSoftware() throws Exception {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/countrypropagation/sample/software")
.getResource("/eu/dnetlib/dhp/countrypropagation/graph/software")
.getPath();
final String preparedInfoPath = getClass()
.getResource("/eu/dnetlib/dhp/countrypropagation/preparedInfo")
.getResource("/eu/dnetlib/dhp/countrypropagation/preparedInfo/software")
.getPath();
SparkCountryPropagationJob
.main(
new String[] {
"--isSparkSessionManaged", Boolean.FALSE.toString(),
"--sourcePath", sourcePath,
"-saveGraph", "true",
"-resultTableName", Software.class.getCanonicalName(),
"-outputPath", workingDir.toString() + "/software",
"-preparedInfoPath", preparedInfoPath
@ -91,8 +91,6 @@ public class CountryPropagationJobTest {
.textFile(workingDir.toString() + "/software")
.map(item -> OBJECT_MAPPER.readValue(item, Software.class));
// tmp.map(s -> new Gson().toJson(s)).foreach(s -> System.out.println(s));
Assertions.assertEquals(10, tmp.count());
Dataset<Software> verificationDs = spark.createDataset(tmp.rdd(), Encoders.bean(Software.class));
@ -259,4 +257,69 @@ public class CountryPropagationJobTest {
7, countryExplodedWithCountryProvenance.filter("_2 = 'propagation'").count());
}
@Test
void testCountryPropagationPublication() throws Exception {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/countrypropagation/graph/publication")
.getPath();
final String preparedInfoPath = getClass()
.getResource("/eu/dnetlib/dhp/countrypropagation/preparedInfo/publication")
.getPath();
SparkCountryPropagationJob
.main(
new String[] {
"--isSparkSessionManaged", Boolean.FALSE.toString(),
"--sourcePath", sourcePath,
"-resultTableName", Publication.class.getCanonicalName(),
"-outputPath", workingDir.toString() + "/publication",
"-preparedInfoPath", preparedInfoPath
});
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<Publication> tmp = sc
.textFile(workingDir.toString() + "/publication")
.map(item -> OBJECT_MAPPER.readValue(item, Publication.class));
Assertions.assertEquals(12, tmp.count());
Assertions.assertEquals(5, tmp.filter(r -> r.getCountry().size() > 0).count());
tmp.foreach(r -> r.getCountry().stream().forEach(c -> Assertions.assertEquals("dnet:countries", c.getSchemeid())));
tmp.foreach(r -> r.getCountry().stream().forEach(c -> Assertions.assertEquals("dnet:countries", c.getSchemename())));
tmp.foreach(r -> r.getCountry().stream().forEach(c -> Assertions.assertFalse(c.getDataInfo().getDeletedbyinference())));
tmp.foreach(r -> r.getCountry().stream().forEach(c -> Assertions.assertFalse(c.getDataInfo().getInvisible())));
tmp.foreach(r -> r.getCountry().stream().forEach(c -> Assertions.assertTrue(c.getDataInfo().getInferred())));
tmp.foreach(r -> r.getCountry().stream().forEach(c -> Assertions.assertEquals("0.85", c.getDataInfo().getTrust())));
tmp.foreach(r -> r.getCountry().stream().forEach(c -> Assertions.assertEquals("propagation", c.getDataInfo().getInferenceprovenance())));
tmp.foreach(r -> r.getCountry().stream().forEach(c -> Assertions.assertEquals("country:instrepos", c.getDataInfo().getProvenanceaction().getClassid())));
tmp.foreach(r -> r.getCountry().stream().forEach(c -> Assertions.assertEquals("dnet:provenanceActions", c.getDataInfo().getProvenanceaction().getSchemeid())));
tmp.foreach(r -> r.getCountry().stream().forEach(c -> Assertions.assertEquals("dnet:provenanceActions", c.getDataInfo().getProvenanceaction().getSchemename())));
List<Country> countries = tmp.filter(r -> r.getId().equals("50|06cdd3ff4700::49ec404cee4e1452808aabeaffbd3072")).collect().get(0).getCountry();
Assertions.assertEquals(1, countries.size());
Assertions.assertEquals("NL",countries.get(0).getClassid());
Assertions.assertEquals("Netherlands",countries.get(0).getClassname());
countries = tmp.filter(r -> r.getId().equals("50|07b5c0ccd4fe::e7f5459cc97865f2af6e3da964c1250b")).collect().get(0).getCountry();
Assertions.assertEquals(1, countries.size());
Assertions.assertEquals("NL",countries.get(0).getClassid());
Assertions.assertEquals("Netherlands",countries.get(0).getClassname());
countries = tmp.filter(r -> r.getId().equals("50|355e65625b88::e7d48a470b13bda61f7ebe3513e20cb6")).collect().get(0).getCountry();
Assertions.assertEquals(2, countries.size());
Assertions.assertTrue(countries.stream().anyMatch(cs -> cs.getClassid().equals("IT") && cs.getClassname().equals("Italy")));
Assertions.assertTrue(countries.stream().anyMatch(cs -> cs.getClassid().equals("FR") && cs.getClassname().equals("France")));
countries = tmp.filter(r -> r.getId().equals("50|355e65625b88::74009c567c81b4aa55c813db658734df")).collect().get(0).getCountry();
Assertions.assertEquals(2, countries.size());
Assertions.assertTrue(countries.stream().anyMatch(cs -> cs.getClassid().equals("IT") && cs.getClassname().equals("Italy")));
Assertions.assertTrue(countries.stream().anyMatch(cs -> cs.getClassid().equals("NL") && cs.getClassname().equals("Netherlands")));
countries = tmp.filter(r -> r.getId().equals("50|355e65625b88::54a1c76f520bb2c8da27d12e42891088")).collect().get(0).getCountry();
Assertions.assertEquals(2, countries.size());
Assertions.assertTrue(countries.stream().anyMatch(cs -> cs.getClassid().equals("IT") && cs.getClassname().equals("Italy")));
Assertions.assertTrue(countries.stream().anyMatch(cs -> cs.getClassid().equals("FR") && cs.getClassname().equals("France")));
}
}

@ -0,0 +1,111 @@
package eu.dnetlib.dhp.countrypropagation;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.commons.io.FileUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
public class DatasourceCountryPreparationTest {
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static SparkSession spark;
private static Path workingDir;
@BeforeAll
public static void beforeAll() throws IOException {
workingDir = Files.createTempDirectory(DatasourceCountryPreparationTest.class.getSimpleName());
SparkConf conf = new SparkConf();
conf.setAppName(DatasourceCountryPreparationTest.class.getSimpleName());
conf.setMaster("local[*]");
conf.set("spark.driver.host", "localhost");
conf.set("hive.metastore.local", "true");
conf.set("spark.ui.enabled", "false");
conf.set("spark.sql.warehouse.dir", workingDir.toString());
conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString());
spark = SparkSession
.builder()
.appName(DatasourceCountryPreparationTest.class.getSimpleName())
.config(conf)
.getOrCreate();
}
@AfterAll
public static void afterAll() throws IOException {
FileUtils.deleteDirectory(workingDir.toFile());
spark.stop();
}
@Test
void testPrepareDatasourceCountry() throws Exception {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/countrypropagation/graph")
.getPath();
PrepareDatasourceCountryAssociation
.main(
new String[] {
"--isSparkSessionManaged", Boolean.FALSE.toString(),
"--sourcePath", sourcePath,
"--outputPath", workingDir.toString() + "/datasourceCountry",
"--allowedtypes","pubsrepository::institutional" ,
"--whitelist","10|openaire____::3795d6478e30e2c9f787d427ff160944;10|opendoar____::16e6a3326dd7d868cbc926602a61e4d0;10|eurocrisdris::fe4903425d9040f680d8610d9079ea14;10|openaire____::5b76240cc27a58c6f7ceef7d8c36660e;10|openaire____::172bbccecf8fca44ab6a6653e84cb92a;10|openaire____::149c6590f8a06b46314eed77bfca693f;10|eurocrisdris::a6026877c1a174d60f81fd71f62df1c1;10|openaire____::4692342f0992d91f9e705c26959f09e0;10|openaire____::8d529dbb05ec0284662b391789e8ae2a;10|openaire____::345c9d171ef3c5d706d08041d506428c;10|opendoar____::1c1d4df596d01da60385f0bb17a4a9e0;10|opendoar____::7a614fd06c325499f1680b9896beedeb;10|opendoar____::1ee3dfcd8a0645a25a35977997223d22;10|opendoar____::d296c101daa88a51f6ca8cfc1ac79b50;10|opendoar____::798ed7d4ee7138d49b8828958048130a;10|openaire____::c9d2209ecc4d45ba7b4ca7597acb88a2;10|eurocrisdris::c49e0fe4b9ba7b7fab717d1f0f0a674d;10|eurocrisdris::9ae43d14471c4b33661fedda6f06b539;10|eurocrisdris::432ca599953ff50cd4eeffe22faf3e48"
});
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<DatasourceCountry> tmp = sc
.textFile(workingDir.toString() + "/datasourceCountry")
.map(item -> OBJECT_MAPPER.readValue(item, DatasourceCountry.class));
Assertions.assertEquals(3, tmp.count());
Assertions.assertEquals(1, tmp.filter(dsc -> dsc.getDataSourceId()
.equals("10|eurocrisdris::fe4903425d9040f680d8610d9079ea14")).count());
Assertions.assertEquals(1, tmp.filter(dsc -> dsc.getDataSourceId()
.equals("10|opendoar____::f0dd4a99fba6075a9494772b58f95280")).count());
Assertions.assertEquals(1, tmp.filter(dsc -> dsc.getDataSourceId()
.equals("10|eurocrisdris::9ae43d14471c4b33661fedda6f06b539")).count());
Assertions.assertEquals("NL", tmp.filter(dsc -> dsc.getDataSourceId()
.equals("10|eurocrisdris::fe4903425d9040f680d8610d9079ea14")).collect()
.get(0).getCountry().getClassid());
Assertions.assertEquals("Netherlands", tmp.filter(dsc -> dsc.getDataSourceId()
.equals("10|eurocrisdris::fe4903425d9040f680d8610d9079ea14")).collect()
.get(0).getCountry().getClassname());
Assertions.assertEquals("IT", tmp.filter(dsc -> dsc.getDataSourceId()
.equals("10|opendoar____::f0dd4a99fba6075a9494772b58f95280")).collect()
.get(0).getCountry().getClassid());
Assertions.assertEquals("Italy", tmp.filter(dsc -> dsc.getDataSourceId()
.equals("10|opendoar____::f0dd4a99fba6075a9494772b58f95280")).collect()
.get(0).getCountry().getClassname());
Assertions.assertEquals("FR", tmp.filter(dsc -> dsc.getDataSourceId()
.equals("10|eurocrisdris::9ae43d14471c4b33661fedda6f06b539")).collect()
.get(0).getCountry().getClassid());
Assertions.assertEquals("France", tmp.filter(dsc -> dsc.getDataSourceId()
.equals("10|eurocrisdris::9ae43d14471c4b33661fedda6f06b539")).collect()
.get(0).getCountry().getClassname());
tmp.foreach(e -> System.out.println(OBJECT_MAPPER.writeValueAsString(e)));
}
}

@ -0,0 +1,115 @@
package eu.dnetlib.dhp.countrypropagation;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.schema.oaf.Publication;
import org.apache.commons.io.FileUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import static eu.dnetlib.dhp.PropagationConstant.isSparkSessionManaged;
public class ResultCountryPreparationTest {
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static SparkSession spark;
private static Path workingDir;
@BeforeAll
public static void beforeAll() throws IOException {
workingDir = Files.createTempDirectory(ResultCountryPreparationTest.class.getSimpleName());
SparkConf conf = new SparkConf();
conf.setAppName(ResultCountryPreparationTest.class.getSimpleName());
conf.setMaster("local[*]");
conf.set("spark.driver.host", "localhost");
conf.set("hive.metastore.local", "true");
conf.set("spark.ui.enabled", "false");
conf.set("spark.sql.warehouse.dir", workingDir.toString());
conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString());
spark = SparkSession
.builder()
.appName(ResultCountryPreparationTest.class.getSimpleName())
.config(conf)
.getOrCreate();
}
@AfterAll
public static void afterAll() throws IOException {
FileUtils.deleteDirectory(workingDir.toFile());
spark.stop();
}
@Test
void testPrepareResultCountry() throws Exception {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/countrypropagation/graph/publication")
.getPath();
final String preparedInfoPath = getClass()
.getResource("/eu/dnetlib/dhp/countrypropagation/datasourcecountry")
.getPath();
PrepareResultCountrySet
.main(
new String[] {
"--isSparkSessionManaged", Boolean.FALSE.toString(),
"--workingPath", workingDir.toString() + "/working",
"--sourcePath", sourcePath,
"--outputPath", workingDir.toString() + "/resultCountry",
"--preparedInfoPath", preparedInfoPath,
"--resultTableName", Publication.class.getCanonicalName()
});
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<ResultCountrySet> tmp = sc
.textFile(workingDir.toString() + "/resultCountry")
.map(item -> OBJECT_MAPPER.readValue(item, ResultCountrySet.class));
Assertions.assertEquals(5, tmp.count());
ResultCountrySet rc = tmp.filter(r -> r.getResultId().equals("50|06cdd3ff4700::49ec404cee4e1452808aabeaffbd3072")).collect().get(0);
Assertions.assertEquals(1, rc.getCountrySet().size());
Assertions.assertEquals("NL",rc.getCountrySet().get(0).getClassid());
Assertions.assertEquals("Netherlands",rc.getCountrySet().get(0).getClassname());
rc = tmp.filter(r -> r.getResultId().equals("50|07b5c0ccd4fe::e7f5459cc97865f2af6e3da964c1250b")).collect().get(0);
Assertions.assertEquals(1, rc.getCountrySet().size());
Assertions.assertEquals("NL",rc.getCountrySet().get(0).getClassid());
Assertions.assertEquals("Netherlands",rc.getCountrySet().get(0).getClassname());
rc = tmp.filter(r -> r.getResultId().equals("50|355e65625b88::e7d48a470b13bda61f7ebe3513e20cb6")).collect().get(0);
Assertions.assertEquals(2, rc.getCountrySet().size());
Assertions.assertTrue(rc.getCountrySet().stream().anyMatch(cs -> cs.getClassid().equals("IT") && cs.getClassname().equals("Italy")));
Assertions.assertTrue(rc.getCountrySet().stream().anyMatch(cs -> cs.getClassid().equals("FR") && cs.getClassname().equals("France")));
rc = tmp.filter(r -> r.getResultId().equals("50|355e65625b88::74009c567c81b4aa55c813db658734df")).collect().get(0);
Assertions.assertEquals(2, rc.getCountrySet().size());
Assertions.assertTrue(rc.getCountrySet().stream().anyMatch(cs -> cs.getClassid().equals("IT") && cs.getClassname().equals("Italy")));
Assertions.assertTrue(rc.getCountrySet().stream().anyMatch(cs -> cs.getClassid().equals("NL") && cs.getClassname().equals("Netherlands")));
rc = tmp.filter(r -> r.getResultId().equals("50|355e65625b88::54a1c76f520bb2c8da27d12e42891088")).collect().get(0);
Assertions.assertEquals(2, rc.getCountrySet().size());
Assertions.assertTrue(rc.getCountrySet().stream().anyMatch(cs -> cs.getClassid().equals("IT") && cs.getClassname().equals("Italy")));
Assertions.assertTrue(rc.getCountrySet().stream().anyMatch(cs -> cs.getClassid().equals("FR") && cs.getClassname().equals("France")));
}
}

@ -0,0 +1,3 @@
{"dataSourceId":"10|eurocrisdris::fe4903425d9040f680d8610d9079ea14","country":{"classid":"NL","classname":"Netherlands"}}
{"dataSourceId":"10|opendoar____::f0dd4a99fba6075a9494772b58f95280","country":{"classid":"IT","classname":"Italy"}}
{"dataSourceId":"10|eurocrisdris::9ae43d14471c4b33661fedda6f06b539","country":{"classid":"FR","classname":"France"}}

@ -0,0 +1,11 @@
{"accessinfopackage":[],"collectedfrom":[{"key":"10|openaire____::081b82f96300b6a6e3d282bad31cb6e2","value":"Crossref"}],"contentpolicies":[{"classid":"Journal articles","classname":"Journal articles","schemeid":"eosc:contentpolicies","schemename":"eosc:contentpolicies"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dataprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"datasourcetype":{"classid":"pubsrepository::journal","classname":"Journal","schemeid":"dnet:datasource_typologies","schemename":"dnet:datasource_typologies"},"datasourcetypeui":{"classid":"Journal archive","classname":"Journal archive","schemeid":"dnet:datasource_typologies_ui","schemename":"dnet:datasource_typologies_ui"},"dateofcollection":"2020-07-10","englishname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Estudos Portugueses"},"extraInfo":[],"id":"10|issn___print::a7a2010e75d849442790955162ef4e42","journal":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"issnPrinted":"2447-0546","name":"Estudos Portugueses"},"knowledgegraph":false,"lastupdatetimestamp":1646230297782,"latitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"longitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"namespaceprefix":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"jrnl24470546"},"odcontenttypes":[],"odlanguages":[],"odnumberofitems":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"officialname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Estudos Portugueses"},"openairecompatibility":{"classid":"hostedBy","classname":"collected from a compatible aggregator","schemeid":"dnet:datasourceCompatibilityLevel","schemename":"dnet:datasourceCompatibilityLevel"},"originalId":["issn___print::2447-0546"],"pid":[],"policies":[],"serviceprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"subjects":[],"thematic":false,"versioning":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false}}
{"accessinfopackage":[],"collectedfrom":[{"key":"10|openaire____::081b82f96300b6a6e3d282bad31cb6e2","value":"Crossref"}],"contentpolicies":[{"classid":"Journal articles","classname":"Journal articles","schemeid":"eosc:contentpolicies","schemename":"eosc:contentpolicies"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dataprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"datasourcetype":{"classid":"crissystem","classname":"Journal","schemeid":"dnet:datasource_typologies","schemename":"dnet:datasource_typologies"},"datasourcetypeui":{"classid":"Journal archive","classname":"Journal archive","schemeid":"dnet:datasource_typologies_ui","schemename":"dnet:datasource_typologies_ui"},"dateofcollection":"2020-07-10","englishname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"NARCIS"},"extraInfo":[],"id":"10|eurocrisdris::fe4903425d9040f680d8610d9079ea14","journal":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"issnPrinted":"2447-0546","name":"Estudos Portugueses"},"knowledgegraph":false,"lastupdatetimestamp":1646230297782,"latitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"longitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"namespaceprefix":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"jrnl24470546"},"odcontenttypes":[],"odlanguages":[],"odnumberofitems":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"officialname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Estudos Portugueses"},"openairecompatibility":{"classid":"hostedBy","classname":"collected from a compatible aggregator","schemeid":"dnet:datasourceCompatibilityLevel","schemename":"dnet:datasourceCompatibilityLevel"},"originalId":["issn___print::2447-0546"],"pid":[],"policies":[],"serviceprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"subjects":[],"thematic":false,"versioning":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false}}
{"accessinfopackage":[],"collectedfrom":[{"key":"10|openaire____::081b82f96300b6a6e3d282bad31cb6e2","value":"Crossref"}],"contentpolicies":[{"classid":"Journal articles","classname":"Journal articles","schemeid":"eosc:contentpolicies","schemename":"eosc:contentpolicies"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dataprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"datasourcetype":{"classid":"pubsrepository::institutional","classname":"Journal","schemeid":"dnet:datasource_typologies","schemename":"dnet:datasource_typologies"},"datasourcetypeui":{"classid":"Journal archive","classname":"Journal archive","schemeid":"dnet:datasource_typologies_ui","schemename":"dnet:datasource_typologies_ui"},"dateofcollection":"2020-07-10","englishname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"BELARUS"},"extraInfo":[],"id":"10|opendoar____::fd272fe04b7d4e68effd01bddcc6bb34","journal":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"issnPrinted":"2447-0546","name":"Estudos Portugueses"},"knowledgegraph":false,"lastupdatetimestamp":1646230297782,"latitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"longitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"namespaceprefix":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"jrnl24470546"},"odcontenttypes":[],"odlanguages":[],"odnumberofitems":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"officialname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Estudos Portugueses"},"openairecompatibility":{"classid":"hostedBy","classname":"collected from a compatible aggregator","schemeid":"dnet:datasourceCompatibilityLevel","schemename":"dnet:datasourceCompatibilityLevel"},"originalId":["issn___print::2447-0546"],"pid":[],"policies":[],"serviceprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"subjects":[],"thematic":false,"versioning":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false}}
{"accessinfopackage":[],"collectedfrom":[{"key":"10|openaire____::081b82f96300b6a6e3d282bad31cb6e2","value":"Crossref"}],"contentpolicies":[{"classid":"Journal articles","classname":"Journal articles","schemeid":"eosc:contentpolicies","schemename":"eosc:contentpolicies"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dataprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"datasourcetype":{"classid":"pubsrepository::institutional","classname":"Journal","schemeid":"dnet:datasource_typologies","schemename":"dnet:datasource_typologies"},"datasourcetypeui":{"classid":"Journal archive","classname":"Journal archive","schemeid":"dnet:datasource_typologies_ui","schemename":"dnet:datasource_typologies_ui"},"dateofcollection":"2020-07-10","englishname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"UNIGE"},"extraInfo":[],"id":"10|opendoar____::f0dd4a99fba6075a9494772b58f95280","journal":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"issnPrinted":"2447-0546","name":"Estudos Portugueses"},"knowledgegraph":false,"lastupdatetimestamp":1646230297782,"latitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"longitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"namespaceprefix":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"jrnl24470546"},"odcontenttypes":[],"odlanguages":[],"odnumberofitems":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"officialname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Estudos Portugueses"},"openairecompatibility":{"classid":"hostedBy","classname":"collected from a compatible aggregator","schemeid":"dnet:datasourceCompatibilityLevel","schemename":"dnet:datasourceCompatibilityLevel"},"originalId":["issn___print::2447-0546"],"pid":[],"policies":[],"serviceprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"subjects":[],"thematic":false,"versioning":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false}}
{"accessinfopackage":[],"collectedfrom":[{"key":"10|openaire____::081b82f96300b6a6e3d282bad31cb6e2","value":"Crossref"}],"contentpolicies":[{"classid":"Journal articles","classname":"Journal articles","schemeid":"eosc:contentpolicies","schemename":"eosc:contentpolicies"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dataprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"datasourcetype":{"classid":"crissystem","classname":"Journal","schemeid":"dnet:datasource_typologies","schemename":"dnet:datasource_typologies"},"datasourcetypeui":{"classid":"Journal archive","classname":"Journal archive","schemeid":"dnet:datasource_typologies_ui","schemename":"dnet:datasource_typologies_ui"},"dateofcollection":"2020-07-10","englishname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"TUHH"},"extraInfo":[],"id":"10|eurocrisdris::9ae43d14471c4b33661fedda6f06b539","journal":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"issnPrinted":"2447-0546","name":"Estudos Portugueses"},"knowledgegraph":false,"lastupdatetimestamp":1646230297782,"latitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"longitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"namespaceprefix":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"jrnl24470546"},"odcontenttypes":[],"odlanguages":[],"odnumberofitems":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"officialname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Estudos Portugueses"},"openairecompatibility":{"classid":"hostedBy","classname":"collected from a compatible aggregator","schemeid":"dnet:datasourceCompatibilityLevel","schemename":"dnet:datasourceCompatibilityLevel"},"originalId":["issn___print::2447-0546"],"pid":[],"policies":[],"serviceprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"subjects":[],"thematic":false,"versioning":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false}}
{"accessinfopackage":[],"collectedfrom":[{"key":"10|openaire____::081b82f96300b6a6e3d282bad31cb6e2","value":"Crossref"}],"contentpolicies":[{"classid":"Journal articles","classname":"Journal articles","schemeid":"eosc:contentpolicies","schemename":"eosc:contentpolicies"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dataprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"datasourcetype":{"classid":"pubsrepository::journal","classname":"Journal","schemeid":"dnet:datasource_typologies","schemename":"dnet:datasource_typologies"},"datasourcetypeui":{"classid":"Journal archive","classname":"Journal archive","schemeid":"dnet:datasource_typologies_ui","schemename":"dnet:datasource_typologies_ui"},"dateofcollection":"2020-07-10","englishname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Estudos Portugueses"},"extraInfo":[],"id":"10|issn___print::a7a2010e75d849442790955162ef4e43","journal":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"issnPrinted":"2447-0546","name":"Estudos Portugueses"},"knowledgegraph":false,"lastupdatetimestamp":1646230297782,"latitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"longitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"namespaceprefix":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"jrnl24470546"},"odcontenttypes":[],"odlanguages":[],"odnumberofitems":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"officialname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Estudos Portugueses"},"openairecompatibility":{"classid":"hostedBy","classname":"collected from a compatible aggregator","schemeid":"dnet:datasourceCompatibilityLevel","schemename":"dnet:datasourceCompatibilityLevel"},"originalId":["issn___print::2447-0546"],"pid":[],"policies":[],"serviceprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"subjects":[],"thematic":false,"versioning":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false}}
{"accessinfopackage":[],"collectedfrom":[{"key":"10|openaire____::081b82f96300b6a6e3d282bad31cb6e2","value":"Crossref"}],"contentpolicies":[{"classid":"Journal articles","classname":"Journal articles","schemeid":"eosc:contentpolicies","schemename":"eosc:contentpolicies"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dataprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"datasourcetype":{"classid":"pubsrepository::journal","classname":"Journal","schemeid":"dnet:datasource_typologies","schemename":"dnet:datasource_typologies"},"datasourcetypeui":{"classid":"Journal archive","classname":"Journal archive","schemeid":"dnet:datasource_typologies_ui","schemename":"dnet:datasource_typologies_ui"},"dateofcollection":"2020-07-10","englishname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Estudos Portugueses"},"extraInfo":[],"id":"10|issn___print::a7a2010e75d849442790955162ef4e44","journal":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"issnPrinted":"2447-0546","name":"Estudos Portugueses"},"knowledgegraph":false,"lastupdatetimestamp":1646230297782,"latitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"longitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"namespaceprefix":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"jrnl24470546"},"odcontenttypes":[],"odlanguages":[],"odnumberofitems":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"officialname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Estudos Portugueses"},"openairecompatibility":{"classid":"hostedBy","classname":"collected from a compatible aggregator","schemeid":"dnet:datasourceCompatibilityLevel","schemename":"dnet:datasourceCompatibilityLevel"},"originalId":["issn___print::2447-0546"],"pid":[],"policies":[],"serviceprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"subjects":[],"thematic":false,"versioning":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false}}
{"accessinfopackage":[],"collectedfrom":[{"key":"10|openaire____::081b82f96300b6a6e3d282bad31cb6e2","value":"Crossref"}],"contentpolicies":[{"classid":"Journal articles","classname":"Journal articles","schemeid":"eosc:contentpolicies","schemename":"eosc:contentpolicies"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dataprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"datasourcetype":{"classid":"pubsrepository::journal","classname":"Journal","schemeid":"dnet:datasource_typologies","schemename":"dnet:datasource_typologies"},"datasourcetypeui":{"classid":"Journal archive","classname":"Journal archive","schemeid":"dnet:datasource_typologies_ui","schemename":"dnet:datasource_typologies_ui"},"dateofcollection":"2020-07-10","englishname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Estudos Portugueses"},"extraInfo":[],"id":"10|issn___print::a7a2010e75d849442790955162ef4e45","journal":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"issnPrinted":"2447-0546","name":"Estudos Portugueses"},"knowledgegraph":false,"lastupdatetimestamp":1646230297782,"latitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"longitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"namespaceprefix":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"jrnl24470546"},"odcontenttypes":[],"odlanguages":[],"odnumberofitems":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"officialname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Estudos Portugueses"},"openairecompatibility":{"classid":"hostedBy","classname":"collected from a compatible aggregator","schemeid":"dnet:datasourceCompatibilityLevel","schemename":"dnet:datasourceCompatibilityLevel"},"originalId":["issn___print::2447-0546"],"pid":[],"policies":[],"serviceprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"subjects":[],"thematic":false,"versioning":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false}}
{"accessinfopackage":[],"collectedfrom":[{"key":"10|openaire____::081b82f96300b6a6e3d282bad31cb6e2","value":"Crossref"}],"contentpolicies":[{"classid":"Journal articles","classname":"Journal articles","schemeid":"eosc:contentpolicies","schemename":"eosc:contentpolicies"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dataprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"datasourcetype":{"classid":"pubsrepository::journal","classname":"Journal","schemeid":"dnet:datasource_typologies","schemename":"dnet:datasource_typologies"},"datasourcetypeui":{"classid":"Journal archive","classname":"Journal archive","schemeid":"dnet:datasource_typologies_ui","schemename":"dnet:datasource_typologies_ui"},"dateofcollection":"2020-07-10","englishname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Estudos Portugueses"},"extraInfo":[],"id":"10|issn___print::a7a2010e75d849442790955162ef4e46","journal":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"issnPrinted":"2447-0546","name":"Estudos Portugueses"},"knowledgegraph":false,"lastupdatetimestamp":1646230297782,"latitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"longitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"namespaceprefix":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"jrnl24470546"},"odcontenttypes":[],"odlanguages":[],"odnumberofitems":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"officialname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Estudos Portugueses"},"openairecompatibility":{"classid":"hostedBy","classname":"collected from a compatible aggregator","schemeid":"dnet:datasourceCompatibilityLevel","schemename":"dnet:datasourceCompatibilityLevel"},"originalId":["issn___print::2447-0546"],"pid":[],"policies":[],"serviceprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"subjects":[],"thematic":false,"versioning":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false}}
{"accessinfopackage":[],"collectedfrom":[{"key":"10|openaire____::081b82f96300b6a6e3d282bad31cb6e2","value":"Crossref"}],"contentpolicies":[{"classid":"Journal articles","classname":"Journal articles","schemeid":"eosc:contentpolicies","schemename":"eosc:contentpolicies"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dataprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"datasourcetype":{"classid":"pubsrepository::journal","classname":"Journal","schemeid":"dnet:datasource_typologies","schemename":"dnet:datasource_typologies"},"datasourcetypeui":{"classid":"Journal archive","classname":"Journal archive","schemeid":"dnet:datasource_typologies_ui","schemename":"dnet:datasource_typologies_ui"},"dateofcollection":"2020-07-10","englishname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Estudos Portugueses"},"extraInfo":[],"id":"10|issn___print::a7a2010e75d849442790955162ef4e47","journal":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"issnPrinted":"2447-0546","name":"Estudos Portugueses"},"knowledgegraph":false,"lastupdatetimestamp":1646230297782,"latitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"longitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"namespaceprefix":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"jrnl24470546"},"odcontenttypes":[],"odlanguages":[],"odnumberofitems":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"officialname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Estudos Portugueses"},"openairecompatibility":{"classid":"hostedBy","classname":"collected from a compatible aggregator","schemeid":"dnet:datasourceCompatibilityLevel","schemename":"dnet:datasourceCompatibilityLevel"},"originalId":["issn___print::2447-0546"],"pid":[],"policies":[],"serviceprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"subjects":[],"thematic":false,"versioning":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false}}
{"accessinfopackage":[],"collectedfrom":[{"key":"10|openaire____::081b82f96300b6a6e3d282bad31cb6e2","value":"Crossref"}],"contentpolicies":[{"classid":"Journal articles","classname":"Journal articles","schemeid":"eosc:contentpolicies","schemename":"eosc:contentpolicies"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dataprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"datasourcetype":{"classid":"pubsrepository::journal","classname":"Journal","schemeid":"dnet:datasource_typologies","schemename":"dnet:datasource_typologies"},"datasourcetypeui":{"classid":"Journal archive","classname":"Journal archive","schemeid":"dnet:datasource_typologies_ui","schemename":"dnet:datasource_typologies_ui"},"dateofcollection":"2020-07-10","englishname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Estudos Portugueses"},"extraInfo":[],"id":"10|issn___print::a7a2010e75d849442790955162ef4e48","journal":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"issnPrinted":"2447-0546","name":"Estudos Portugueses"},"knowledgegraph":false,"lastupdatetimestamp":1646230297782,"latitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"longitude":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"namespaceprefix":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"jrnl24470546"},"odcontenttypes":[],"odlanguages":[],"odnumberofitems":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"0.0"},"officialname":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Estudos Portugueses"},"openairecompatibility":{"classid":"hostedBy","classname":"collected from a compatible aggregator","schemeid":"dnet:datasourceCompatibilityLevel","schemename":"dnet:datasourceCompatibilityLevel"},"originalId":["issn___print::2447-0546"],"pid":[],"policies":[],"serviceprovider":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false},"subjects":[],"thematic":false,"versioning":{"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":false}}

@ -0,0 +1,8 @@
{"alternativeNames":[],"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"country":{"classid":"FI","classname":"Finland","schemeid":"dnet:countries","schemename":"dnet:countries"},"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dateofcollection":"2019-01-25","dateoftransformation":"2022-02-08","ecenterprise":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"echighereducation":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"ecinternationalorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"ecinternationalorganizationeurinterests":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"eclegalbody":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"eclegalperson":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"ecnonprofit":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"ecnutscode":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"ecresearchorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"ecsmevalidated":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"extraInfo":[],"id":"20|aka_________::cffd8c5427c035e5d4bddc5647942ba8","lastupdatetimestamp":1646230297782,"legalname":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Graduate Institute of International Studies"},"originalId":["aka_________::4a12fb514672d706d7e9d4605ad45d78"],"pid":[]}
{"alternativeNames":[],"collectedfrom":[{"key":"10|openaire____::457528c43fabd74e212db2ed61101075","value":"Agence Nationale de la Recherche"}],"country":{"classid":"UNKNOWN","classname":"Unknown","schemeid":"dnet:countries","schemename":"dnet:countries"},"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"dateofcollection":"2021-02-15","dateoftransformation":"2021-05-19","ecenterprise":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"echighereducation":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecinternationalorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecinternationalorganizationeurinterests":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"eclegalbody":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"eclegalperson":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecnonprofit":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecnutscode":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecresearchorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecsmevalidated":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"extraInfo":[],"id":"20|anr_________::357ee61b6fe46c7c07210a1cd9acf6ed","lastupdatetimestamp":1646230297782,"legalname":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"INSTITUT NATIONAL DE LA SANTE ET DE LA RECHERCHE MEDICALE - DELEGATION PARIS XI"},"originalId":["anr_________::145402d7c38cf25af807084e757e1161"],"pid":[]}
{"alternativeNames":[],"collectedfrom":[{"key":"10|openaire____::457528c43fabd74e212db2ed61101075","value":"Agence Nationale de la Recherche"}],"country":{"classid":"UNKNOWN","classname":"Unknown","schemeid":"dnet:countries","schemename":"dnet:countries"},"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"dateofcollection":"2021-02-15","dateoftransformation":"2021-05-19","ecenterprise":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"echighereducation":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecinternationalorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecinternationalorganizationeurinterests":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"eclegalbody":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"eclegalperson":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecnonprofit":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecnutscode":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecresearchorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecsmevalidated":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"extraInfo":[],"id":"20|anr_________::43084487236103c68872cf929c57eaff","lastupdatetimestamp":1646230297782,"legalname":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"Laboratoire dInformatique Paris Descartes"},"originalId":["anr_________::a8cb45ed89911406d924a2c5831c1c5b"],"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"qualifier":{"classid":"RNSR","classname":"RNSR","schemeid":"","schemename":""},"value":"RNSR:200014469G"}]}
{"alternativeNames":[],"collectedfrom":[{"key":"10|openaire____::457528c43fabd74e212db2ed61101075","value":"Agence Nationale de la Recherche"}],"country":{"classid":"UNKNOWN","classname":"Unknown","schemeid":"dnet:countries","schemename":"dnet:countries"},"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"dateofcollection":"2021-05-20","dateoftransformation":"2021-05-19","ecenterprise":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"echighereducation":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecinternationalorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecinternationalorganizationeurinterests":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"eclegalbody":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"eclegalperson":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecnonprofit":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecnutscode":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecresearchorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecsmevalidated":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"extraInfo":[],"id":"20|anr_________::b22add4abf57294cb68882dab6062788","lastupdatetimestamp":1646230297782,"legalname":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"Association de gestion de l'Ecole Centrale Electronique"},"originalId":["anr_________::50ced32bb0d6464fb3c0e3bff9347484"],"pid":[]}
{"alternativeNames":[],"collectedfrom":[{"key":"10|openaire____::6ac933301a3933c8a22ceebea7000326","value":"Academy of Finland"}],"country":{"classid":"NL","classname":"Netherlands","schemeid":"dnet:countries","schemename":"dnet:countries"},"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"dateofcollection":"2019-01-25","dateoftransformation":"2022-02-08","ecenterprise":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"echighereducation":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"ecinternationalorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"ecinternationalorganizationeurinterests":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"eclegalbody":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"eclegalperson":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"ecnonprofit":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"ecnutscode":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"ecresearchorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"ecsmevalidated":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"false"},"extraInfo":[],"id":"20|pending_org_::82f63b2d21ae88596b9d8991780e9888","lastupdatetimestamp":1646230297782,"legalname":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"value":"Graduate Institute of International Studies"},"originalId":["aka_________::4a12fb514672d706d7e9d4605ad45d78"],"pid":[]}
{"alternativeNames":[],"collectedfrom":[{"key":"10|openaire____::457528c43fabd74e212db2ed61101075","value":"Agence Nationale de la Recherche"}],"country":{"classid":"UNKNOWN","classname":"Unknown","schemeid":"dnet:countries","schemename":"dnet:countries"},"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"dateofcollection":"2021-02-15","dateoftransformation":"2021-05-19","ecenterprise":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"echighereducation":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecinternationalorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecinternationalorganizationeurinterests":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"eclegalbody":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"eclegalperson":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecnonprofit":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecnutscode":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecresearchorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecsmevalidated":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"extraInfo":[],"id":"20|pending_org_::5b73b8b2d0df764e13a62291dfedf8f6","lastupdatetimestamp":1646230297782,"legalname":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"INSTITUT NATIONAL DE LA SANTE ET DE LA RECHERCHE MEDICALE - DELEGATION PARIS XI"},"originalId":["anr_________::145402d7c38cf25af807084e757e1161"],"pid":[]}
{"alternativeNames":[],"collectedfrom":[{"key":"10|openaire____::457528c43fabd74e212db2ed61101075","value":"Agence Nationale de la Recherche"}],"country":{"classid":"IT","classname":"Italy","schemeid":"dnet:countries","schemename":"dnet:countries"},"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"dateofcollection":"2021-02-15","dateoftransformation":"2021-05-19","ecenterprise":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"echighereducation":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecinternationalorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecinternationalorganizationeurinterests":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"eclegalbody":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"eclegalperson":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecnonprofit":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecnutscode":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecresearchorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecsmevalidated":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"extraInfo":[],"id":"20|openorgs____::322ff2a6524820640bc5d1311871585e","lastupdatetimestamp":1646230297782,"legalname":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"Laboratoire dInformatique Paris Descartes"},"originalId":["anr_________::a8cb45ed89911406d924a2c5831c1c5b"],"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"qualifier":{"classid":"RNSR","classname":"RNSR","schemeid":"","schemename":""},"value":"RNSR:200014469G"}]}
{"alternativeNames":[],"collectedfrom":[{"key":"10|openaire____::457528c43fabd74e212db2ed61101075","value":"Agence Nationale de la Recherche"}],"country":{"classid":"FR","classname":"France","schemeid":"dnet:countries","schemename":"dnet:countries"},"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"dateofcollection":"2021-02-15","dateoftransformation":"2021-05-19","ecenterprise":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"echighereducation":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecinternationalorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecinternationalorganizationeurinterests":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"eclegalbody":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"eclegalperson":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecnonprofit":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecnutscode":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecresearchorganization":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"ecsmevalidated":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"false"},"extraInfo":[],"id":"20|openorgs____::58e60f1715d219aa6757ba0b0f2ccbce","lastupdatetimestamp":1646230297782,"legalname":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"value":"Laboratoire dInformatique Paris Descartes"},"originalId":["anr_________::a8cb45ed89911406d924a2c5831c1c5b"],"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.810"},"qualifier":{"classid":"RNSR","classname":"RNSR","schemeid":"","schemename":""},"value":"RNSR:200014469G"}]}

@ -0,0 +1,24 @@
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"IsProvidedBy","relType":"resultOrganization","source":"10|eurocrisdris::fe4903425d9040f680d8610d9079ea14","subRelType":"affiliation","target":"20|pending_org_::82f63b2d21ae88596b9d8991780e9888","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"IsProvidedBy","relType":"resultOrganization","source":"10|opendoar____::fd272fe04b7d4e68effd01bddcc6bb34","subRelType":"affiliation","target":"20|pending_org_::5b73b8b2d0df764e13a62291dfedf8f6","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"IsProvidedBy","relType":"resultOrganization","source":"10|opendoar____::f0dd4a99fba6075a9494772b58f95280","subRelType":"affiliation","target":"20|openorgs____::322ff2a6524820640bc5d1311871585e","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"IsProvidedBy","relType":"resultOrganization","source":"10|eurocrisdris::9ae43d14471c4b33661fedda6f06b539","subRelType":"affiliation","target":"20|openorgs____::58e60f1715d219aa6757ba0b0f2ccbce","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"IsProvidedBy","relType":"resultOrganization","target":"20|openorgs____::64badd35233ba2cd4946368ef2f4cf57","subRelType":"affiliation","source":"10|issn___print::a7a2010e75d849442790955162ef4e42","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"IsProvidedBy","relType":"resultOrganization","source":"10|issn___print::a7a2010e75d849442790955162ef4e43","subRelType":"affiliation","target":"20|openorgs____::64badd35233ba2cd4946368ef2f4cf57","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"IsProvidedBy","relType":"resultOrganization","source":"10|issn___print::a7a2010e75d849442790955162ef4e44","subRelType":"affiliation","target":"20|openorgs____::548cbb0c5a93722f3a9aa62aa17a1ba1","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"IsProvidedBy","relType":"resultOrganization","source":"10|issn___print::a7a2010e75d849442790955162ef4e45","subRelType":"affiliation","target":"20|pending_org_::c522a7c935f9fd9578122e60eeec282c","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"isAuthorInstitutionOf","relType":"resultOrganization","source":"20|openorgs____::64badd35233ba2cd4946368ef2f4cf57","subRelType":"affiliation","target":"50|dedup_wf_001::06e51d2bf295531b2d2e7a1b55500783","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"hasAuthorInstitution","relType":"resultOrganization","source":"50|dedup_wf_001::06e51d2bf295531b2d2e7a1b55500783","subRelType":"affiliation","target":"20|openorgs____::64badd35233ba2cd4946368ef2f4cf57","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"isAuthorInstitutionOf","relType":"resultOrganization","source":"20|openorgs____::91a81877815afb4ebf25c1a3f3b03c5d","subRelType":"affiliation","target":"50|dedup_wf_001::08d6f2001319c86d0e69b0f83ad75df2","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"hasAuthorInstitution","relType":"resultOrganization","source":"50|dedup_wf_001::08d6f2001319c86d0e69b0f83ad75df2","subRelType":"affiliation","target":"20|openorgs____::91a81877815afb4ebf25c1a3f3b03c5d","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"isAuthorInstitutionOf","relType":"resultOrganization","source":"20|openorgs____::548cbb0c5a93722f3a9aa62aa17a1ba1","subRelType":"affiliation","target":"50|dedup_wf_001::0a1cdf269375d32ce341fdeb0e92dfa8","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"hasAuthorInstitution","relType":"resultOrganization","source":"50|dedup_wf_001::0a1cdf269375d32ce341fdeb0e92dfa8","subRelType":"affiliation","target":"20|openorgs____::548cbb0c5a93722f3a9aa62aa17a1ba1","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"isAuthorInstitutionOf","relType":"resultOrganization","source":"20|pending_org_::a50fdd7f7e77b74ea2b16823151c391a","subRelType":"affiliation","target":"50|dedup_wf_001::0ab92bed024ee6883c7a1244722e5eec","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"hasAuthorInstitution","relType":"resultOrganization","source":"50|dedup_wf_001::0ab92bed024ee6883c7a1244722e5eec","subRelType":"affiliation","target":"20|pending_org_::a50fdd7f7e77b74ea2b16823151c391a","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"isAuthorInstitutionOf","relType":"resultOrganization","source":"20|openorgs____::64badd35233ba2cd4946368ef2f4cf57","subRelType":"affiliation","target":"50|dedup_wf_001::0ca26c736ad4d15b3d5ee90a4d7853e1","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"hasAuthorInstitution","relType":"resultOrganization","source":"50|dedup_wf_001::0ca26c736ad4d15b3d5ee90a4d7853e1","subRelType":"affiliation","target":"20|openorgs____::64badd35233ba2cd4946368ef2f4cf57","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"isAuthorInstitutionOf","relType":"resultOrganization","source":"20|pending_org_::a50fdd7f7e77b74ea2b16823151c391a","subRelType":"affiliation","target":"50|dedup_wf_001::0ef8dfab3927cb4d69df0d3113f05a42","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"hasAuthorInstitution","relType":"resultOrganization","source":"50|dedup_wf_001::0ef8dfab3927cb4d69df0d3113f05a42","subRelType":"affiliation","target":"20|pending_org_::a50fdd7f7e77b74ea2b16823151c391a","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"isAuthorInstitutionOf","relType":"resultOrganization","source":"20|openorgs____::548cbb0c5a93722f3a9aa62aa17a1ba1","subRelType":"affiliation","target":"50|dedup_wf_001::0f488ad00253126c14a21abe6b2d406c","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"hasAuthorInstitution","relType":"resultOrganization","source":"50|dedup_wf_001::0f488ad00253126c14a21abe6b2d406c","subRelType":"affiliation","target":"20|openorgs____::548cbb0c5a93722f3a9aa62aa17a1ba1","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"isAuthorInstitutionOf","relType":"resultOrganization","source":"20|pending_org_::c522a7c935f9fd9578122e60eeec282c","subRelType":"affiliation","target":"50|dedup_wf_001::12206bf78aabd7d52132477182d19147","validated":false}
{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"result:organization:instrepo","classname":"Propagation of affiliation to result collected from datasources of type institutional repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"},"properties":[],"relClass":"hasAuthorInstitution","relType":"resultOrganization","source":"50|dedup_wf_001::12206bf78aabd7d52132477182d19147","subRelType":"affiliation","target":"20|pending_org_::c522a7c935f9fd9578122e60eeec282c","validated":false}

@ -0,0 +1,5 @@
{"resultId":"50|07b5c0ccd4fe::e7f5459cc97865f2af6e3da964c1250b","countrySet":[{"classid":"NL","classname":"Netherlands"}]}
{"resultId":"50|355e65625b88::54a1c76f520bb2c8da27d12e42891088","countrySet":[{"classid":"IT","classname":"Italy"},{"classid":"FR","classname":"France"}]}
{"resultId":"50|06cdd3ff4700::49ec404cee4e1452808aabeaffbd3072","countrySet":[{"classid":"NL","classname":"Netherlands"}]}
{"resultId":"50|355e65625b88::74009c567c81b4aa55c813db658734df","countrySet":[{"classid":"NL","classname":"Netherlands"},{"classid":"IT","classname":"Italy"}]}
{"resultId":"50|355e65625b88::e7d48a470b13bda61f7ebe3513e20cb6","countrySet":[{"classid":"IT","classname":"Italy"},{"classid":"FR","classname":"France"}]}

@ -8,7 +8,6 @@ import java.io.IOException;
import java.io.StringReader;
import java.util.List;
import eu.dnetlib.dhp.schema.oaf.Datasource;
import org.apache.commons.io.IOUtils;
import org.dom4j.Document;
import org.dom4j.DocumentException;
@ -24,6 +23,7 @@ import eu.dnetlib.dhp.oa.provision.model.RelatedEntity;
import eu.dnetlib.dhp.oa.provision.model.RelatedEntityWrapper;
import eu.dnetlib.dhp.oa.provision.utils.ContextMapper;
import eu.dnetlib.dhp.oa.provision.utils.XmlRecordFactory;
import eu.dnetlib.dhp.schema.oaf.Datasource;
import eu.dnetlib.dhp.schema.oaf.Project;
import eu.dnetlib.dhp.schema.oaf.Publication;
import eu.dnetlib.dhp.schema.oaf.Relation;
@ -146,10 +146,10 @@ public class XmlRecordFactoryTest {
final ContextMapper contextMapper = new ContextMapper();
final XmlRecordFactory xmlRecordFactory = new XmlRecordFactory(contextMapper, false,
XmlConverterJob.schemaLocation);
XmlConverterJob.schemaLocation);
final Datasource d = OBJECT_MAPPER
.readValue(IOUtils.toString(getClass().getResourceAsStream("datasource.json")), Datasource.class);
.readValue(IOUtils.toString(getClass().getResourceAsStream("datasource.json")), Datasource.class);
final String xml = xmlRecordFactory.build(new JoinedEntity<>(d));

Loading…
Cancel
Save