From 3d6be209899806d9c319eb034f26b1248e671091 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 9 Oct 2023 14:26:33 +0200 Subject: [PATCH 01/60] changes to use the API instead of the IS the get the information for the communities to be used during bulktagging and context propagation --- .../bipfinder/SparkAtomicActionScoreJob.java | 2 +- .../dhp/oa/dedup/SparkPropagateRelation.java | 72 +++---- .../eu/dnetlib/dhp/api/QueryCommunityAPI.java | 78 +++++++ .../main/java/eu/dnetlib/dhp/api/Utils.java | 136 +++++++++++++ .../api/model/CommunityContentprovider.java | 43 ++++ .../model/CommunityEntityMap.java} | 6 +- .../dnetlib/dhp/api/model/CommunityModel.java | 108 ++++++++++ .../dhp/api/model/CommunitySummary.java | 16 ++ .../dnetlib/dhp/api/model/ContentModel.java | 50 +++++ .../dnetlib/dhp/api/model/DatasourceList.java | 12 ++ .../dhp/api/model/OrganizationList.java | 15 ++ .../dnetlib/dhp/api/model/ProjectModel.java | 23 +++ .../dnetlib/dhp/bulktag/SparkBulkTagJob.java | 4 +- .../dhp/bulktag/community/Community.java | 10 +- .../community/CommunityConfiguration.java | 8 +- .../CommunityConfigurationFactory.java | 13 +- .../dhp/bulktag/community/Constraint.java | 20 +- .../community/SelectionConstraints.java | 3 +- .../PrepareResultCommunitySet.java | 17 +- .../oozie_app/workflow.xml | 8 +- .../dnetlib/dhp/bulktag/BulkTagJobTest.java | 38 ++++ .../CommunityConfigurationFactoryTest.java | 2 +- .../dhp/bulktag/QueryCommunityAPITest.java | 91 +++++++++ .../group/GroupEntitiesSparkJobTest.java | 190 +++++++++--------- 24 files changed, 787 insertions(+), 178 deletions(-) create mode 100644 dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java create mode 100644 dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java create mode 100644 dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityContentprovider.java rename dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/{resulttocommunityfromorganization/OrganizationMap.java => api/model/CommunityEntityMap.java} (59%) create mode 100644 dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityModel.java create mode 100644 dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunitySummary.java create mode 100644 dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/ContentModel.java create mode 100644 dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/DatasourceList.java create mode 100644 dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/OrganizationList.java create mode 100644 dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/ProjectModel.java create mode 100644 dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/QueryCommunityAPITest.java diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipfinder/SparkAtomicActionScoreJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipfinder/SparkAtomicActionScoreJob.java index 040c89782..8adc88920 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipfinder/SparkAtomicActionScoreJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipfinder/SparkAtomicActionScoreJob.java @@ -95,7 +95,7 @@ public class SparkAtomicActionScoreJob implements Serializable { return projectScores.map((MapFunction) bipProjectScores -> { Project project = new Project(); - project.setId(bipProjectScores.getProjectId()); + // project.setId(bipProjectScores.getProjectId()); project.setMeasures(bipProjectScores.toMeasures()); return project; }, Encoders.bean(Project.class)) diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPropagateRelation.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPropagateRelation.java index 739295c91..cb1c70059 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPropagateRelation.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPropagateRelation.java @@ -67,60 +67,60 @@ public class SparkPropagateRelation extends AbstractSparkAction { log.info("graphOutputPath: '{}'", graphOutputPath); Dataset mergeRels = spark - .read() - .load(DedupUtility.createMergeRelPath(workingPath, "*", "*")) - .as(REL_BEAN_ENC); + .read() + .load(DedupUtility.createMergeRelPath(workingPath, "*", "*")) + .as(REL_BEAN_ENC); // Dataset idsToMerge = mergeRels - .where(col("relClass").equalTo(ModelConstants.MERGES)) - .select(col("source").as("dedupID"), col("target").as("mergedObjectID")) - .distinct(); + .where(col("relClass").equalTo(ModelConstants.MERGES)) + .select(col("source").as("dedupID"), col("target").as("mergedObjectID")) + .distinct(); Dataset allRels = spark - .read() - .schema(REL_BEAN_ENC.schema()) - .json(graphBasePath + "/relation"); + .read() + .schema(REL_BEAN_ENC.schema()) + .json(graphBasePath + "/relation"); Dataset dedupedRels = allRels - .joinWith(idsToMerge, allRels.col("source").equalTo(idsToMerge.col("mergedObjectID")), "left_outer") - .joinWith(idsToMerge, col("_1.target").equalTo(idsToMerge.col("mergedObjectID")), "left_outer") - .select("_1._1", "_1._2.dedupID", "_2.dedupID") - .as(Encoders.tuple(REL_BEAN_ENC, Encoders.STRING(), Encoders.STRING())) - .map((MapFunction, Relation>) t -> { - Relation rel = t._1(); - String newSource = t._2(); - String newTarget = t._3(); + .joinWith(idsToMerge, allRels.col("source").equalTo(idsToMerge.col("mergedObjectID")), "left_outer") + .joinWith(idsToMerge, col("_1.target").equalTo(idsToMerge.col("mergedObjectID")), "left_outer") + .select("_1._1", "_1._2.dedupID", "_2.dedupID") + .as(Encoders.tuple(REL_BEAN_ENC, Encoders.STRING(), Encoders.STRING())) + .map((MapFunction, Relation>) t -> { + Relation rel = t._1(); + String newSource = t._2(); + String newTarget = t._3(); - if (rel.getDataInfo() == null) { - rel.setDataInfo(new DataInfo()); - } + if (rel.getDataInfo() == null) { + rel.setDataInfo(new DataInfo()); + } - if (newSource != null || newTarget != null) { - rel.getDataInfo().setDeletedbyinference(false); + if (newSource != null || newTarget != null) { + rel.getDataInfo().setDeletedbyinference(false); - if (newSource != null) - rel.setSource(newSource); + if (newSource != null) + rel.setSource(newSource); - if (newTarget != null) - rel.setTarget(newTarget); - } + if (newTarget != null) + rel.setTarget(newTarget); + } - return rel; - }, REL_BEAN_ENC); + return rel; + }, REL_BEAN_ENC); // ids of records that are both not deletedbyinference and not invisible Dataset ids = validIds(spark, graphBasePath); // filter relations that point to valid records, can force them to be visible Dataset cleanedRels = dedupedRels - .join(ids, col("source").equalTo(ids.col("id")), "leftsemi") - .join(ids, col("target").equalTo(ids.col("id")), "leftsemi") - .as(REL_BEAN_ENC) - .map((MapFunction) r -> { - r.getDataInfo().setInvisible(false); - return r; - }, REL_KRYO_ENC); + .join(ids, col("source").equalTo(ids.col("id")), "leftsemi") + .join(ids, col("target").equalTo(ids.col("id")), "leftsemi") + .as(REL_BEAN_ENC) + .map((MapFunction) r -> { + r.getDataInfo().setInvisible(false); + return r; + }, REL_KRYO_ENC); Dataset distinctRels = cleanedRels .groupByKey( diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java new file mode 100644 index 000000000..635ee2027 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java @@ -0,0 +1,78 @@ + +package eu.dnetlib.dhp.api; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.net.HttpURLConnection; +import java.net.MalformedURLException; +import java.net.URL; + +import org.apache.http.HttpHeaders; +import org.jetbrains.annotations.NotNull; + +import com.google.gson.Gson; + +/** + * @author miriam.baglioni + * @Date 06/10/23 + */ +public class QueryCommunityAPI { + private static final String baseUrl = "https://services.openaire.eu/openaire/"; + + private static String get(String geturl) throws IOException{ + URL url = new URL(geturl); + HttpURLConnection conn = (HttpURLConnection) url.openConnection(); + conn.setDoOutput(true); + conn.setRequestMethod("GET"); + + int responseCode = conn.getResponseCode(); + String body = getBody(conn); + conn.disconnect(); + if (responseCode != HttpURLConnection.HTTP_OK) + throw new IOException("Unexpected code " + responseCode + body); + + return body; + } + + public static String communities() throws IOException { + return get(baseUrl + "community/communities"); + } + + public static String community(String id) throws IOException { + return get(baseUrl + "community/" + id); + } + + public static String communityDatasource(String id)throws IOException{ + return get(baseUrl + "community/" + id + "/contentproviders"); + + } + + public static String communityPropagationOrganization(String id) throws IOException { + return get(baseUrl + "community/" + id + "/propagationOrganizations"); + } + + public static String communityProjects(String id, String page, String size) throws IOException{ + return get(baseUrl + "community/" + id +"/projects/" + page + "/" + size); + } + + @NotNull + private static String getBody(HttpURLConnection conn) throws IOException { + String body = "{}"; + try (BufferedReader br = new BufferedReader( + new InputStreamReader(conn.getInputStream(), "utf-8"))) { + StringBuilder response = new StringBuilder(); + String responseLine = null; + while ((responseLine = br.readLine()) != null) { + response.append(responseLine.trim()); + } + + body = response.toString(); + + } + return body; + } + + +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java new file mode 100644 index 000000000..5b3004a5d --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java @@ -0,0 +1,136 @@ +package eu.dnetlib.dhp.api; + +import com.amazonaws.util.StringUtils; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Maps; +import eu.dnetlib.dhp.api.model.*; +import eu.dnetlib.dhp.bulktag.community.Community; +import eu.dnetlib.dhp.bulktag.community.CommunityConfiguration; +import eu.dnetlib.dhp.bulktag.community.Provider; +import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; +import eu.dnetlib.dhp.bulktag.criteria.VerbResolverFactory; + +import javax.management.Query; +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * @author miriam.baglioni + * @Date 09/10/23 + */ +public class Utils implements Serializable { + private static final ObjectMapper MAPPER = new ObjectMapper(); + private static final VerbResolver resolver = VerbResolverFactory.newInstance(); + + public static CommunityConfiguration getCommunityConfiguration() throws IOException { + final Map communities = Maps.newHashMap(); + List validCommunities = new ArrayList<>(); + getValidCommunities() + .forEach(community -> { + try { + CommunityModel cm = MAPPER.readValue(QueryCommunityAPI.community(community.getId()), CommunityModel.class); + validCommunities.add(getCommunity(cm)); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + validCommunities.forEach(community ->{ + try { + DatasourceList dl = MAPPER.readValue(QueryCommunityAPI.communityDatasource(community.getId()), DatasourceList.class); + community.setProviders(dl.stream().map(d -> { +// if(d.getEnabled() == null || Boolean.FALSE.equals(d.getEnabled())) +// return null; + Provider p = new Provider(); + p.setOpenaireId("10|" + d.getOpenaireId()); + p.setSelectionConstraints(d.getSelectioncriteria()); + if(p.getSelectionConstraints() != null) + p.getSelectionConstraints().setSelection(resolver); + return p; + }) + .filter(Objects::nonNull) + .collect(Collectors.toList())); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + validCommunities.forEach(community ->{ + if(community.isValid()) + communities.put(community.getId(), community); + }); + return new CommunityConfiguration(communities); + } + + private static Community getCommunity(CommunityModel cm){ + Community c = new Community(); + c.setId(cm.getId()); + c.setZenodoCommunities(cm.getOtherZenodoCommunities()); + if(!StringUtils.isNullOrEmpty(cm.getZenodoCommunity())) + c.getZenodoCommunities().add(cm.getZenodoCommunity()); + c.setSubjects(cm.getSubjects()); + c.getSubjects().addAll(cm.getFos()); + c.getSubjects().addAll(cm.getSdg()); + c.setConstraints(cm.getAdvancedConstraints()); + if(c.getConstraints()!=null) + c.getConstraints().setSelection(resolver); + c.setRemoveConstraints(cm.getRemoveConstraints()); + if(c.getRemoveConstraints()!=null) + c.getRemoveConstraints().setSelection(resolver); + return c; + } + + public static List getValidCommunities() throws IOException { + return MAPPER.readValue(QueryCommunityAPI.communities(), CommunitySummary.class) + .stream() + .filter(community -> !community.getStatus().equals("hidden") && + (community.getType().equals("ri") || community.getType().equals("community"))) + .collect(Collectors.toList()); + } + public static CommunityEntityMap getCommunityOrganization() throws IOException { + CommunityEntityMap organizationMap = new CommunityEntityMap(); + getValidCommunities() + .forEach(community -> { + String id = community.getId(); + try { + List associatedOrgs = MAPPER.readValue(QueryCommunityAPI.communityPropagationOrganization(id), OrganizationList.class); + if(associatedOrgs.size() >0){ + organizationMap.put(id, associatedOrgs); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + return organizationMap; + } + + public static CommunityEntityMap getCommunityProjects()throws IOException{ + CommunityEntityMap projectMap = new CommunityEntityMap(); + getValidCommunities() + .forEach(community ->{ + int page = -1; + int size = 100; + ContentModel cm = new ContentModel(); + List projectList = new ArrayList<>(); + do{ + page ++; + try { + cm = MAPPER.readValue( QueryCommunityAPI.communityProjects(community.getId(), String.valueOf(page), String.valueOf(size)), ContentModel.class); + if (cm.getContent().size() > 0){ + + cm.getContent().forEach(p -> + projectList.add ("40|" + p.getOpenaireId())); + projectMap.put(community.getId(), projectList); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + }while (!cm.getLast()); + }); + return projectMap; + } +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityContentprovider.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityContentprovider.java new file mode 100644 index 000000000..5378e556a --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityContentprovider.java @@ -0,0 +1,43 @@ +package eu.dnetlib.dhp.api.model; + +import com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.google.gson.Gson; +import eu.dnetlib.dhp.bulktag.community.SelectionConstraints; + + +@JsonAutoDetect +@JsonIgnoreProperties(ignoreUnknown = true) +public class CommunityContentprovider { + private String openaireId; + private SelectionConstraints selectioncriteria; + + private String enabled; + + public String getEnabled() { + return enabled; + } + + public void setEnabled(String enabled) { + this.enabled = enabled; + } + + public String getOpenaireId() { + return openaireId; + } + + public void setOpenaireId(final String openaireId) { + this.openaireId = openaireId; + } + + + public SelectionConstraints getSelectioncriteria() { + + return this.selectioncriteria; + } + + public void setSelectioncriteria(SelectionConstraints selectioncriteria) { + this.selectioncriteria = selectioncriteria; + + } +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityEntityMap.java similarity index 59% rename from dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java rename to dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityEntityMap.java index 7d786058a..ca3eb2857 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/OrganizationMap.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityEntityMap.java @@ -1,13 +1,13 @@ -package eu.dnetlib.dhp.resulttocommunityfromorganization; +package eu.dnetlib.dhp.api.model; import java.util.ArrayList; import java.util.HashMap; import java.util.List; -public class OrganizationMap extends HashMap> { +public class CommunityEntityMap extends HashMap> { - public OrganizationMap() { + public CommunityEntityMap() { super(); } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityModel.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityModel.java new file mode 100644 index 000000000..c5b3fac7c --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityModel.java @@ -0,0 +1,108 @@ + +package eu.dnetlib.dhp.api.model; + +import java.io.Serializable; +import java.util.List; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import eu.dnetlib.dhp.bulktag.community.SelectionConstraints; + + +/** + * @author miriam.baglioni + * @Date 06/10/23 + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class CommunityModel implements Serializable { + private String id; + private String type; + private String status; + + private String zenodoCommunity; + private List subjects; + private List otherZenodoCommunities; + private List fos; + private List sdg; + private SelectionConstraints advancedConstraints; + private SelectionConstraints removeConstraints; + + public String getZenodoCommunity() { + return zenodoCommunity; + } + + public void setZenodoCommunity(String zenodoCommunity) { + this.zenodoCommunity = zenodoCommunity; + } + + public List getSubjects() { + return subjects; + } + + public void setSubjects(List subjects) { + this.subjects = subjects; + } + + public List getOtherZenodoCommunities() { + return otherZenodoCommunities; + } + + public void setOtherZenodoCommunities(List otherZenodoCommunities) { + this.otherZenodoCommunities = otherZenodoCommunities; + } + + public List getFos() { + return fos; + } + + public void setFos(List fos) { + this.fos = fos; + } + + public List getSdg() { + return sdg; + } + + public void setSdg(List sdg) { + this.sdg = sdg; + } + + public SelectionConstraints getRemoveConstraints() { + return removeConstraints; + } + + public void setRemoveConstraints(SelectionConstraints removeConstraints) { + this.removeConstraints = removeConstraints; + } + + public SelectionConstraints getAdvancedConstraints() { + return advancedConstraints; + } + + public void setAdvancedConstraints(SelectionConstraints advancedConstraints) { + this.advancedConstraints = advancedConstraints; + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + public String getStatus() { + return status; + } + + public void setStatus(String status) { + this.status = status; + } +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunitySummary.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunitySummary.java new file mode 100644 index 000000000..774b6c874 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunitySummary.java @@ -0,0 +1,16 @@ +package eu.dnetlib.dhp.api.model; + +import java.io.Serializable; +import java.util.ArrayList; + +/** + * @author miriam.baglioni + * @Date 06/10/23 + */ +public class CommunitySummary extends ArrayList implements Serializable { + public CommunitySummary() { + super(); + } + } + + diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/ContentModel.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/ContentModel.java new file mode 100644 index 000000000..a755a9455 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/ContentModel.java @@ -0,0 +1,50 @@ +package eu.dnetlib.dhp.api.model; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; + +import java.io.Serializable; +import java.util.List; + +/** + * @author miriam.baglioni + * @Date 09/10/23 + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class ContentModel implements Serializable { + private List content; + private Integer totalPages; + private Boolean last; + private Integer number; + + public List getContent() { + return content; + } + + public void setContent(List content) { + this.content = content; + } + + public Integer getTotalPages() { + return totalPages; + } + + public void setTotalPages(Integer totalPages) { + this.totalPages = totalPages; + } + + public Boolean getLast() { + return last; + } + + public void setLast(Boolean last) { + this.last = last; + } + + public Integer getNumber() { + return number; + } + + public void setNumber(Integer number) { + this.number = number; + } +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/DatasourceList.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/DatasourceList.java new file mode 100644 index 000000000..8e31d7612 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/DatasourceList.java @@ -0,0 +1,12 @@ +package eu.dnetlib.dhp.api.model; + +import eu.dnetlib.dhp.api.model.CommunityContentprovider; + + +import java.io.Serializable; +import java.util.ArrayList; +public class DatasourceList extends ArrayList implements Serializable { + public DatasourceList(){ + super(); + } +} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/OrganizationList.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/OrganizationList.java new file mode 100644 index 000000000..6895967ff --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/OrganizationList.java @@ -0,0 +1,15 @@ +package eu.dnetlib.dhp.api.model; + +import java.io.Serializable; +import java.util.ArrayList; + +/** + * @author miriam.baglioni + * @Date 09/10/23 + */ +public class OrganizationList extends ArrayList implements Serializable { + + public OrganizationList(){ + super(); + } +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/ProjectModel.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/ProjectModel.java new file mode 100644 index 000000000..50bdcf649 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/ProjectModel.java @@ -0,0 +1,23 @@ +package eu.dnetlib.dhp.api.model; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; + +import java.io.Serializable; + +/** + * @author miriam.baglioni + * @Date 09/10/23 + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class ProjectModel implements Serializable { + + private String openaireId; + + public String getOpenaireId() { + return openaireId; + } + + public void setOpenaireId(String openaireId) { + this.openaireId = openaireId; + } +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java index 3186ed5c0..b61bc2c9f 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java @@ -6,10 +6,10 @@ import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import java.util.*; +import eu.dnetlib.dhp.api.Utils; 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; @@ -87,7 +87,7 @@ public class SparkBulkTagJob { if (isTest) { cc = CommunityConfigurationFactory.newInstance(taggingConf); } else { - cc = QueryInformationSystem.getCommunityConfiguration(parser.get("isLookUpUrl")); + cc = Utils.getCommunityConfiguration();//QueryInformationSystem.getCommunityConfiguration(parser.get("isLookUpUrl")); } runWithSparkSession( diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Community.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Community.java index b44376e22..d281f8783 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Community.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Community.java @@ -4,16 +4,18 @@ package eu.dnetlib.dhp.bulktag.community; import java.io.Serializable; import java.util.ArrayList; import java.util.List; +import java.util.Optional; import com.google.gson.Gson; + /** Created by miriam on 01/08/2018. */ public class Community implements Serializable { private String id; private List subjects = new ArrayList<>(); private List providers = new ArrayList<>(); - private List zenodoCommunities = new ArrayList<>(); + private List zenodoCommunities = new ArrayList<>(); private SelectionConstraints constraints = new SelectionConstraints(); private SelectionConstraints removeConstraints = new SelectionConstraints(); @@ -26,7 +28,7 @@ public class Community implements Serializable { return !getSubjects().isEmpty() || !getProviders().isEmpty() || !getZenodoCommunities().isEmpty() - || getConstraints().getCriteria() != null; + || (Optional.ofNullable(getConstraints()).isPresent() && getConstraints().getCriteria() != null); } public String getId() { @@ -53,11 +55,11 @@ public class Community implements Serializable { this.providers = providers; } - public List getZenodoCommunities() { + public List getZenodoCommunities() { return zenodoCommunities; } - public void setZenodoCommunities(List zenodoCommunities) { + public void setZenodoCommunities(List zenodoCommunities) { this.zenodoCommunities = zenodoCommunities; } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfiguration.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfiguration.java index e061ccd5e..a658c7ff5 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfiguration.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfiguration.java @@ -81,7 +81,7 @@ public class CommunityConfiguration implements Serializable { this.removeConstraintsMap = removeConstraintsMap; } - CommunityConfiguration(final Map communities) { + public CommunityConfiguration(final Map communities) { this.communities = communities; init(); } @@ -117,10 +117,10 @@ public class CommunityConfiguration implements Serializable { add(d.getOpenaireId(), new Pair<>(id, d.getSelectionConstraints()), datasourceMap); } // get zenodo communities - for (ZenodoCommunity zc : c.getZenodoCommunities()) { + for (String zc : c.getZenodoCommunities()) { add( - zc.getZenodoCommunityId(), - new Pair<>(id, zc.getSelCriteria()), + zc, + new Pair<>(id, null), zenodocommunityMap); } selectionConstraintsMap.put(id, c.getConstraints()); diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfigurationFactory.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfigurationFactory.java index 7b9e03ef6..013bf7168 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfigurationFactory.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfigurationFactory.java @@ -5,7 +5,6 @@ import java.io.StringReader; import java.util.ArrayList; import java.util.List; import java.util.Map; - import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -143,16 +142,16 @@ public class CommunityConfigurationFactory { return providerList; } - private static List parseZenodoCommunities(final Node node) { + private static List parseZenodoCommunities(final Node node) { final List list = node.selectNodes("./zenodocommunities/zenodocommunity"); - final List zenodoCommunityList = new ArrayList<>(); + final List zenodoCommunityList = new ArrayList<>(); for (Node n : list) { - ZenodoCommunity zc = new ZenodoCommunity(); - zc.setZenodoCommunityId(n.selectSingleNode("./zenodoid").getText()); - zc.setSelCriteria(n.selectSingleNode("./selcriteria")); +// ZenodoCommunity zc = new ZenodoCommunity(); +// zc.setZenodoCommunityId(n.selectSingleNode("./zenodoid").getText()); +// zc.setSelCriteria(n.selectSingleNode("./selcriteria")); - zenodoCommunityList.add(zc); + zenodoCommunityList.add(n.selectSingleNode("./zenodoid").getText()); } log.info("size of the zenodo community list " + zenodoCommunityList.size()); diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java index ed58cc14d..03ae1117b 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java @@ -6,12 +6,14 @@ import java.lang.reflect.InvocationTargetException; import eu.dnetlib.dhp.bulktag.criteria.Selection; import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; +import org.apache.htrace.fasterxml.jackson.annotation.JsonIgnore; public class Constraint implements Serializable { private String verb; private String field; private String value; // private String element; + @JsonIgnore private Selection selection; public String getVerb() { @@ -37,11 +39,11 @@ public class Constraint implements Serializable { public void setValue(String value) { this.value = value; } - - public void setSelection(Selection sel) { - selection = sel; - } - +//@JsonIgnore + //public void setSelection(Selection sel) { +// selection = sel; +// } +@JsonIgnore public void setSelection(VerbResolver resolver) throws InvocationTargetException, NoSuchMethodException, InstantiationException, IllegalAccessException { @@ -52,11 +54,5 @@ public class Constraint implements Serializable { return selection.apply(metadata); } -// public String getElement() { -// return element; -// } -// -// public void setElement(String element) { -// this.element = element; -// } + } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java index c7dcce812..8e6d25e9b 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java @@ -7,11 +7,12 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import com.fasterxml.jackson.annotation.JsonAutoDetect; import com.google.gson.Gson; import com.google.gson.reflect.TypeToken; import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; - +@JsonAutoDetect public class SelectionConstraints implements Serializable { private List criteria; diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java index 0fc8cb390..ff6b73286 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java @@ -6,12 +6,12 @@ import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; import java.util.*; +import eu.dnetlib.dhp.api.Utils; +import eu.dnetlib.dhp.api.model.CommunityEntityMap; 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.MapFunction; -import org.apache.spark.api.java.function.MapGroupsFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; @@ -48,10 +48,11 @@ public class PrepareResultCommunitySet { final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); - final OrganizationMap organizationMap = new Gson() - .fromJson( - parser.get("organizationtoresultcommunitymap"), - OrganizationMap.class); +// final CommunityEntityMap organizationMap = new Gson() +// .fromJson( +// parser.get("organizationtoresultcommunitymap"), +// CommunityEntityMap.class); + final CommunityEntityMap organizationMap = Utils.getCommunityOrganization(); log.info("organizationMap: {}", new Gson().toJson(organizationMap)); SparkConf conf = new SparkConf(); @@ -70,7 +71,7 @@ public class PrepareResultCommunitySet { SparkSession spark, String inputPath, String outputPath, - OrganizationMap organizationMap) { + CommunityEntityMap organizationMap) { Dataset relation = readPath(spark, inputPath, Relation.class); relation.createOrReplaceTempView("relation"); @@ -115,7 +116,7 @@ public class PrepareResultCommunitySet { } private static MapFunction mapResultCommunityFn( - OrganizationMap organizationMap) { + CommunityEntityMap organizationMap) { return value -> { String rId = value.getResultId(); Optional> orgs = Optional.ofNullable(value.getMerges()); diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml index 6a329fdc4..d9805ab7b 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml @@ -4,10 +4,10 @@ sourcePath the source path - - organizationtoresultcommunitymap - organization community map - + + + + outputPath the output path diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java index 660a55472..745cd7e6f 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java @@ -1568,4 +1568,42 @@ public class BulkTagJobTest { } + @Test + void newConfTest() throws Exception { + final String pathMap = BulkTagJobTest.pathMap; + SparkBulkTagJob + .main( + new String[] { + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", + getClass().getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates").getPath(), + "-taggingConf", taggingConf, + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", workingDir.toString() + "/dataset", + "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + "-pathMap", pathMap + }); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + + Assertions.assertEquals(10, tmp.count()); + org.apache.spark.sql.Dataset verificationDataset = spark + .createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + + verificationDataset.createOrReplaceTempView("dataset"); + + String query = "select id, MyT.id community " + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'bulktagging'"; + + Assertions.assertEquals(0, spark.sql(query).count()); + } + } diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java index c8fd62c8e..7e12bf9c3 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java @@ -47,7 +47,7 @@ class CommunityConfigurationFactoryTest { sc.setVerb("not_contains"); sc.setField("contributor"); sc.setValue("DARIAH"); - sc.setSelection(resolver.getSelectionCriteria(sc.getVerb(), sc.getValue())); + sc.setSelection(resolver);//.getSelectionCriteria(sc.getVerb(), sc.getValue())); String metadata = "This work has been partially supported by DARIAH-EU infrastructure"; Assertions.assertFalse(sc.verifyCriteria(metadata)); } diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/QueryCommunityAPITest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/QueryCommunityAPITest.java new file mode 100644 index 000000000..1b3403535 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/QueryCommunityAPITest.java @@ -0,0 +1,91 @@ + +package eu.dnetlib.dhp.bulktag; + +import eu.dnetlib.dhp.api.Utils; +import eu.dnetlib.dhp.api.model.CommunityEntityMap; +import eu.dnetlib.dhp.bulktag.community.Community; +import eu.dnetlib.dhp.bulktag.community.CommunityConfiguration; + +import eu.dnetlib.dhp.api.model.CommunityModel; +import eu.dnetlib.dhp.api.model.CommunitySummary; +import eu.dnetlib.dhp.api.model.DatasourceList; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.api.QueryCommunityAPI; + +import java.util.List; + +/** + * @author miriam.baglioni + * @Date 06/10/23 + */ +public class QueryCommunityAPITest { + + @Test + void communityList() throws Exception { + String body = QueryCommunityAPI.communities(); + new ObjectMapper() + .readValue(body, CommunitySummary.class) + .forEach(p -> { + try { + System.out.println(new ObjectMapper().writeValueAsString(p)); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }); + } + + @Test + void community() throws Exception { + String id = "dh-ch"; + String body = QueryCommunityAPI.community(id); + System.out.println(new ObjectMapper().writeValueAsString(new ObjectMapper() + .readValue(body, CommunityModel.class))) + ; + } + + @Test + void communityDatasource() throws Exception { + String id = "dh-ch"; + String body = QueryCommunityAPI.communityDatasource(id); + new ObjectMapper() + .readValue(body, DatasourceList.class) + .forEach(ds-> { + try { + System.out.println(new ObjectMapper().writeValueAsString(ds)); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }); + ; + } + + @Test + void validCommunities() throws Exception { + CommunityConfiguration cc = Utils.getCommunityConfiguration(); + System.out.println(cc.getCommunities().keySet()); + Community community =cc.getCommunities().get("aurora"); + Assertions.assertEquals(0, community.getSubjects().size()); + Assertions.assertEquals(null, community.getConstraints()); + Assertions.assertEquals(null, community.getRemoveConstraints()); + Assertions.assertEquals(2, community.getZenodoCommunities().size()); + Assertions.assertTrue(community.getZenodoCommunities().stream().anyMatch(c -> c.equals("aurora-universities-network"))); + Assertions.assertTrue(community.getZenodoCommunities().stream().anyMatch(c -> c.equals("university-of-innsbruck"))); + Assertions.assertEquals(35, community.getProviders().size()); + Assertions.assertEquals(35, community.getProviders().stream().filter(p->p.getSelectionConstraints()==null).count()); + } + + @Test + void getCommunityProjects() throws Exception { + CommunityEntityMap projectMap = Utils.getCommunityProjects(); + Assertions.assertFalse(projectMap.containsKey("mes")); + Assertions.assertEquals(33, projectMap.size()); + Assertions.assertTrue(projectMap.keySet().stream().allMatch(k -> projectMap.get(k).stream().allMatch(p -> p.startsWith("40|")))); + } + +} + diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/group/GroupEntitiesSparkJobTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/group/GroupEntitiesSparkJobTest.java index b878e778e..0887adf45 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/group/GroupEntitiesSparkJobTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/group/GroupEntitiesSparkJobTest.java @@ -1,14 +1,14 @@ package eu.dnetlib.dhp.oa.graph.group; -import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.ObjectMapper; -import eu.dnetlib.dhp.common.HdfsSupport; -import eu.dnetlib.dhp.oa.merge.GroupEntitiesSparkJob; -import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.OafEntity; -import eu.dnetlib.dhp.schema.oaf.Result; -import eu.dnetlib.dhp.utils.DHPUtils; +import static org.junit.jupiter.api.Assertions.assertEquals; + +import java.io.IOException; +import java.net.URISyntaxException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; + import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.FilterFunction; @@ -18,108 +18,108 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; import org.junit.jupiter.api.*; -import java.io.IOException; -import java.net.URISyntaxException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; -import static org.junit.jupiter.api.Assertions.assertEquals; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.oa.merge.GroupEntitiesSparkJob; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.OafEntity; +import eu.dnetlib.dhp.schema.oaf.Result; +import eu.dnetlib.dhp.utils.DHPUtils; @TestMethodOrder(MethodOrderer.OrderAnnotation.class) public class GroupEntitiesSparkJobTest { - private static SparkSession spark; + private static SparkSession spark; - private static ObjectMapper mapper = new ObjectMapper() - .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + private static ObjectMapper mapper = new ObjectMapper() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - private static Path workingDir; - private Path dataInputPath; + private static Path workingDir; + private Path dataInputPath; - private Path checkpointPath; + private Path checkpointPath; - private Path outputPath; + private Path outputPath; - @BeforeAll - public static void beforeAll() throws IOException { - workingDir = Files.createTempDirectory(GroupEntitiesSparkJob.class.getSimpleName()); + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(GroupEntitiesSparkJob.class.getSimpleName()); - SparkConf conf = new SparkConf(); - conf.setAppName(GroupEntitiesSparkJob.class.getSimpleName()); - conf.setMaster("local"); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.registerKryoClasses(ModelSupport.getOafModelClasses()); - spark = SparkSession.builder().config(conf).getOrCreate(); - } + SparkConf conf = new SparkConf(); + conf.setAppName(GroupEntitiesSparkJob.class.getSimpleName()); + conf.setMaster("local"); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); + spark = SparkSession.builder().config(conf).getOrCreate(); + } - @BeforeEach - public void beforeEach() throws IOException, URISyntaxException { - dataInputPath = Paths.get(ClassLoader.getSystemResource("eu/dnetlib/dhp/oa/graph/group").toURI()); - checkpointPath = workingDir.resolve("grouped_entity"); - outputPath = workingDir.resolve("dispatched_entity"); - } + @BeforeEach + public void beforeEach() throws IOException, URISyntaxException { + dataInputPath = Paths.get(ClassLoader.getSystemResource("eu/dnetlib/dhp/oa/graph/group").toURI()); + checkpointPath = workingDir.resolve("grouped_entity"); + outputPath = workingDir.resolve("dispatched_entity"); + } - @AfterAll - public static void afterAll() throws IOException { - spark.stop(); - FileUtils.deleteDirectory(workingDir.toFile()); - } + @AfterAll + public static void afterAll() throws IOException { + spark.stop(); + FileUtils.deleteDirectory(workingDir.toFile()); + } - @Test - @Order(1) - void testGroupEntities() throws Exception { - GroupEntitiesSparkJob.main(new String[]{ - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-graphInputPath", - dataInputPath.toString(), - "-checkpointPath", - checkpointPath.toString(), - "-outputPath", - outputPath.toString(), - "-filterInvisible", - Boolean.FALSE.toString() - }); + @Test + @Order(1) + void testGroupEntities() throws Exception { + GroupEntitiesSparkJob.main(new String[] { + "-isSparkSessionManaged", + Boolean.FALSE.toString(), + "-graphInputPath", + dataInputPath.toString(), + "-checkpointPath", + checkpointPath.toString(), + "-outputPath", + outputPath.toString(), + "-filterInvisible", + Boolean.FALSE.toString() + }); - Dataset checkpointTable = spark - .read() - .load(checkpointPath.toString()) - .selectExpr("COALESCE(*)") - .as(Encoders.kryo(OafEntity.class)); + Dataset checkpointTable = spark + .read() + .load(checkpointPath.toString()) + .selectExpr("COALESCE(*)") + .as(Encoders.kryo(OafEntity.class)); + assertEquals( + 1, + checkpointTable + .filter( + (FilterFunction) r -> "50|doi_________::09821844208a5cd6300b2bfb13bca1b9" + .equals(r.getId()) && + r.getCollectedfrom().stream().anyMatch(kv -> kv.getValue().equalsIgnoreCase("zenodo"))) + .count()); - assertEquals( - 1, - checkpointTable - .filter( - (FilterFunction) r -> "50|doi_________::09821844208a5cd6300b2bfb13bca1b9" - .equals(r.getId()) && - r.getCollectedfrom().stream().anyMatch(kv -> kv.getValue().equalsIgnoreCase("zenodo"))) - .count()); + Dataset output = spark + .read() + .textFile( + DHPUtils + .toSeq( + HdfsSupport + .listFiles(outputPath.toString(), spark.sparkContext().hadoopConfiguration()))) + .map((MapFunction) s -> mapper.readValue(s, Result.class), Encoders.bean(Result.class)); - - Dataset output = spark - .read() - .textFile( - DHPUtils - .toSeq( - HdfsSupport - .listFiles(outputPath.toString(), spark.sparkContext().hadoopConfiguration()))) - .map((MapFunction) s -> mapper.readValue(s, Result.class), Encoders.bean(Result.class)); - - assertEquals(3, output.count()); - assertEquals( - 2, - output - .map((MapFunction) r -> r.getResulttype().getClassid(), Encoders.STRING()) - .filter((FilterFunction) s -> s.equals("publication")) - .count()); - assertEquals( - 1, - output - .map((MapFunction) r -> r.getResulttype().getClassid(), Encoders.STRING()) - .filter((FilterFunction) s -> s.equals("dataset")) - .count()); - } -} \ No newline at end of file + assertEquals(3, output.count()); + assertEquals( + 2, + output + .map((MapFunction) r -> r.getResulttype().getClassid(), Encoders.STRING()) + .filter((FilterFunction) s -> s.equals("publication")) + .count()); + assertEquals( + 1, + output + .map((MapFunction) r -> r.getResulttype().getClassid(), Encoders.STRING()) + .filter((FilterFunction) s -> s.equals("dataset")) + .count()); + } +} From a3d01ccb246e52e7977aef8c57b15d964dcc9834 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 9 Oct 2023 14:52:17 +0200 Subject: [PATCH 02/60] refactoring --- .../CreateActionSetSparkJob.java | 2 +- .../doiboost/crossref/Crossref2Oaf.scala | 16 +- .../crossref/CrossrefMappingTest.scala | 2 +- .../eu/dnetlib/dhp/api/QueryCommunityAPI.java | 9 +- .../main/java/eu/dnetlib/dhp/api/Utils.java | 236 +++++++++--------- .../api/model/CommunityContentprovider.java | 6 +- .../dnetlib/dhp/api/model/CommunityModel.java | 2 +- .../dhp/api/model/CommunitySummary.java | 11 +- .../dnetlib/dhp/api/model/ContentModel.java | 61 ++--- .../dnetlib/dhp/api/model/DatasourceList.java | 11 +- .../dhp/api/model/OrganizationList.java | 7 +- .../dnetlib/dhp/api/model/ProjectModel.java | 19 +- .../dnetlib/dhp/bulktag/SparkBulkTagJob.java | 4 +- .../dhp/bulktag/community/Community.java | 1 - .../CommunityConfigurationFactory.java | 1 + .../dhp/bulktag/community/Constraint.java | 9 +- .../community/SelectionConstraints.java | 1 + .../PrepareResultCommunitySet.java | 4 +- .../dnetlib/dhp/bulktag/BulkTagJobTest.java | 38 +-- .../CommunityConfigurationFactoryTest.java | 2 +- .../dhp/bulktag/QueryCommunityAPITest.java | 65 +++-- 21 files changed, 268 insertions(+), 239 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/CreateActionSetSparkJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/CreateActionSetSparkJob.java index a367ba852..b707fdcd3 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/CreateActionSetSparkJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/opencitations/CreateActionSetSparkJob.java @@ -7,7 +7,6 @@ import java.io.IOException; import java.io.Serializable; import java.util.*; -import eu.dnetlib.dhp.schema.oaf.utils.*; import org.apache.commons.cli.ParseException; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.Text; @@ -30,6 +29,7 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.action.AtomicAction; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.utils.*; import eu.dnetlib.dhp.utils.DHPUtils; import scala.Tuple2; diff --git a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala index e0fdb9ce4..565d34e62 100644 --- a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala @@ -31,9 +31,7 @@ case class mappingAuthor( affiliation: Option[mappingAffiliation] ) {} -case class funderInfo(id:String,uri:String, name:String,synonym:List[String] ) {} - - +case class funderInfo(id: String, uri: String, name: String, synonym: List[String]) {} case class mappingFunder(name: String, DOI: Option[String], award: Option[List[String]]) {} @@ -41,7 +39,9 @@ case object Crossref2Oaf { val logger: Logger = LoggerFactory.getLogger(Crossref2Oaf.getClass) val irishFunder: List[funderInfo] = { - val s = Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/crossref/irish_funder.json")).mkString + val s = Source + .fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/doiboost/crossref/irish_funder.json")) + .mkString implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats lazy val json: org.json4s.JValue = parse(s) json.extract[List[funderInfo]] @@ -100,9 +100,11 @@ case object Crossref2Oaf { "report" -> "0017 Report" ) - def getIrishId(doi:String):Option[String] = { - val id =doi.split("/").last - irishFunder.find(f => id.equalsIgnoreCase(f.id) || (f.synonym.nonEmpty && f.synonym.exists(s => s.equalsIgnoreCase(id)))).map(f => f.id) + def getIrishId(doi: String): Option[String] = { + val id = doi.split("/").last + irishFunder + .find(f => id.equalsIgnoreCase(f.id) || (f.synonym.nonEmpty && f.synonym.exists(s => s.equalsIgnoreCase(id)))) + .map(f => f.id) } def mappingResult(result: Result, json: JValue, cobjCategory: String): Result = { diff --git a/dhp-workflows/dhp-doiboost/src/test/scala/eu/dnetlib/dhp/doiboost/crossref/CrossrefMappingTest.scala b/dhp-workflows/dhp-doiboost/src/test/scala/eu/dnetlib/dhp/doiboost/crossref/CrossrefMappingTest.scala index 7961376c5..fbf6f72c0 100644 --- a/dhp-workflows/dhp-doiboost/src/test/scala/eu/dnetlib/dhp/doiboost/crossref/CrossrefMappingTest.scala +++ b/dhp-workflows/dhp-doiboost/src/test/scala/eu/dnetlib/dhp/doiboost/crossref/CrossrefMappingTest.scala @@ -50,7 +50,7 @@ class CrossrefMappingTest { } } - def checkRelation(generatedOAF: List[Oaf]): Unit = { + def checkRelation(generatedOAF: List[Oaf]): Unit = { val rels: List[Relation] = generatedOAF.filter(p => p.isInstanceOf[Relation]).asInstanceOf[List[Relation]] diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java index 635ee2027..cc615ba46 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java @@ -21,7 +21,7 @@ import com.google.gson.Gson; public class QueryCommunityAPI { private static final String baseUrl = "https://services.openaire.eu/openaire/"; - private static String get(String geturl) throws IOException{ + private static String get(String geturl) throws IOException { URL url = new URL(geturl); HttpURLConnection conn = (HttpURLConnection) url.openConnection(); conn.setDoOutput(true); @@ -44,7 +44,7 @@ public class QueryCommunityAPI { return get(baseUrl + "community/" + id); } - public static String communityDatasource(String id)throws IOException{ + public static String communityDatasource(String id) throws IOException { return get(baseUrl + "community/" + id + "/contentproviders"); } @@ -53,8 +53,8 @@ public class QueryCommunityAPI { return get(baseUrl + "community/" + id + "/propagationOrganizations"); } - public static String communityProjects(String id, String page, String size) throws IOException{ - return get(baseUrl + "community/" + id +"/projects/" + page + "/" + size); + public static String communityProjects(String id, String page, String size) throws IOException { + return get(baseUrl + "community/" + id + "/projects/" + page + "/" + size); } @NotNull @@ -74,5 +74,4 @@ public class QueryCommunityAPI { return body; } - } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java index 5b3004a5d..c1aaa14c4 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java @@ -1,16 +1,6 @@ + package eu.dnetlib.dhp.api; -import com.amazonaws.util.StringUtils; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.Maps; -import eu.dnetlib.dhp.api.model.*; -import eu.dnetlib.dhp.bulktag.community.Community; -import eu.dnetlib.dhp.bulktag.community.CommunityConfiguration; -import eu.dnetlib.dhp.bulktag.community.Provider; -import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; -import eu.dnetlib.dhp.bulktag.criteria.VerbResolverFactory; - -import javax.management.Query; import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; @@ -19,118 +9,140 @@ import java.util.Map; import java.util.Objects; import java.util.stream.Collectors; +import javax.management.Query; + +import com.amazonaws.util.StringUtils; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Maps; + +import eu.dnetlib.dhp.api.model.*; +import eu.dnetlib.dhp.bulktag.community.Community; +import eu.dnetlib.dhp.bulktag.community.CommunityConfiguration; +import eu.dnetlib.dhp.bulktag.community.Provider; +import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; +import eu.dnetlib.dhp.bulktag.criteria.VerbResolverFactory; + /** * @author miriam.baglioni * @Date 09/10/23 */ public class Utils implements Serializable { - private static final ObjectMapper MAPPER = new ObjectMapper(); - private static final VerbResolver resolver = VerbResolverFactory.newInstance(); + private static final ObjectMapper MAPPER = new ObjectMapper(); + private static final VerbResolver resolver = VerbResolverFactory.newInstance(); - public static CommunityConfiguration getCommunityConfiguration() throws IOException { - final Map communities = Maps.newHashMap(); - List validCommunities = new ArrayList<>(); - getValidCommunities() - .forEach(community -> { - try { - CommunityModel cm = MAPPER.readValue(QueryCommunityAPI.community(community.getId()), CommunityModel.class); - validCommunities.add(getCommunity(cm)); - } catch (IOException e) { - throw new RuntimeException(e); - } - }); - validCommunities.forEach(community ->{ - try { - DatasourceList dl = MAPPER.readValue(QueryCommunityAPI.communityDatasource(community.getId()), DatasourceList.class); - community.setProviders(dl.stream().map(d -> { + public static CommunityConfiguration getCommunityConfiguration() throws IOException { + final Map communities = Maps.newHashMap(); + List validCommunities = new ArrayList<>(); + getValidCommunities() + .forEach(community -> { + try { + CommunityModel cm = MAPPER + .readValue(QueryCommunityAPI.community(community.getId()), CommunityModel.class); + validCommunities.add(getCommunity(cm)); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + validCommunities.forEach(community -> { + try { + DatasourceList dl = MAPPER + .readValue(QueryCommunityAPI.communityDatasource(community.getId()), DatasourceList.class); + community.setProviders(dl.stream().map(d -> { // if(d.getEnabled() == null || Boolean.FALSE.equals(d.getEnabled())) // return null; - Provider p = new Provider(); - p.setOpenaireId("10|" + d.getOpenaireId()); - p.setSelectionConstraints(d.getSelectioncriteria()); - if(p.getSelectionConstraints() != null) - p.getSelectionConstraints().setSelection(resolver); - return p; - }) - .filter(Objects::nonNull) - .collect(Collectors.toList())); - } catch (IOException e) { - throw new RuntimeException(e); - } - }); + Provider p = new Provider(); + p.setOpenaireId("10|" + d.getOpenaireId()); + p.setSelectionConstraints(d.getSelectioncriteria()); + if (p.getSelectionConstraints() != null) + p.getSelectionConstraints().setSelection(resolver); + return p; + }) + .filter(Objects::nonNull) + .collect(Collectors.toList())); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); - validCommunities.forEach(community ->{ - if(community.isValid()) - communities.put(community.getId(), community); - }); - return new CommunityConfiguration(communities); - } + validCommunities.forEach(community -> { + if (community.isValid()) + communities.put(community.getId(), community); + }); + return new CommunityConfiguration(communities); + } - private static Community getCommunity(CommunityModel cm){ - Community c = new Community(); - c.setId(cm.getId()); - c.setZenodoCommunities(cm.getOtherZenodoCommunities()); - if(!StringUtils.isNullOrEmpty(cm.getZenodoCommunity())) - c.getZenodoCommunities().add(cm.getZenodoCommunity()); - c.setSubjects(cm.getSubjects()); - c.getSubjects().addAll(cm.getFos()); - c.getSubjects().addAll(cm.getSdg()); - c.setConstraints(cm.getAdvancedConstraints()); - if(c.getConstraints()!=null) - c.getConstraints().setSelection(resolver); - c.setRemoveConstraints(cm.getRemoveConstraints()); - if(c.getRemoveConstraints()!=null) - c.getRemoveConstraints().setSelection(resolver); - return c; - } + private static Community getCommunity(CommunityModel cm) { + Community c = new Community(); + c.setId(cm.getId()); + c.setZenodoCommunities(cm.getOtherZenodoCommunities()); + if (!StringUtils.isNullOrEmpty(cm.getZenodoCommunity())) + c.getZenodoCommunities().add(cm.getZenodoCommunity()); + c.setSubjects(cm.getSubjects()); + c.getSubjects().addAll(cm.getFos()); + c.getSubjects().addAll(cm.getSdg()); + c.setConstraints(cm.getAdvancedConstraints()); + if (c.getConstraints() != null) + c.getConstraints().setSelection(resolver); + c.setRemoveConstraints(cm.getRemoveConstraints()); + if (c.getRemoveConstraints() != null) + c.getRemoveConstraints().setSelection(resolver); + return c; + } - public static List getValidCommunities() throws IOException { - return MAPPER.readValue(QueryCommunityAPI.communities(), CommunitySummary.class) - .stream() - .filter(community -> !community.getStatus().equals("hidden") && - (community.getType().equals("ri") || community.getType().equals("community"))) - .collect(Collectors.toList()); - } - public static CommunityEntityMap getCommunityOrganization() throws IOException { - CommunityEntityMap organizationMap = new CommunityEntityMap(); - getValidCommunities() - .forEach(community -> { - String id = community.getId(); - try { - List associatedOrgs = MAPPER.readValue(QueryCommunityAPI.communityPropagationOrganization(id), OrganizationList.class); - if(associatedOrgs.size() >0){ - organizationMap.put(id, associatedOrgs); - } - } catch (IOException e) { - throw new RuntimeException(e); - } - }); - return organizationMap; - } + public static List getValidCommunities() throws IOException { + return MAPPER + .readValue(QueryCommunityAPI.communities(), CommunitySummary.class) + .stream() + .filter( + community -> !community.getStatus().equals("hidden") && + (community.getType().equals("ri") || community.getType().equals("community"))) + .collect(Collectors.toList()); + } - public static CommunityEntityMap getCommunityProjects()throws IOException{ - CommunityEntityMap projectMap = new CommunityEntityMap(); - getValidCommunities() - .forEach(community ->{ - int page = -1; - int size = 100; - ContentModel cm = new ContentModel(); - List projectList = new ArrayList<>(); - do{ - page ++; - try { - cm = MAPPER.readValue( QueryCommunityAPI.communityProjects(community.getId(), String.valueOf(page), String.valueOf(size)), ContentModel.class); - if (cm.getContent().size() > 0){ + public static CommunityEntityMap getCommunityOrganization() throws IOException { + CommunityEntityMap organizationMap = new CommunityEntityMap(); + getValidCommunities() + .forEach(community -> { + String id = community.getId(); + try { + List associatedOrgs = MAPPER + .readValue(QueryCommunityAPI.communityPropagationOrganization(id), OrganizationList.class); + if (associatedOrgs.size() > 0) { + organizationMap.put(id, associatedOrgs); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + return organizationMap; + } - cm.getContent().forEach(p -> - projectList.add ("40|" + p.getOpenaireId())); - projectMap.put(community.getId(), projectList); - } - } catch (IOException e) { - throw new RuntimeException(e); - } - }while (!cm.getLast()); - }); - return projectMap; - } + public static CommunityEntityMap getCommunityProjects() throws IOException { + CommunityEntityMap projectMap = new CommunityEntityMap(); + getValidCommunities() + .forEach(community -> { + int page = -1; + int size = 100; + ContentModel cm = new ContentModel(); + List projectList = new ArrayList<>(); + do { + page++; + try { + cm = MAPPER + .readValue( + QueryCommunityAPI + .communityProjects(community.getId(), String.valueOf(page), String.valueOf(size)), + ContentModel.class); + if (cm.getContent().size() > 0) { + + cm.getContent().forEach(p -> projectList.add("40|" + p.getOpenaireId())); + projectMap.put(community.getId(), projectList); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } while (!cm.getLast()); + }); + return projectMap; + } } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityContentprovider.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityContentprovider.java index 5378e556a..9fab5a80c 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityContentprovider.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityContentprovider.java @@ -1,10 +1,11 @@ + package eu.dnetlib.dhp.api.model; import com.fasterxml.jackson.annotation.JsonAutoDetect; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.google.gson.Gson; -import eu.dnetlib.dhp.bulktag.community.SelectionConstraints; +import eu.dnetlib.dhp.bulktag.community.SelectionConstraints; @JsonAutoDetect @JsonIgnoreProperties(ignoreUnknown = true) @@ -30,10 +31,9 @@ public class CommunityContentprovider { this.openaireId = openaireId; } - public SelectionConstraints getSelectioncriteria() { - return this.selectioncriteria; + return this.selectioncriteria; } public void setSelectioncriteria(SelectionConstraints selectioncriteria) { diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityModel.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityModel.java index c5b3fac7c..745e7efc2 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityModel.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunityModel.java @@ -5,8 +5,8 @@ import java.io.Serializable; import java.util.List; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; -import eu.dnetlib.dhp.bulktag.community.SelectionConstraints; +import eu.dnetlib.dhp.bulktag.community.SelectionConstraints; /** * @author miriam.baglioni diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunitySummary.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunitySummary.java index 774b6c874..a0541f7ee 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunitySummary.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/CommunitySummary.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.api.model; import java.io.Serializable; @@ -8,9 +9,7 @@ import java.util.ArrayList; * @Date 06/10/23 */ public class CommunitySummary extends ArrayList implements Serializable { - public CommunitySummary() { - super(); - } - } - - + public CommunitySummary() { + super(); + } +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/ContentModel.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/ContentModel.java index a755a9455..469709f59 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/ContentModel.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/ContentModel.java @@ -1,50 +1,51 @@ -package eu.dnetlib.dhp.api.model; -import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +package eu.dnetlib.dhp.api.model; import java.io.Serializable; import java.util.List; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; + /** * @author miriam.baglioni * @Date 09/10/23 */ @JsonIgnoreProperties(ignoreUnknown = true) public class ContentModel implements Serializable { - private List content; - private Integer totalPages; - private Boolean last; - private Integer number; + private List content; + private Integer totalPages; + private Boolean last; + private Integer number; - public List getContent() { - return content; - } + public List getContent() { + return content; + } - public void setContent(List content) { - this.content = content; - } + public void setContent(List content) { + this.content = content; + } - public Integer getTotalPages() { - return totalPages; - } + public Integer getTotalPages() { + return totalPages; + } - public void setTotalPages(Integer totalPages) { - this.totalPages = totalPages; - } + public void setTotalPages(Integer totalPages) { + this.totalPages = totalPages; + } - public Boolean getLast() { - return last; - } + public Boolean getLast() { + return last; + } - public void setLast(Boolean last) { - this.last = last; - } + public void setLast(Boolean last) { + this.last = last; + } - public Integer getNumber() { - return number; - } + public Integer getNumber() { + return number; + } - public void setNumber(Integer number) { - this.number = number; - } + public void setNumber(Integer number) { + this.number = number; + } } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/DatasourceList.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/DatasourceList.java index 8e31d7612..30d0241c3 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/DatasourceList.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/DatasourceList.java @@ -1,12 +1,13 @@ + package eu.dnetlib.dhp.api.model; -import eu.dnetlib.dhp.api.model.CommunityContentprovider; - - import java.io.Serializable; import java.util.ArrayList; + +import eu.dnetlib.dhp.api.model.CommunityContentprovider; + public class DatasourceList extends ArrayList implements Serializable { - public DatasourceList(){ + public DatasourceList() { super(); } -} \ No newline at end of file +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/OrganizationList.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/OrganizationList.java index 6895967ff..3c81ad179 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/OrganizationList.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/OrganizationList.java @@ -1,3 +1,4 @@ + package eu.dnetlib.dhp.api.model; import java.io.Serializable; @@ -9,7 +10,7 @@ import java.util.ArrayList; */ public class OrganizationList extends ArrayList implements Serializable { - public OrganizationList(){ - super(); - } + public OrganizationList() { + super(); + } } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/ProjectModel.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/ProjectModel.java index 50bdcf649..3495d6a63 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/ProjectModel.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/model/ProjectModel.java @@ -1,9 +1,10 @@ + package eu.dnetlib.dhp.api.model; -import com.fasterxml.jackson.annotation.JsonIgnoreProperties; - import java.io.Serializable; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; + /** * @author miriam.baglioni * @Date 09/10/23 @@ -11,13 +12,13 @@ import java.io.Serializable; @JsonIgnoreProperties(ignoreUnknown = true) public class ProjectModel implements Serializable { - private String openaireId; + private String openaireId; - public String getOpenaireId() { - return openaireId; - } + public String getOpenaireId() { + return openaireId; + } - public void setOpenaireId(String openaireId) { - this.openaireId = openaireId; - } + public void setOpenaireId(String openaireId) { + this.openaireId = openaireId; + } } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java index b61bc2c9f..0d98e4958 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java @@ -6,7 +6,6 @@ import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import java.util.*; -import eu.dnetlib.dhp.api.Utils; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.FilterFunction; @@ -21,6 +20,7 @@ import org.slf4j.LoggerFactory; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.gson.Gson; +import eu.dnetlib.dhp.api.Utils; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.bulktag.community.*; import eu.dnetlib.dhp.schema.oaf.Datasource; @@ -87,7 +87,7 @@ public class SparkBulkTagJob { if (isTest) { cc = CommunityConfigurationFactory.newInstance(taggingConf); } else { - cc = Utils.getCommunityConfiguration();//QueryInformationSystem.getCommunityConfiguration(parser.get("isLookUpUrl")); + cc = Utils.getCommunityConfiguration();// QueryInformationSystem.getCommunityConfiguration(parser.get("isLookUpUrl")); } runWithSparkSession( diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Community.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Community.java index d281f8783..9cd3a8f82 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Community.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Community.java @@ -8,7 +8,6 @@ import java.util.Optional; import com.google.gson.Gson; - /** Created by miriam on 01/08/2018. */ public class Community implements Serializable { diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfigurationFactory.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfigurationFactory.java index 013bf7168..955ca3856 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfigurationFactory.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/CommunityConfigurationFactory.java @@ -5,6 +5,7 @@ import java.io.StringReader; import java.util.ArrayList; import java.util.List; import java.util.Map; + import org.apache.commons.lang3.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java index 03ae1117b..82a6a3b85 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/Constraint.java @@ -4,9 +4,10 @@ package eu.dnetlib.dhp.bulktag.community; import java.io.Serializable; import java.lang.reflect.InvocationTargetException; +import org.apache.htrace.fasterxml.jackson.annotation.JsonIgnore; + import eu.dnetlib.dhp.bulktag.criteria.Selection; import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; -import org.apache.htrace.fasterxml.jackson.annotation.JsonIgnore; public class Constraint implements Serializable { private String verb; @@ -39,11 +40,12 @@ public class Constraint implements Serializable { public void setValue(String value) { this.value = value; } + //@JsonIgnore - //public void setSelection(Selection sel) { + // public void setSelection(Selection sel) { // selection = sel; // } -@JsonIgnore + @JsonIgnore public void setSelection(VerbResolver resolver) throws InvocationTargetException, NoSuchMethodException, InstantiationException, IllegalAccessException { @@ -54,5 +56,4 @@ public class Constraint implements Serializable { return selection.apply(metadata); } - } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java index 8e6d25e9b..57cc658fc 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/SelectionConstraints.java @@ -12,6 +12,7 @@ import com.google.gson.Gson; import com.google.gson.reflect.TypeToken; import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; + @JsonAutoDetect public class SelectionConstraints implements Serializable { private List criteria; diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java index ff6b73286..e0670b80f 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java @@ -6,8 +6,6 @@ import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; import java.util.*; -import eu.dnetlib.dhp.api.Utils; -import eu.dnetlib.dhp.api.model.CommunityEntityMap; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; @@ -20,6 +18,8 @@ import org.slf4j.LoggerFactory; import com.google.gson.Gson; +import eu.dnetlib.dhp.api.Utils; +import eu.dnetlib.dhp.api.model.CommunityEntityMap; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.Relation; diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java index 745cd7e6f..11dad9055 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java @@ -1572,36 +1572,36 @@ public class BulkTagJobTest { void newConfTest() throws Exception { final String pathMap = BulkTagJobTest.pathMap; SparkBulkTagJob - .main( - new String[] { - "-isTest", Boolean.TRUE.toString(), - "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", - getClass().getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates").getPath(), - "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", workingDir.toString() + "/dataset", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, - "-pathMap", pathMap - }); + .main( + new String[] { + "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", + getClass().getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates").getPath(), + "-taggingConf", taggingConf, + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", + "-outputPath", workingDir.toString() + "/dataset", + "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + "-pathMap", pathMap + }); final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc - .textFile(workingDir.toString() + "/dataset") - .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + .textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); Assertions.assertEquals(10, tmp.count()); org.apache.spark.sql.Dataset verificationDataset = spark - .createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); + .createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); verificationDataset.createOrReplaceTempView("dataset"); String query = "select id, MyT.id community " - + "from dataset " - + "lateral view explode(context) c as MyT " - + "lateral view explode(MyT.datainfo) d as MyD " - + "where MyD.inferenceprovenance = 'bulktagging'"; + + "from dataset " + + "lateral view explode(context) c as MyT " + + "lateral view explode(MyT.datainfo) d as MyD " + + "where MyD.inferenceprovenance = 'bulktagging'"; Assertions.assertEquals(0, spark.sql(query).count()); } diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java index 7e12bf9c3..5f0b1d7f1 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/CommunityConfigurationFactoryTest.java @@ -47,7 +47,7 @@ class CommunityConfigurationFactoryTest { sc.setVerb("not_contains"); sc.setField("contributor"); sc.setValue("DARIAH"); - sc.setSelection(resolver);//.getSelectionCriteria(sc.getVerb(), sc.getValue())); + sc.setSelection(resolver);// .getSelectionCriteria(sc.getVerb(), sc.getValue())); String metadata = "This work has been partially supported by DARIAH-EU infrastructure"; Assertions.assertFalse(sc.verifyCriteria(metadata)); } diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/QueryCommunityAPITest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/QueryCommunityAPITest.java index 1b3403535..b0043d1e8 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/QueryCommunityAPITest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/QueryCommunityAPITest.java @@ -1,14 +1,8 @@ package eu.dnetlib.dhp.bulktag; -import eu.dnetlib.dhp.api.Utils; -import eu.dnetlib.dhp.api.model.CommunityEntityMap; -import eu.dnetlib.dhp.bulktag.community.Community; -import eu.dnetlib.dhp.bulktag.community.CommunityConfiguration; +import java.util.List; -import eu.dnetlib.dhp.api.model.CommunityModel; -import eu.dnetlib.dhp.api.model.CommunitySummary; -import eu.dnetlib.dhp.api.model.DatasourceList; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -16,8 +10,13 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.api.QueryCommunityAPI; - -import java.util.List; +import eu.dnetlib.dhp.api.Utils; +import eu.dnetlib.dhp.api.model.CommunityEntityMap; +import eu.dnetlib.dhp.api.model.CommunityModel; +import eu.dnetlib.dhp.api.model.CommunitySummary; +import eu.dnetlib.dhp.api.model.DatasourceList; +import eu.dnetlib.dhp.bulktag.community.Community; +import eu.dnetlib.dhp.bulktag.community.CommunityConfiguration; /** * @author miriam.baglioni @@ -43,9 +42,12 @@ public class QueryCommunityAPITest { void community() throws Exception { String id = "dh-ch"; String body = QueryCommunityAPI.community(id); - System.out.println(new ObjectMapper().writeValueAsString(new ObjectMapper() - .readValue(body, CommunityModel.class))) - ; + System.out + .println( + new ObjectMapper() + .writeValueAsString( + new ObjectMapper() + .readValue(body, CommunityModel.class))); } @Test @@ -53,14 +55,14 @@ public class QueryCommunityAPITest { String id = "dh-ch"; String body = QueryCommunityAPI.communityDatasource(id); new ObjectMapper() - .readValue(body, DatasourceList.class) - .forEach(ds-> { - try { - System.out.println(new ObjectMapper().writeValueAsString(ds)); - } catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - }); + .readValue(body, DatasourceList.class) + .forEach(ds -> { + try { + System.out.println(new ObjectMapper().writeValueAsString(ds)); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }); ; } @@ -68,24 +70,33 @@ public class QueryCommunityAPITest { void validCommunities() throws Exception { CommunityConfiguration cc = Utils.getCommunityConfiguration(); System.out.println(cc.getCommunities().keySet()); - Community community =cc.getCommunities().get("aurora"); + Community community = cc.getCommunities().get("aurora"); Assertions.assertEquals(0, community.getSubjects().size()); Assertions.assertEquals(null, community.getConstraints()); Assertions.assertEquals(null, community.getRemoveConstraints()); Assertions.assertEquals(2, community.getZenodoCommunities().size()); - Assertions.assertTrue(community.getZenodoCommunities().stream().anyMatch(c -> c.equals("aurora-universities-network"))); - Assertions.assertTrue(community.getZenodoCommunities().stream().anyMatch(c -> c.equals("university-of-innsbruck"))); + Assertions + .assertTrue( + community.getZenodoCommunities().stream().anyMatch(c -> c.equals("aurora-universities-network"))); + Assertions + .assertTrue(community.getZenodoCommunities().stream().anyMatch(c -> c.equals("university-of-innsbruck"))); Assertions.assertEquals(35, community.getProviders().size()); - Assertions.assertEquals(35, community.getProviders().stream().filter(p->p.getSelectionConstraints()==null).count()); + Assertions + .assertEquals( + 35, community.getProviders().stream().filter(p -> p.getSelectionConstraints() == null).count()); } @Test void getCommunityProjects() throws Exception { CommunityEntityMap projectMap = Utils.getCommunityProjects(); Assertions.assertFalse(projectMap.containsKey("mes")); - Assertions.assertEquals(33, projectMap.size()); - Assertions.assertTrue(projectMap.keySet().stream().allMatch(k -> projectMap.get(k).stream().allMatch(p -> p.startsWith("40|")))); + Assertions.assertEquals(33, projectMap.size()); + Assertions + .assertTrue( + projectMap + .keySet() + .stream() + .allMatch(k -> projectMap.get(k).stream().allMatch(p -> p.startsWith("40|")))); } } - From 554551682d0501fa7664fab0a9bee8be6b0309d3 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 11 Oct 2023 16:09:19 +0200 Subject: [PATCH 03/60] [raw graph] adopting the new COAR based vocabularies for the resource typing --- .../common/vocabulary/VocabularyGroup.java | 21 ++++++++++++ .../dhp/schema/oaf/utils/OafMapperUtils.java | 10 +++++- .../raw/AbstractMdRecordToOafMapper.java | 31 ++++++++++++++++++ .../dhp/oa/graph/raw/OafToOafMapper.java | 32 ++++++++++++++++--- .../dhp/oa/graph/raw/OdfToOafMapper.java | 27 ++++++++++++++++ .../oa/graph/raw/OriginalTypeComparator.java | 32 +++++++++++++++++++ .../dnetlib/dhp/oa/graph/raw/MappersTest.java | 21 ++++++++++++ .../dnetlib/dhp/oa/graph/clean/synonyms.txt | 7 +++- .../eu/dnetlib/dhp/oa/graph/clean/terms.txt | 4 ++- .../dnetlib/dhp/oa/graph/raw/oaf_record.xml | 4 ++- pom.xml | 2 +- 11 files changed, 182 insertions(+), 9 deletions(-) create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OriginalTypeComparator.java diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/common/vocabulary/VocabularyGroup.java b/dhp-common/src/main/java/eu/dnetlib/dhp/common/vocabulary/VocabularyGroup.java index fc7175270..4c1feac45 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/common/vocabulary/VocabularyGroup.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/common/vocabulary/VocabularyGroup.java @@ -135,6 +135,27 @@ public class VocabularyGroup implements Serializable { return vocs.get(vocId.toLowerCase()).getSynonymAsQualifier(syn); } + public Qualifier lookupTermBySynonym(final String vocId, final String syn) { + if (StringUtils.isBlank(vocId)) { + return OafMapperUtils.unknown("", ""); + } + + final Vocabulary vocabulary = vocs.get(vocId.toLowerCase()); + + return Optional + .ofNullable(vocabulary.getTerm(syn)) + .map( + term -> OafMapperUtils + .qualifier(term.getId(), term.getName(), vocabulary.getId(), vocabulary.getName())) + .orElse( + Optional + .ofNullable(vocabulary.getTermBySynonym(syn)) + .map( + term -> OafMapperUtils + .qualifier(term.getId(), term.getName(), vocabulary.getId(), vocabulary.getName())) + .orElse(null)); + } + /** * getSynonymAsQualifierCaseSensitive * diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/OafMapperUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/OafMapperUtils.java index c58096d35..1c557c805 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/OafMapperUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/OafMapperUtils.java @@ -14,7 +14,6 @@ import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; import eu.dnetlib.dhp.schema.common.AccessRightComparator; -import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.*; @@ -141,6 +140,15 @@ public class OafMapperUtils { .collect(Collectors.toList()); } + public static InstanceTypeMapping instanceTypeMapping(String originalType, Qualifier term) { + final InstanceTypeMapping m = new InstanceTypeMapping(); + m.setVocabularyName(term.getSchemeid()); + m.setOriginalType(originalType); + m.setTypeCode(term.getClassid()); + m.setTypeLabel(term.getClassname()); + return m; + } + public static Qualifier unknown(final String schemeid, final String schemename) { return qualifier(UNKNOWN, "Unknown", schemeid, schemename); } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java index b37e6a755..2fed7d627 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java @@ -41,6 +41,11 @@ public abstract class AbstractMdRecordToOafMapper { protected static final String DATACITE_SCHEMA_KERNEL_4_SLASH = "http://datacite.org/schema/kernel-4/"; protected static final String DATACITE_SCHEMA_KERNEL_3 = "http://datacite.org/schema/kernel-3"; protected static final String DATACITE_SCHEMA_KERNEL_3_SLASH = "http://datacite.org/schema/kernel-3/"; + + protected static final String OPENAIRE_COAR_RESOURCE_TYPES_3_1 = "openaire::coar_resource_types_3_1"; + + public static final String OPENAIRE_USER_RESOURCE_TYPES = "openaire::user_resource_types"; + protected static final Qualifier ORCID_PID_TYPE = qualifier( ModelConstants.ORCID_PENDING, ModelConstants.ORCID_CLASSNAME, @@ -516,6 +521,32 @@ public abstract class AbstractMdRecordToOafMapper { protected abstract Field prepareDatasetStorageDate(Document doc, DataInfo info); + protected abstract String findOriginalType(Document doc); + + protected List prepareInstanceTypeMapping(Document doc) { + return Optional.ofNullable(findOriginalType(doc)) + .map(originalType -> { + final List mappings = Lists.newArrayList(); + + if (vocs.vocabularyExists(OPENAIRE_COAR_RESOURCE_TYPES_3_1)) { + + // TODO verify what the vocabs return when a synonym is not defined + Qualifier coarTerm = vocs.lookupTermBySynonym(OPENAIRE_COAR_RESOURCE_TYPES_3_1, originalType); + mappings.add(OafMapperUtils.instanceTypeMapping(originalType, coarTerm)); + + if (vocs.vocabularyExists(OPENAIRE_USER_RESOURCE_TYPES)) { + + // TODO verify what the vocabs return when a synonym is not defined + Qualifier userTerm = vocs.lookupTermBySynonym(OPENAIRE_USER_RESOURCE_TYPES, coarTerm.getClassid()); + mappings.add(OafMapperUtils.instanceTypeMapping(originalType, userTerm)); + } + } + + return mappings; + }) + .orElse(new ArrayList<>()); + } + private Journal prepareJournal(final Document doc, final DataInfo info) { final Node n = doc.selectSingleNode("//oaf:journal"); if (n != null) { diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java index a9f9367af..5cdb434f1 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java @@ -5,12 +5,10 @@ import static eu.dnetlib.dhp.schema.common.ModelConstants.*; import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.*; import java.net.URLDecoder; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; +import java.util.*; import java.util.stream.Collectors; +import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; import org.apache.commons.lang3.StringUtils; import org.dom4j.Document; import org.dom4j.Element; @@ -25,6 +23,8 @@ import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions; import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; import eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits; +import static org.apache.commons.lang3.StringUtils.contains; + public class OafToOafMapper extends AbstractMdRecordToOafMapper { public OafToOafMapper(final VocabularyGroup vocs, final boolean invisible, final boolean shouldHashId, @@ -139,6 +139,8 @@ public class OafToOafMapper extends AbstractMdRecordToOafMapper { final List alternateIdentifier = prepareResultPids(doc, info); final List pid = IdentifierFactory.getPids(alternateIdentifier, collectedfrom); + instance.setInstanceTypeMapping(prepareInstanceTypeMapping(doc)); + final Set pids = new HashSet<>(pid); instance @@ -187,6 +189,28 @@ public class OafToOafMapper extends AbstractMdRecordToOafMapper { return Lists.newArrayList(instance); } + /** + * The Dublin Core element dc:type can be repeated, but we need to base our mapping on a single value + * So this method tries to give precedence to the COAR resource type, when available. Otherwise, it looks for the + * openaire's info:eu-repo type, and as last resort picks the 1st type text available + * + * http://purl.org/coar/resource_type/c_5794 + * info:eu-repo/semantics/article + * Conference article + * + * @param doc the input document + * @return the chosen resource type + */ + @Override + protected String findOriginalType(Document doc) { + return (String) doc.selectNodes("//dc:type") + .stream() + .map(o -> "" + ((Node) o).getText().trim()) + .sorted(new OriginalTypeComparator()) + .findFirst() + .orElse(null); + } + @Override protected List> prepareSources(final Document doc, final DataInfo info) { return prepareListFields(doc, "//dc:source", info); diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java index bbd1e7ab1..c01775327 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java @@ -9,6 +9,7 @@ import java.net.URLDecoder; import java.util.*; import java.util.stream.Collectors; +import org.apache.commons.lang3.ObjectUtils; import org.apache.commons.lang3.StringUtils; import org.dom4j.Document; import org.dom4j.Element; @@ -139,6 +140,8 @@ public class OdfToOafMapper extends AbstractMdRecordToOafMapper { final List alternateIdentifier = prepareResultPids(doc, info); final List pid = IdentifierFactory.getPids(alternateIdentifier, collectedfrom); + instance.setInstanceTypeMapping(prepareInstanceTypeMapping(doc)); + final Set pids = new HashSet<>(pid); instance @@ -217,6 +220,30 @@ public class OdfToOafMapper extends AbstractMdRecordToOafMapper { } } + @Override + protected List prepareInstanceTypeMapping(Document doc) { + return null; + } + + /** + * The Datacite element + * + * journal article + * + * @param doc the input document + * @return the chosen resource type + */ + @Override + protected String findOriginalType(Document doc) { + final Element resourceType = (Element) doc.selectSingleNode( + "//metadata/*[local-name() = 'resource']/*[local-name() = 'resourceType']"); + + final String resourceTypeURI = resourceType.attributeValue("anyURI"); + final String resourceTypeTxt = resourceType.getText(); + + return ObjectUtils.firstNonNull(resourceTypeURI, resourceTypeTxt); + } + @Override protected List> prepareSources(final Document doc, final DataInfo info) { return new ArrayList<>(); // Not present in ODF ??? diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OriginalTypeComparator.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OriginalTypeComparator.java new file mode 100644 index 000000000..2eeead32e --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OriginalTypeComparator.java @@ -0,0 +1,32 @@ +package eu.dnetlib.dhp.oa.graph.raw; + +import java.util.Comparator; + +import static org.apache.commons.lang3.StringUtils.contains; +import static org.apache.commons.lang3.StringUtils.startsWith; + +public class OriginalTypeComparator implements Comparator { + + @Override + public int compare(String t1, String t2) { + + if (t1.equals(t2)) { + return 0; + } + if (startsWith(t1, "http") && contains(t1, "coar") && contains(t1, "resource_type")) { + return -1; + } + if (startsWith(t2, "http") && contains(t2, "coar") && contains(t2, "resource_type")) { + return 1; + } + if (startsWith(t1, "info:eu-repo/semantics")) { + return -1; + } + if (startsWith(t2, "info:eu-repo/semantics")) { + return 1; + } + + return t1.compareTo(t2); + } + +} diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java index b506d3a62..5df2b7a3b 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java @@ -116,6 +116,27 @@ class MappersTest { assertNotNull(instance.getPid()); assertTrue(instance.getPid().isEmpty()); + assertNotNull(instance.getInstanceTypeMapping()); + assertEquals(2, instance.getInstanceTypeMapping().size()); + + Optional coarType = instance.getInstanceTypeMapping() + .stream() + .filter(itm -> AbstractMdRecordToOafMapper.OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(itm.getVocabularyName())) + .findFirst(); + + assertTrue(coarType.isPresent()); + assertEquals("http://purl.org/coar/resource_type/c_5794", coarType.get().getTypeCode()); + assertEquals("conference paper", coarType.get().getTypeLabel()); + + Optional userType = instance.getInstanceTypeMapping() + .stream() + .filter(itm -> AbstractMdRecordToOafMapper.OPENAIRE_USER_RESOURCE_TYPES.equals(itm.getVocabularyName())) + .findFirst(); + + assertTrue(userType.isPresent()); + assertEquals("Article", userType.get().getTypeCode()); + assertEquals("Article", userType.get().getTypeLabel()); + assertFalse(instance.getAlternateIdentifier().isEmpty()); assertEquals("doi", instance.getAlternateIdentifier().get(0).getQualifier().getClassid()); assertEquals("10.3897/oneeco.2.e13718", instance.getAlternateIdentifier().get(0).getValue()); diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/synonyms.txt b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/synonyms.txt index 409dfd5dc..b3fa94b20 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/synonyms.txt +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/synonyms.txt @@ -1245,4 +1245,9 @@ dnet:relation_relClass @=@ IsRelatedTo @=@ isRelatedTo dnet:relation_subRelType @=@ relationship @=@ publicationDataset dnet:provenanceActions @=@ iis @=@ erroneous label to be cleaned FOS @=@ 0101 mathematics @=@ FOS: Mathematics -FOS @=@ 0102 computer and information sciences @=@ FOS: Computer and information sciences \ No newline at end of file +FOS @=@ 0102 computer and information sciences @=@ FOS: Computer and information sciences +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ Proceedings paper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ Conference article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ http://purl.org/eprint/type/ConferencePaper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ Conference article +openaire::user_resource_types @=@ Article @=@ http://purl.org/coar/resource_type/c_5794 \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/terms.txt b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/terms.txt index 24ce42fc2..a38c0e987 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/terms.txt +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/terms.txt @@ -1121,4 +1121,6 @@ dnet:relation_subRelType @=@ dnet:relation_subRelType @=@ supplement @=@ supplem dnet:relation_subRelType @=@ dnet:relation_subRelType @=@ version @=@ version FOS @=@ Fields of Science and Technology classification @=@ 0101 mathematics @=@ 0101 mathematics FOS @=@ Fields of Science and Technology classification @=@ 0102 computer and information sciences @=@ 0102 computer and information sciences -FOS @=@ Fields of Science and Technology classification @=@ 0103 physical sciences @=@ 0103 physical sciences \ No newline at end of file +FOS @=@ Fields of Science and Technology classification @=@ 0103 physical sciences @=@ 0103 physical sciences +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ conference paper +openaire::user_resource_types @=@ openaire::user_resource_types @=@ Article @=@ Article \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_record.xml b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_record.xml index 277578185..492fc9a7a 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_record.xml +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_record.xml @@ -47,7 +47,9 @@ provisioning services regulating services supporting services - Research Article + conference paper + http://purl.org/coar/resource_type/c_5794 + info:eu-repo/semantics/article 0001 2017-01-01 diff --git a/pom.xml b/pom.xml index 9cd82a343..0f1ec78af 100644 --- a/pom.xml +++ b/pom.xml @@ -888,7 +888,7 @@ 3.3.3 3.4.2 [2.12,3.0) - [3.17.1] + [4.17.2-SNAPSHOT] [4.0.3] [6.0.5] [3.1.6] From 89184d5b4f1af674af33fff2a46961804b8da751 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Wed, 11 Oct 2023 18:17:35 +0200 Subject: [PATCH 04/60] used the API instead of the IS for bulktagging and propagation for community through organization. Added a new propagation step for communities through projects. Still using the API and not the IS --- .../eu/dnetlib/dhp/PropagationConstant.java | 3 + .../eu/dnetlib/dhp/api/QueryCommunityAPI.java | 38 ++-- .../main/java/eu/dnetlib/dhp/api/Utils.java | 64 +++--- .../dnetlib/dhp/bulktag/SparkBulkTagJob.java | 70 ++++--- .../dhp/bulktag/community/ResultTagger.java | 28 +-- .../PrepareResultCommunitySet.java | 9 +- ...kResultToCommunityFromOrganizationJob.java | 12 +- .../PrepareResultCommunitySet.java | 121 ++++++++++++ .../ResultProjectList.java | 26 +++ .../SparkResultToCommunityFromProject.java | 156 +++++++++++++++ .../dhp/bulktag/input_bulkTag_parameters.json | 28 +-- .../dhp/bulktag/oozie_app/workflow.xml | 108 +--------- .../resources/eu/dnetlib/dhp/bulktag/query.xq | 62 ------ .../input_communitytoresult_parameters.json | 12 -- .../oozie_app/workflow.xml | 7 +- .../dnetlib/dhp/bulktag/BulkTagJobTest.java | 187 +++++++++--------- .../dhp/bulktag/QueryCommunityAPITest.java | 19 +- .../ResultToCommunityJobTest.java | 2 +- .../eosc/dataset/{dataset_10.json => dataset} | 0 .../bulktag/eosc/dataset/otherresearchproduct | 0 .../dhp/bulktag/eosc/dataset/publication | 0 .../dnetlib/dhp/bulktag/eosc/dataset/software | 0 .../bulktag/sample/dataset/no_updates/dataset | 10 + .../dataset/no_updates/dataset_10.json.gz | Bin 6736 -> 0 bytes .../sample/dataset/no_updates/datasource | 0 .../dataset/no_updates/otherresearchproduct | 0 .../sample/dataset/no_updates/publication | 0 .../sample/dataset/no_updates/software | 0 .../{dataset_10.json => dataset} | 0 .../datasource | 0 .../otherresearchproduct | 10 + .../publication | 0 .../update_datasourcewithconstraints/software | 0 .../contextnoprovenance/dataset | 10 + .../contextnoprovenance/dataset_10.json.gz | Bin 6888 -> 0 bytes .../contextnoprovenance/datasource | 0 .../contextnoprovenance/otherresearchproduct | 0 .../contextnoprovenance/publication | 0 .../contextnoprovenance/software | 0 .../dataset/update_subject/nocontext/dataset | 10 + .../nocontext/dataset_10.json.gz | Bin 6755 -> 0 bytes .../update_subject/nocontext/datasource | 0 .../nocontext/otherresearchproduct | 0 .../update_subject/nocontext/publication | 0 .../dataset/update_subject/nocontext/software | 0 .../{dataset_10.json => dataset} | 6 +- .../update_subject_datasource/datasource | 0 .../otherresearchproduct | 0 .../update_subject_datasource/publication | 0 .../update_subject_datasource/software | 0 .../update_zenodocommunity/dataset | 0 .../update_zenodocommunity/datasource | 0 .../otherresearchproduct | 10 + .../otherresearchproduct_10.json.gz | Bin 8300 -> 0 bytes .../update_zenodocommunity/publication | 0 .../update_zenodocommunity/software | 0 .../publication/update_datasource/dataset | 0 .../publication/update_datasource/datasource | 0 .../update_datasource/otherresearchproduct | 0 .../publication/update_datasource/publication | 10 + .../update_datasource/publication_10.json.gz | Bin 5257 -> 0 bytes .../publication/update_datasource/software | 0 .../dhp/bulktag/sample/software/dataset | 0 .../sample/software/otherresearchproduct | 0 .../dhp/bulktag/sample/software/publication | 0 .../dhp/bulktag/sample/software/software | 10 + .../sample/software/software_10.json.gz | Bin 6727 -> 0 bytes .../eu/dnetlib/dhp/eosctag/galaxy/publication | 0 .../dnetlib/dhp/eosctag/jupyter/publication | 0 .../dnetlib/dhp/eosctag/twitter/publication | 0 70 files changed, 622 insertions(+), 406 deletions(-) create mode 100644 dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java create mode 100644 dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/ResultProjectList.java create mode 100644 dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/SparkResultToCommunityFromProject.java delete mode 100644 dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/query.xq rename dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/eosc/dataset/{dataset_10.json => dataset} (100%) create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/eosc/dataset/otherresearchproduct create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/eosc/dataset/publication create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/eosc/dataset/software create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/dataset delete mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/dataset_10.json.gz create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/datasource create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/otherresearchproduct create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/publication create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/software rename dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/{dataset_10.json => dataset} (100%) create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/datasource create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/otherresearchproduct create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/publication create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/software create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/dataset delete mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/dataset_10.json.gz create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/datasource create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/otherresearchproduct create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/publication create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/software create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/nocontext/dataset delete mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/nocontext/dataset_10.json.gz create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/nocontext/datasource create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/nocontext/otherresearchproduct create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/nocontext/publication create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/nocontext/software rename dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject_datasource/{dataset_10.json => dataset} (93%) create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject_datasource/datasource create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject_datasource/otherresearchproduct create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject_datasource/publication create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject_datasource/software create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/otherresearchproduct/update_zenodocommunity/dataset create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/otherresearchproduct/update_zenodocommunity/datasource create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/otherresearchproduct/update_zenodocommunity/otherresearchproduct delete mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/otherresearchproduct/update_zenodocommunity/otherresearchproduct_10.json.gz create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/otherresearchproduct/update_zenodocommunity/publication create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/otherresearchproduct/update_zenodocommunity/software create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/dataset create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/datasource create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/otherresearchproduct create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/publication delete mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/publication_10.json.gz create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/software create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/software/dataset create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/software/otherresearchproduct create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/software/publication create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/software/software delete mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/software/software_10.json.gz create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/eosctag/galaxy/publication create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/eosctag/jupyter/publication create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/eosctag/twitter/publication diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/PropagationConstant.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/PropagationConstant.java index 053300696..7c4186663 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/PropagationConstant.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/PropagationConstant.java @@ -71,6 +71,9 @@ public class PropagationConstant { public static final String PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_ID = "result:community:organization"; public static final String PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME = " Propagation of result belonging to community through organization"; + public static final String PROPAGATION_RESULT_COMMUNITY_PROJECT_CLASS_ID = "result:community:project"; + public static final String PROPAGATION_RESULT_COMMUNITY_PROJECT_CLASS_NAME = " Propagation of result belonging to community through project"; + public static final String PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID = "authorpid:result"; public static final String PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME = "Propagation of authors pid to result through semantic relations"; diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java index cc615ba46..262ca0290 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java @@ -4,22 +4,18 @@ package eu.dnetlib.dhp.api; import java.io.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; -import java.io.OutputStream; import java.net.HttpURLConnection; -import java.net.MalformedURLException; import java.net.URL; -import org.apache.http.HttpHeaders; import org.jetbrains.annotations.NotNull; -import com.google.gson.Gson; - /** * @author miriam.baglioni * @Date 06/10/23 */ public class QueryCommunityAPI { - private static final String baseUrl = "https://services.openaire.eu/openaire/"; + private static final String PRODUCTION_BASE_URL = "https://services.openaire.eu/openaire/"; + private static final String BETA_BASE_URL = "https://beta.services.openaire.eu/openaire/"; private static String get(String geturl) throws IOException { URL url = new URL(geturl); @@ -36,25 +32,35 @@ public class QueryCommunityAPI { return body; } - public static String communities() throws IOException { - return get(baseUrl + "community/communities"); + public static String communities(boolean production) throws IOException { + if (production) + return get(PRODUCTION_BASE_URL + "community/communities"); + return get(BETA_BASE_URL + "community/communities"); } - public static String community(String id) throws IOException { - return get(baseUrl + "community/" + id); + public static String community(String id, boolean production) throws IOException { + if (production) + return get(PRODUCTION_BASE_URL + "community/" + id); + return get(BETA_BASE_URL + "community/" + id); } - public static String communityDatasource(String id) throws IOException { - return get(baseUrl + "community/" + id + "/contentproviders"); + public static String communityDatasource(String id, boolean production) throws IOException { + if (production) + return get(PRODUCTION_BASE_URL + "community/" + id + "/contentproviders"); + return (BETA_BASE_URL + "community/" + id + "/contentproviders"); } - public static String communityPropagationOrganization(String id) throws IOException { - return get(baseUrl + "community/" + id + "/propagationOrganizations"); + public static String communityPropagationOrganization(String id, boolean production) throws IOException { + if (production) + return get(PRODUCTION_BASE_URL + "community/" + id + "/propagationOrganizations"); + return get(BETA_BASE_URL + "community/" + id + "/propagationOrganizations"); } - public static String communityProjects(String id, String page, String size) throws IOException { - return get(baseUrl + "community/" + id + "/projects/" + page + "/" + size); + public static String communityProjects(String id, String page, String size, boolean production) throws IOException { + if (production) + return get(PRODUCTION_BASE_URL + "community/" + id + "/projects/" + page + "/" + size); + return get(BETA_BASE_URL + "community/" + id + "/projects/" + page + "/" + size); } @NotNull diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java index c1aaa14c4..a0eacb774 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java @@ -30,14 +30,14 @@ public class Utils implements Serializable { private static final ObjectMapper MAPPER = new ObjectMapper(); private static final VerbResolver resolver = VerbResolverFactory.newInstance(); - public static CommunityConfiguration getCommunityConfiguration() throws IOException { + public static CommunityConfiguration getCommunityConfiguration(boolean production) throws IOException { final Map communities = Maps.newHashMap(); List validCommunities = new ArrayList<>(); - getValidCommunities() + getValidCommunities(production) .forEach(community -> { try { CommunityModel cm = MAPPER - .readValue(QueryCommunityAPI.community(community.getId()), CommunityModel.class); + .readValue(QueryCommunityAPI.community(community.getId(), production), CommunityModel.class); validCommunities.add(getCommunity(cm)); } catch (IOException e) { throw new RuntimeException(e); @@ -46,10 +46,11 @@ public class Utils implements Serializable { validCommunities.forEach(community -> { try { DatasourceList dl = MAPPER - .readValue(QueryCommunityAPI.communityDatasource(community.getId()), DatasourceList.class); + .readValue( + QueryCommunityAPI.communityDatasource(community.getId(), production), DatasourceList.class); community.setProviders(dl.stream().map(d -> { -// if(d.getEnabled() == null || Boolean.FALSE.equals(d.getEnabled())) -// return null; + if (d.getEnabled() == null || Boolean.FALSE.equals(d.getEnabled())) + return null; Provider p = new Provider(); p.setOpenaireId("10|" + d.getOpenaireId()); p.setSelectionConstraints(d.getSelectioncriteria()); @@ -80,18 +81,20 @@ public class Utils implements Serializable { c.setSubjects(cm.getSubjects()); c.getSubjects().addAll(cm.getFos()); c.getSubjects().addAll(cm.getSdg()); - c.setConstraints(cm.getAdvancedConstraints()); - if (c.getConstraints() != null) + if (cm.getAdvancedConstraints() != null) { + c.setConstraints(cm.getAdvancedConstraints()); c.getConstraints().setSelection(resolver); - c.setRemoveConstraints(cm.getRemoveConstraints()); - if (c.getRemoveConstraints() != null) + } + if (cm.getRemoveConstraints() != null) { + c.setRemoveConstraints(cm.getRemoveConstraints()); c.getRemoveConstraints().setSelection(resolver); + } return c; } - public static List getValidCommunities() throws IOException { + public static List getValidCommunities(boolean production) throws IOException { return MAPPER - .readValue(QueryCommunityAPI.communities(), CommunitySummary.class) + .readValue(QueryCommunityAPI.communities(production), CommunitySummary.class) .stream() .filter( community -> !community.getStatus().equals("hidden") && @@ -99,17 +102,26 @@ public class Utils implements Serializable { .collect(Collectors.toList()); } - public static CommunityEntityMap getCommunityOrganization() throws IOException { + /** + * it returns for each organization the list of associated communities + */ + public static CommunityEntityMap getCommunityOrganization(boolean production) throws IOException { CommunityEntityMap organizationMap = new CommunityEntityMap(); - getValidCommunities() + getValidCommunities(production) .forEach(community -> { String id = community.getId(); try { List associatedOrgs = MAPPER - .readValue(QueryCommunityAPI.communityPropagationOrganization(id), OrganizationList.class); - if (associatedOrgs.size() > 0) { - organizationMap.put(id, associatedOrgs); - } + .readValue( + QueryCommunityAPI.communityPropagationOrganization(id, production), OrganizationList.class); + associatedOrgs.forEach(o -> { + if (!organizationMap + .keySet() + .contains( + "20|" + o)) + organizationMap.put("20|" + o, new ArrayList<>()); + organizationMap.get("20|" + o).add(community.getId()); + }); } catch (IOException e) { throw new RuntimeException(e); } @@ -117,26 +129,28 @@ public class Utils implements Serializable { return organizationMap; } - public static CommunityEntityMap getCommunityProjects() throws IOException { + public static CommunityEntityMap getCommunityProjects(boolean production) throws IOException { CommunityEntityMap projectMap = new CommunityEntityMap(); - getValidCommunities() + getValidCommunities(production) .forEach(community -> { int page = -1; int size = 100; ContentModel cm = new ContentModel(); - List projectList = new ArrayList<>(); do { page++; try { cm = MAPPER .readValue( QueryCommunityAPI - .communityProjects(community.getId(), String.valueOf(page), String.valueOf(size)), + .communityProjects( + community.getId(), String.valueOf(page), String.valueOf(size), production), ContentModel.class); if (cm.getContent().size() > 0) { - - cm.getContent().forEach(p -> projectList.add("40|" + p.getOpenaireId())); - projectMap.put(community.getId(), projectList); + cm.getContent().forEach(p -> { + if (!projectMap.keySet().contains("40|" + p.getOpenaireId())) + projectMap.put("40|" + p.getOpenaireId(), new ArrayList<>()); + projectMap.get("40|" + p.getOpenaireId()).add(community.getId()); + }); } } catch (IOException e) { throw new RuntimeException(e); diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java index 0d98e4958..b24ee129a 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java @@ -23,6 +23,8 @@ import com.google.gson.Gson; import eu.dnetlib.dhp.api.Utils; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.bulktag.community.*; +import eu.dnetlib.dhp.schema.common.EntityType; +import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.Datasource; import eu.dnetlib.dhp.schema.oaf.Result; @@ -53,50 +55,38 @@ public class SparkBulkTagJob { .orElse(Boolean.TRUE); log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - Boolean isTest = Optional - .ofNullable(parser.get("isTest")) - .map(Boolean::valueOf) - .orElse(Boolean.FALSE); - log.info("isTest: {} ", isTest); - final String inputPath = parser.get("sourcePath"); log.info("inputPath: {}", inputPath); final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); + final boolean production = Boolean.valueOf(parser.get("production")); + log.info("production: {}", production); + ProtoMap protoMappingParams = new Gson().fromJson(parser.get("pathMap"), ProtoMap.class); log.info("pathMap: {}", new Gson().toJson(protoMappingParams)); - 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 resultClazz = (Class) Class.forName(resultClassName); - SparkConf conf = new SparkConf(); CommunityConfiguration cc; - String taggingConf = parser.get("taggingConf"); + String taggingConf = Optional + .ofNullable(parser.get("taggingConf")) + .map(String::valueOf) + .orElse(null); - if (isTest) { + if (taggingConf != null) { cc = CommunityConfigurationFactory.newInstance(taggingConf); } else { - cc = Utils.getCommunityConfiguration();// QueryInformationSystem.getCommunityConfiguration(parser.get("isLookUpUrl")); + cc = Utils.getCommunityConfiguration(production); } runWithSparkSession( conf, isSparkSessionManaged, spark -> { - removeOutputDir(spark, outputPath); extendCommunityConfigurationForEOSC(spark, inputPath, cc); - execBulkTag(spark, inputPath, outputPath, protoMappingParams, resultClazz, cc); + execBulkTag(spark, inputPath, outputPath, protoMappingParams, cc); }); } @@ -141,22 +131,30 @@ public class SparkBulkTagJob { String inputPath, String outputPath, ProtoMap protoMappingParams, - Class resultClazz, CommunityConfiguration communityConfiguration) { - ResultTagger resultTagger = new ResultTagger(); - readPath(spark, inputPath, resultClazz) - .map(patchResult(), Encoders.bean(resultClazz)) - .filter(Objects::nonNull) - .map( - (MapFunction) value -> resultTagger - .enrichContextCriteria( - value, communityConfiguration, protoMappingParams), - Encoders.bean(resultClazz)) - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .json(outputPath); + ModelSupport.entityTypes + .keySet() + .parallelStream() + .filter(e -> ModelSupport.isResult(e)) + .forEach(e -> { + removeOutputDir(spark, outputPath + e.name()); + ResultTagger resultTagger = new ResultTagger(); + Class resultClazz = ModelSupport.entityTypes.get(e); + readPath(spark, inputPath + e.name(), resultClazz) + .map(patchResult(), Encoders.bean(resultClazz)) + .filter(Objects::nonNull) + .map( + (MapFunction) value -> resultTagger + .enrichContextCriteria( + value, communityConfiguration, protoMappingParams), + Encoders.bean(resultClazz)) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath + e.name()); + }); + } public static Dataset readPath( diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java index 5f62c10f4..98c46cf00 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/ResultTagger.java @@ -82,19 +82,23 @@ public class ResultTagger implements Serializable { // communities contains all the communities to be not added to the context final Set removeCommunities = new HashSet<>(); + // if (conf.getRemoveConstraintsMap().keySet().size() > 0) conf .getRemoveConstraintsMap() .keySet() - .forEach(communityId -> { - if (conf.getRemoveConstraintsMap().get(communityId).getCriteria() != null && - conf - .getRemoveConstraintsMap() - .get(communityId) - .getCriteria() - .stream() - .anyMatch(crit -> crit.verifyCriteria(param))) - removeCommunities.add(communityId); - }); + .forEach( + communityId -> { + // log.info("Remove constraints for " + communityId); + if (conf.getRemoveConstraintsMap().keySet().contains(communityId) && + conf.getRemoveConstraintsMap().get(communityId).getCriteria() != null && + conf + .getRemoveConstraintsMap() + .get(communityId) + .getCriteria() + .stream() + .anyMatch(crit -> crit.verifyCriteria(param))) + removeCommunities.add(communityId); + }); // communities contains all the communities to be added as context for the result final Set communities = new HashSet<>(); @@ -124,10 +128,10 @@ public class ResultTagger implements Serializable { if (Objects.nonNull(result.getInstance())) { for (Instance i : result.getInstance()) { if (Objects.nonNull(i.getCollectedfrom()) && Objects.nonNull(i.getCollectedfrom().getKey())) { - collfrom.add(StringUtils.substringAfter(i.getCollectedfrom().getKey(), "|")); + collfrom.add(i.getCollectedfrom().getKey()); } if (Objects.nonNull(i.getHostedby()) && Objects.nonNull(i.getHostedby().getKey())) { - hostdby.add(StringUtils.substringAfter(i.getHostedby().getKey(), "|")); + hostdby.add(i.getHostedby().getKey()); } } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java index e0670b80f..e32e94a4b 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java @@ -48,11 +48,10 @@ public class PrepareResultCommunitySet { final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); -// final CommunityEntityMap organizationMap = new Gson() -// .fromJson( -// parser.get("organizationtoresultcommunitymap"), -// CommunityEntityMap.class); - final CommunityEntityMap organizationMap = Utils.getCommunityOrganization(); + final boolean production = Boolean.valueOf(parser.get("production")); + log.info("production: {}", production); + + final CommunityEntityMap organizationMap = Utils.getCommunityOrganization(production); log.info("organizationMap: {}", new Gson().toJson(organizationMap)); SparkConf conf = new SparkConf(); diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java index 50df08f8c..43f425b68 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java @@ -56,12 +56,6 @@ public class SparkResultToCommunityFromOrganizationJob { 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); - @SuppressWarnings("unchecked") Class resultClazz = (Class) Class.forName(resultClassName); @@ -73,9 +67,9 @@ public class SparkResultToCommunityFromOrganizationJob { isSparkSessionManaged, spark -> { removeOutputDir(spark, outputPath); - if (saveGraph) { - execPropagation(spark, inputPath, outputPath, resultClazz, possibleupdatespath); - } + + execPropagation(spark, inputPath, outputPath, resultClazz, possibleupdatespath); + }); } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java new file mode 100644 index 000000000..b0fbf8056 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java @@ -0,0 +1,121 @@ + +package eu.dnetlib.dhp.resulttocommunityfromproject; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +import java.util.*; + +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.MapFunction; +import org.apache.spark.api.java.function.MapGroupsFunction; +import org.apache.spark.sql.*; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.gson.Gson; + +import eu.dnetlib.dhp.api.Utils; +import eu.dnetlib.dhp.api.model.CommunityEntityMap; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; +import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultOrganizations; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.Relation; +import scala.Tuple2; + +public class PrepareResultCommunitySet { + + private static final Logger log = LoggerFactory.getLogger(PrepareResultCommunitySet.class); + + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + PrepareResultCommunitySet.class + .getResourceAsStream( + "/eu/dnetlib/dhp/resulttocommunityfromproject/input_preparecommunitytoresult_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + + final boolean production = Boolean.valueOf(parser.get("outputPath")); + log.info("production: {}", production); + + final CommunityEntityMap projectsMap = Utils.getCommunityProjects(production); + log.info("projectsMap: {}", new Gson().toJson(projectsMap)); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + prepareInfo(spark, inputPath, outputPath, projectsMap); + }); + } + + private static void prepareInfo( + SparkSession spark, + String inputPath, + String outputPath, + CommunityEntityMap projectMap) { + + final StructType structureSchema = new StructType() + .add( + "dataInfo", new StructType() + .add("deletedbyinference", DataTypes.BooleanType) + .add("invisible", DataTypes.BooleanType)) + .add("source", DataTypes.StringType) + .add("target", DataTypes.StringType) + .add("relClass", DataTypes.StringType); + + spark + .read() + .schema(structureSchema) + .json(inputPath) + .filter( + "dataInfo.deletedbyinference != true " + + "and relClass == '" + ModelConstants.IS_PRODUCED_BY + "'") + .select( + new Column("source").as("resultId"), + new Column("target").as("projectId")) + .groupByKey((MapFunction) r -> (String) r.getAs("source"), Encoders.STRING()) + .mapGroups((MapGroupsFunction) (k, v) -> { + ResultProjectList rpl = new ResultProjectList(); + rpl.setResultId(k); + ArrayList cl = new ArrayList<>(); + cl.addAll(projectMap.get(v.next().getAs("target"))); + v.forEachRemaining(r -> { + projectMap + .get(r.getAs("target")) + .forEach(c -> { + if (!cl.contains(c)) + cl.add(c); + }); + + }); + rpl.setCommunityList(cl); + return rpl; + }, Encoders.bean(ResultProjectList.class)) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath); + } + +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/ResultProjectList.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/ResultProjectList.java new file mode 100644 index 000000000..44798a1f3 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/ResultProjectList.java @@ -0,0 +1,26 @@ + +package eu.dnetlib.dhp.resulttocommunityfromproject; + +import java.io.Serializable; +import java.util.ArrayList; + +public class ResultProjectList implements Serializable { + private String resultId; + private ArrayList communityList; + + public String getResultId() { + return resultId; + } + + public void setResultId(String resultId) { + this.resultId = resultId; + } + + public ArrayList getCommunityList() { + return communityList; + } + + public void setCommunityList(ArrayList communityList) { + this.communityList = communityList; + } +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/SparkResultToCommunityFromProject.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/SparkResultToCommunityFromProject.java new file mode 100644 index 000000000..6d4779ea1 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/SparkResultToCommunityFromProject.java @@ -0,0 +1,156 @@ + +package eu.dnetlib.dhp.resulttocommunityfromproject; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.PropagationConstant.PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +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; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; +import eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.Context; +import eu.dnetlib.dhp.schema.oaf.Result; +import scala.Tuple2; + +/** + * @author miriam.baglioni + * @Date 11/10/23 + */ +public class SparkResultToCommunityFromProject implements Serializable { + private static final Logger log = LoggerFactory.getLogger(SparkResultToCommunityFromProject.class); + + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + SparkResultToCommunityFromProject.class + .getResourceAsStream( + "/eu/dnetlib/dhp/resulttocommunityfromproject/input_communitytoresult_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + + final String possibleupdatespath = parser.get("preparedInfoPath"); + log.info("preparedInfoPath: {}", possibleupdatespath); + + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); + + @SuppressWarnings("unchecked") + Class resultClazz = (Class) Class.forName(resultClassName); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { +// removeOutputDir(spark, outputPath); + + execPropagation(spark, inputPath, outputPath, possibleupdatespath); + + }); + } + + private static void execPropagation( + SparkSession spark, + String inputPath, + String outputPath, + + String possibleUpdatesPath) { + + Dataset possibleUpdates = readPath(spark, possibleUpdatesPath, ResultProjectList.class); + + ModelSupport.entityTypes + .keySet() + .parallelStream() + .forEach(e -> { + if (ModelSupport.isResult(e)) { + removeOutputDir(spark, outputPath + e.name()); + Class resultClazz = ModelSupport.entityTypes.get(e); + Dataset result = readPath(spark, inputPath + e.name(), resultClazz); + + result + .joinWith( + possibleUpdates, + result.col("id").equalTo(possibleUpdates.col("resultId")), + "left_outer") + .map(resultCommunityFn(), Encoders.bean(resultClazz)) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath); + } + }); + + } + + private static MapFunction, R> resultCommunityFn() { + return value -> { + R ret = value._1(); + Optional rcl = Optional.ofNullable(value._2()); + if (rcl.isPresent()) { + ArrayList communitySet = rcl.get().getCommunityList(); + List contextList = ret + .getContext() + .stream() + .map(Context::getId) + .collect(Collectors.toList()); + + @SuppressWarnings("unchecked") + R res = (R) ret.getClass().newInstance(); + + res.setId(ret.getId()); + List propagatedContexts = new ArrayList<>(); + for (String cId : communitySet) { + if (!contextList.contains(cId)) { + Context newContext = new Context(); + newContext.setId(cId); + newContext + .setDataInfo( + Arrays + .asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RESULT_COMMUNITY_PROJECT_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_PROJECT_CLASS_NAME, + ModelConstants.DNET_PROVENANCE_ACTIONS))); + propagatedContexts.add(newContext); + } + } + res.setContext(propagatedContexts); + ret.mergeFrom(res); + } + return ret; + }; + } +} diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json index a8be7c32e..dbe2d088f 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json @@ -1,10 +1,5 @@ [ - { - "paramName":"is", - "paramLongName":"isLookUpUrl", - "paramDescription": "URL of the isLookUp Service", - "paramRequired": true - }, + { "paramName":"s", "paramLongName":"sourcePath", @@ -17,12 +12,7 @@ "paramDescription": "the json path associated to each selection field", "paramRequired": true }, - { - "paramName":"tn", - "paramLongName":"resultTableName", - "paramDescription": "the name of the result table we are currently working on", - "paramRequired": true - }, + { "paramName": "out", "paramLongName": "outputPath", @@ -35,17 +25,19 @@ "paramDescription": "true if the spark session is managed, false otherwise", "paramRequired": false }, - { - "paramName": "test", - "paramLongName": "isTest", - "paramDescription": "Parameter intended for testing purposes only. True if the reun is relatesd to a test and so the taggingConf parameter should be loaded", - "paramRequired": false - }, + { "paramName": "tg", "paramLongName": "taggingConf", "paramDescription": "this parameter is intended for testing purposes only. It is a possible tagging configuration obtained via the XQUERY. Intended to be removed", "paramRequired": false + }, + + { + "paramName": "p", + "paramLongName": "production", + "paramDescription": "this parameter is intended for testing purposes only. It is a possible tagging configuration obtained via the XQUERY. Intended to be removed", + "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml index b868e4c72..56eed4703 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml @@ -4,10 +4,6 @@ sourcePath the source path - - isLookUpUrl - the isLookup service endpoint - pathMap the json path associated to each selection field @@ -44,7 +40,7 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] @@ -102,16 +98,9 @@ - + - - - - - - - - + yarn-cluster cluster @@ -128,98 +117,15 @@ --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --sourcePath${sourcePath}/publication - --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication - --outputPath${outputPath}/publication + --sourcePath${sourcePath}/ + --outputPath${outputPath}/ --pathMap${pathMap} - --isLookUpUrl${isLookUpUrl} + --production${production} - + - - - yarn-cluster - cluster - bulkTagging-dataset - eu.dnetlib.dhp.bulktag.SparkBulkTagJob - dhp-enrichment-${projectVersion}.jar - - --num-executors=${sparkExecutorNumber} - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - --sourcePath${sourcePath}/dataset - --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset - --outputPath${outputPath}/dataset - --pathMap${pathMap} - --isLookUpUrl${isLookUpUrl} - - - - - - - - yarn-cluster - cluster - bulkTagging-orp - eu.dnetlib.dhp.bulktag.SparkBulkTagJob - dhp-enrichment-${projectVersion}.jar - - --num-executors=${sparkExecutorNumber} - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - --sourcePath${sourcePath}/otherresearchproduct - --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct - --outputPath${outputPath}/otherresearchproduct - --pathMap${pathMap} - --isLookUpUrl${isLookUpUrl} - - - - - - - - yarn-cluster - cluster - bulkTagging-software - eu.dnetlib.dhp.bulktag.SparkBulkTagJob - dhp-enrichment-${projectVersion}.jar - - --num-executors=${sparkExecutorNumber} - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - --sourcePath${sourcePath}/software - --resultTableNameeu.dnetlib.dhp.schema.oaf.Software - --outputPath${outputPath}/software - --pathMap${pathMap} - --isLookUpUrl${isLookUpUrl} - - - - - - diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/query.xq b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/query.xq deleted file mode 100644 index 052037efb..000000000 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/query.xq +++ /dev/null @@ -1,62 +0,0 @@ -for $x in collection('/db/DRIVER/ContextDSResources/ContextDSResourceType') -let $subj := $x//CONFIGURATION/context/param[./@name='subject']/text() -let $datasources := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::contentproviders')]/concept -let $organizations := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::resultorganizations')]/concept -let $communities := $x//CONFIGURATION/context/category[./@id=concat($x//CONFIGURATION/context/@id,'::zenodocommunities')]/concept -let $fos := $x//CONFIGURATION/context/param[./@name='fos']/text() -let $sdg := $x//CONFIGURATION/context/param[./@name='sdg']/text() -let $zenodo := $x//param[./@name='zenodoCommunity']/text() -where $x//CONFIGURATION/context[./@type='community' or ./@type='ri'] and $x//context/param[./@name = 'status']/text() != 'hidden' -return - -{ $x//CONFIGURATION/context/@id} - -{$x//CONFIGURATION/context/param[./@name='removeConstraints']/text() } - - -{$x//CONFIGURATION/context/param[./@name='advancedConstraints']/text() } - - - {for $y in tokenize($subj,',') - return - {$y}} - {for $y in tokenize($fos,',') - return - {$y}} - {for $y in tokenize($sdg,',') - return - {$y}} - - - {for $d in $datasources - where $d/param[./@name='enabled']/text()='true' - return - - - {$d//param[./@name='openaireId']/text()} - - - {$d/param[./@name='selcriteria']/text()} - - } - - -{for $zc in $zenodo -return - - -{$zc} - -} -{for $zc in $communities -return - - -{$zc/param[./@name='zenodoid']/text()} - - -{$zc/param[./@name='selcriteria']/text()} - -} - - \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json index eebc1a0ca..b6eb309a5 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json @@ -11,18 +11,6 @@ "paramDescription": "the hive metastore uris", "paramRequired": true }, - { - "paramName":"sg", - "paramLongName":"saveGraph", - "paramDescription": "true if the new version of the graph must be saved", - "paramRequired": false - }, - { - "paramName":"test", - "paramLongName":"isTest", - "paramDescription": "true if it is executing a test", - "paramRequired": false - }, { "paramName": "out", "paramLongName": "outputPath", diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml index d9805ab7b..b25822ad0 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml @@ -4,10 +4,7 @@ sourcePath the source path - - - - + outputPath the output path @@ -106,7 +103,7 @@ --sourcePath${sourcePath}/relation --outputPath${workingDir}/preparedInfo/resultCommunityList --hive_metastore_uris${hive_metastore_uris} - --organizationtoresultcommunitymap${organizationtoresultcommunitymap} + diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java index 11dad9055..7cbbcaafb 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java @@ -6,6 +6,7 @@ import static eu.dnetlib.dhp.bulktag.community.TaggingConstants.ZENODO_COMMUNITY import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.util.HashMap; import java.util.List; import org.apache.commons.io.FileUtils; @@ -98,14 +99,11 @@ public class BulkTagJobTest { SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", - getClass().getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates").getPath(), + getClass().getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/").getPath(), "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", workingDir.toString() + "/dataset", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + "-outputPath", workingDir.toString() + "/", "-pathMap", pathMap }); @@ -133,19 +131,16 @@ public class BulkTagJobTest { @Test void bulktagBySubjectNoPreviousContextTest() throws Exception { final String sourcePath = getClass() - .getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/nocontext") + .getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/nocontext/") .getPath(); final String pathMap = BulkTagJobTest.pathMap; SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", sourcePath, "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", workingDir.toString() + "/dataset", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + "-outputPath", workingDir.toString() + "/", "-pathMap", pathMap }); @@ -230,19 +225,19 @@ public class BulkTagJobTest { void bulktagBySubjectPreviousContextNoProvenanceTest() throws Exception { final String sourcePath = getClass() .getResource( - "/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance") + "/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/") .getPath(); final String pathMap = BulkTagJobTest.pathMap; SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", sourcePath, "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", workingDir.toString() + "/dataset", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + + "-outputPath", workingDir.toString() + "/", + "-pathMap", pathMap }); @@ -311,18 +306,18 @@ public class BulkTagJobTest { @Test void bulktagByDatasourceTest() throws Exception { final String sourcePath = getClass() - .getResource("/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource") + .getResource("/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/") .getPath(); SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", sourcePath, "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Publication", - "-outputPath", workingDir.toString() + "/publication", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + + "-outputPath", workingDir.toString() + "/", + "-pathMap", pathMap }); @@ -384,25 +379,25 @@ public class BulkTagJobTest { void bulktagByZenodoCommunityTest() throws Exception { final String sourcePath = getClass() .getResource( - "/eu/dnetlib/dhp/bulktag/sample/otherresearchproduct/update_zenodocommunity") + "/eu/dnetlib/dhp/bulktag/sample/otherresearchproduct/update_zenodocommunity/") .getPath(); SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", sourcePath, "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.OtherResearchProduct", - "-outputPath", workingDir.toString() + "/orp", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + + "-outputPath", workingDir.toString() + "/", + "-pathMap", pathMap }); final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); JavaRDD tmp = sc - .textFile(workingDir.toString() + "/orp") + .textFile(workingDir.toString() + "/otherresearchproduct") .map(item -> OBJECT_MAPPER.readValue(item, OtherResearchProduct.class)); Assertions.assertEquals(10, tmp.count()); @@ -505,18 +500,18 @@ public class BulkTagJobTest { @Test void bulktagBySubjectDatasourceTest() throws Exception { final String sourcePath = getClass() - .getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject_datasource") + .getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject_datasource/") .getPath(); SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", sourcePath, "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", workingDir.toString() + "/dataset", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + + "-outputPath", workingDir.toString() + "/", + "-pathMap", pathMap }); @@ -636,14 +631,14 @@ public class BulkTagJobTest { SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", - getClass().getResource("/eu/dnetlib/dhp/bulktag/sample/software/software_10.json.gz").getPath(), + getClass().getResource("/eu/dnetlib/dhp/bulktag/sample/software/").getPath(), "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Software", - "-outputPath", workingDir.toString() + "/software", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + + "-outputPath", workingDir.toString() + "/", + "-pathMap", pathMap }); @@ -732,18 +727,18 @@ public class BulkTagJobTest { final String sourcePath = getClass() .getResource( - "/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints") + "/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/") .getPath(); SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", sourcePath, "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", workingDir.toString() + "/dataset", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + + "-outputPath", workingDir.toString() + "/", + "-pathMap", pathMap }); @@ -774,19 +769,19 @@ public class BulkTagJobTest { void bulkTagOtherJupyter() throws Exception { final String sourcePath = getClass() .getResource( - "/eu/dnetlib/dhp/eosctag/jupyter/otherresearchproduct") + "/eu/dnetlib/dhp/eosctag/jupyter/") .getPath(); SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", sourcePath, "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.OtherResearchProduct", - "-outputPath", workingDir.toString() + "/otherresearchproduct", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + + "-outputPath", workingDir.toString() + "/", + "-pathMap", pathMap }); @@ -829,18 +824,18 @@ public class BulkTagJobTest { public void bulkTagDatasetJupyter() throws Exception { final String sourcePath = getClass() .getResource( - "/eu/dnetlib/dhp/eosctag/jupyter/dataset") + "/eu/dnetlib/dhp/eosctag/jupyter/") .getPath(); SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", sourcePath, "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", workingDir.toString() + "/dataset", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + + "-outputPath", workingDir.toString() + "/", + "-pathMap", pathMap }); @@ -878,18 +873,18 @@ public class BulkTagJobTest { final String sourcePath = getClass() .getResource( - "/eu/dnetlib/dhp/eosctag/jupyter/software") + "/eu/dnetlib/dhp/eosctag/jupyter/") .getPath(); SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", sourcePath, "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Software", - "-outputPath", workingDir.toString() + "/software", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + + "-outputPath", workingDir.toString() + "/", + "-pathMap", pathMap }); @@ -1096,18 +1091,18 @@ public class BulkTagJobTest { void galaxyOtherTest() throws Exception { final String sourcePath = getClass() .getResource( - "/eu/dnetlib/dhp/eosctag/galaxy/otherresearchproduct") + "/eu/dnetlib/dhp/eosctag/galaxy/") .getPath(); SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", sourcePath, "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.OtherResearchProduct", - "-outputPath", workingDir.toString() + "/otherresearchproduct", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + + "-outputPath", workingDir.toString() + "/", + "-pathMap", pathMap }); @@ -1214,18 +1209,18 @@ public class BulkTagJobTest { void galaxySoftwareTest() throws Exception { final String sourcePath = getClass() .getResource( - "/eu/dnetlib/dhp/eosctag/galaxy/software") + "/eu/dnetlib/dhp/eosctag/galaxy/") .getPath(); SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", sourcePath, "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Software", - "-outputPath", workingDir.toString() + "/software", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + + "-outputPath", workingDir.toString() + "/", + "-pathMap", pathMap }); @@ -1333,19 +1328,19 @@ public class BulkTagJobTest { void twitterDatasetTest() throws Exception { final String sourcePath = getClass() .getResource( - "/eu/dnetlib/dhp/eosctag/twitter/dataset") + "/eu/dnetlib/dhp/eosctag/twitter/") .getPath(); SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", sourcePath, "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", workingDir.toString() + "/dataset", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + + "-outputPath", workingDir.toString() + "/", + "-pathMap", pathMap }); @@ -1373,19 +1368,19 @@ public class BulkTagJobTest { void twitterOtherTest() throws Exception { final String sourcePath = getClass() .getResource( - "/eu/dnetlib/dhp/eosctag/twitter/otherresearchproduct") + "/eu/dnetlib/dhp/eosctag/twitter/") .getPath(); SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", sourcePath, "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.OtherResearchProduct", - "-outputPath", workingDir.toString() + "/otherresearchproduct", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + + "-outputPath", workingDir.toString() + "/", + "-pathMap", pathMap }); @@ -1418,19 +1413,19 @@ public class BulkTagJobTest { void twitterSoftwareTest() throws Exception { final String sourcePath = getClass() .getResource( - "/eu/dnetlib/dhp/eosctag/twitter/software") + "/eu/dnetlib/dhp/eosctag/twitter/") .getPath(); SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", sourcePath, "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Software", - "-outputPath", workingDir.toString() + "/software", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + + "-outputPath", workingDir.toString() + "/", + "-pathMap", pathMap }); @@ -1455,19 +1450,19 @@ public class BulkTagJobTest { void EoscContextTagTest() throws Exception { final String sourcePath = getClass() .getResource( - "/eu/dnetlib/dhp/bulktag/eosc/dataset/dataset_10.json") + "/eu/dnetlib/dhp/bulktag/eosc/dataset/") .getPath(); SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", sourcePath, "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", workingDir.toString() + "/dataset", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + + "-outputPath", workingDir.toString() + "/", + "-pathMap", pathMap }); @@ -1533,16 +1528,16 @@ public class BulkTagJobTest { SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", getClass() - .getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints") + .getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/") .getPath(), "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", workingDir.toString() + "/dataset", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + + "-outputPath", workingDir.toString() + "/", + "-pathMap", pathMap }); final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); @@ -1574,14 +1569,14 @@ public class BulkTagJobTest { SparkBulkTagJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", - getClass().getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates").getPath(), + getClass().getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/").getPath(), "-taggingConf", taggingConf, - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", workingDir.toString() + "/dataset", - "-isLookUpUrl", MOCK_IS_LOOK_UP_URL, + + "-outputPath", workingDir.toString() + "/", + "-production", Boolean.TRUE.toString(), "-pathMap", pathMap }); diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/QueryCommunityAPITest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/QueryCommunityAPITest.java index b0043d1e8..a0083dab8 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/QueryCommunityAPITest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/QueryCommunityAPITest.java @@ -26,7 +26,7 @@ public class QueryCommunityAPITest { @Test void communityList() throws Exception { - String body = QueryCommunityAPI.communities(); + String body = QueryCommunityAPI.communities(true); new ObjectMapper() .readValue(body, CommunitySummary.class) .forEach(p -> { @@ -41,7 +41,7 @@ public class QueryCommunityAPITest { @Test void community() throws Exception { String id = "dh-ch"; - String body = QueryCommunityAPI.community(id); + String body = QueryCommunityAPI.community(id, true); System.out .println( new ObjectMapper() @@ -53,7 +53,7 @@ public class QueryCommunityAPITest { @Test void communityDatasource() throws Exception { String id = "dh-ch"; - String body = QueryCommunityAPI.communityDatasource(id); + String body = QueryCommunityAPI.communityDatasource(id, true); new ObjectMapper() .readValue(body, DatasourceList.class) .forEach(ds -> { @@ -68,7 +68,7 @@ public class QueryCommunityAPITest { @Test void validCommunities() throws Exception { - CommunityConfiguration cc = Utils.getCommunityConfiguration(); + CommunityConfiguration cc = Utils.getCommunityConfiguration(true); System.out.println(cc.getCommunities().keySet()); Community community = cc.getCommunities().get("aurora"); Assertions.assertEquals(0, community.getSubjects().size()); @@ -84,11 +84,20 @@ public class QueryCommunityAPITest { Assertions .assertEquals( 35, community.getProviders().stream().filter(p -> p.getSelectionConstraints() == null).count()); + + } + + @Test + void eutopiaCommunityConfiguration() throws Exception { + CommunityConfiguration cc = Utils.getCommunityConfiguration(true); + System.out.println(cc.getCommunities().keySet()); + Community community = cc.getCommunities().get("eutopia"); + community.getProviders().forEach(p -> System.out.println(p.getOpenaireId())); } @Test void getCommunityProjects() throws Exception { - CommunityEntityMap projectMap = Utils.getCommunityProjects(); + CommunityEntityMap projectMap = Utils.getCommunityProjects(true); Assertions.assertFalse(projectMap.containsKey("mes")); Assertions.assertEquals(33, projectMap.size()); Assertions diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java index 4dd8b976c..da3be8b35 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java @@ -78,7 +78,7 @@ public class ResultToCommunityJobTest { .getResource("/eu/dnetlib/dhp/resulttocommunityfromorganization/sample") .getPath(), "-hive_metastore_uris", "", - "-saveGraph", "true", + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", "-outputPath", workingDir.toString() + "/dataset", "-preparedInfoPath", preparedInfoPath diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/eosc/dataset/dataset_10.json b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/eosc/dataset/dataset similarity index 100% rename from dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/eosc/dataset/dataset_10.json rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/eosc/dataset/dataset diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/eosc/dataset/otherresearchproduct b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/eosc/dataset/otherresearchproduct new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/eosc/dataset/publication b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/eosc/dataset/publication new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/eosc/dataset/software b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/eosc/dataset/software new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/dataset b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/dataset new file mode 100644 index 000000000..b3ac15d4c --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/dataset @@ -0,0 +1,10 @@ +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055868909,"id":"50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529","originalId":["od______3989::02dd5d2c222191b0b9bd4f33c8e96529"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[{"value":"10.4185/RLCS-2018-1243","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofcollection":"2020-03-03T13:05:26.091Z","dateoftransformation":"2020-03-03T13:06:53.161Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:26.091Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/562","datestamp":"2018-01-23T15:06:07Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Gallardo-Camacho, Jorge","name":"Jorge","surname":"Gallardo-Camacho","rank":1,"pid":null,"affiliation":null},{"fullname":"Trujillo Fernández, José Ramón","name":"José Ramón","surname":"Trujillo Fernández","rank":2,"pid":null,"affiliation":null},{"fullname":"Jorge Alonso, Ana","name":"Ana","surname":"Jorge Alonso","rank":3,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Comunicación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"yihadismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"viralidad","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"vídeo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"propaganda","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"terrorismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"terrorism","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"jihadism","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"virality","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"video","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"5903 Ideologías Políticas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"6310.13 Terrorismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"El individualismo como estrategia","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Individualism as an improvised strategy","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Se analiza como el Yihadismo distribuye su mensaje propaganístico en You Tube.\nUniversidad Camilo José Cela","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"MediaLab - UCJC","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/562"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055868954,"id":"50|od______3989::05d8c751462f9bb8d2b06956dfbc5c7b","originalId":["od______3989::05d8c751462f9bb8d2b06956dfbc5c7b"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:26.079Z","dateoftransformation":"2020-03-03T13:06:53.239Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:26.079Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/227","datestamp":"2017-11-15T16:29:55Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Rico Pérez, Marta","name":"Marta","surname":"Rico Pérez","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Actividad Física y del Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Eventos deportivos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Retos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"I Congreso Nuevos Retos en los Eventos Deportivos","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"I Congreso Nuevos Retos en los Eventos Deportivos, celebrado en la Universidad Camilo José Cela.\n\n\n\nDerivado del proyecto de investigación \"La situación de los eventos deportivos en España a través de su estructuras y de sus herramientas\", tiene como finalidad conocer la situación de los eventos deportivos en España. Se realizó un estudio descriptivo holístico en el que se investigaron elementos culturales, el perfil de los recursos humanos, la estructura de los comités, entre otras.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2015-06-20","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/227"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2015-06-20","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055869097,"id":"50|od______3989::0f89464c4ac4c398fe0c71433b175a62","originalId":["od______3989::0f89464c4ac4c398fe0c71433b175a62"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:26.069Z","dateoftransformation":"2020-03-03T13:06:53.54Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:26.069Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/802","datestamp":"2018-11-14T15:30:47Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Ciencias de la Actividad Física y del Deporte","name":"Ciencias La Actividad Física Y. Del Deporte","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Actividad Física y del Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Actividades deportivas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Impacto físico","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Proyectos de investigación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"2411.06 Fisiología del Ejercicio","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Investigaciones en Ciencias de la Actividad Física y del Deporte CCAFD - UCJC","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Blanca Romero, investigadora y docente de Ciencias de la Actividad Física y del Deporte en la Universidad Camilo José Cela, describe dos de las líneas de investigación que se han desarrollado en el departamento CCAFD de UCJC: el impacto físico de las diferentes actividades deportivas y los métodos de recuperación más eficaces tras el ejercicio físico.\nBlanca también destaca la importancia en el ámbito de la salud los estudios en Ciencias de la Actividad Física y del Deporte.\nCiencias de la Actividad Física y del Deporte en la Universidad Camilo José Cela","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2016-03-21","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Universidad Camilo José Cela, Ciencias de la Actividad Física y del Deporte","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/802"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2016-03-21","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055869215,"id":"50|od______3989::1729c3988199b95d1d566851af7d3c55","originalId":["od______3989::1729c3988199b95d1d566851af7d3c55"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:25.653Z","dateoftransformation":"2020-03-03T13:06:53.764Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:25.653Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/801","datestamp":"2018-11-14T15:19:38Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Ciencias de la Actividad Física y del Deporte","name":"Ciencias La Actividad Física Y. Del Deporte","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Actividad Física y del Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Prevención","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Hidratación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deportistas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Electrolitos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deportes de resistencia","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"2411.06 Fisiología del Ejercicio","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"La importancia de la hidratación en competiciones resistencia, por Juan del Coso","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Juan del Coso, investigador y profesor en el Grado en Ciencias de la Actividad Física y del Deporte UCJC, presenta el proyecto de investigación que ha realizado el departamento CCAFD de UCJC: la prevención de desequilibrios de agua y electrolitos que se dan en los deportes de resistencia para que los deportistas realicen su actividad de una manera más segura.\nMás información: http://www.ucjc.edu/2016/03/investiga...\nUniversidad Camilo José Cela, Ciencias de la Actividad Física y del Deporte","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2016-03-21","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Grado en Ciencias de la Actividad Física y del Deporte UCJC","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/801"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2016-03-21","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055869487,"id":"50|od______3989::2e3f34ce90520fae350a7e1148d7dcea","originalId":["od______3989::2e3f34ce90520fae350a7e1148d7dcea"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.912Z","dateoftransformation":"2020-03-03T13:06:54.218Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.912Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/800","datestamp":"2018-11-14T13:43:21Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Actividad Física y del Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Cafeína","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Efectos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Alto rendimiento","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Competiciones","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Beneficios","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"2411.06 Fisiología del Ejercicio","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Investigación - Efectos positivos y negativos del uso de la cafeína en el deporte","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Proyecto de investigación que se están desarrollando en la Universidad Camilo José Cela (UCJC) sobre la cafeína y sus efectos cuando se realiza deporte. https://www.ucjc.edu/2016/08/la-bbc-s...\n\nJuan Del Coso, director del Laboratorio de Fisiología del Ejercicio de la Universidad Camilo José Cela, nos describe cómo está siendo la investigación que desarrolla con deportistas de alto rendimiento y el consumo de cafeína.\n\n\"Nuestra investigación se enmarca dentro del campo de la fisiología del ejercicio y el objetivo que tenemos es investigar los efectos que tiene la cafeína en los deportistas, principalmente saber si es una ayuda ergogénica en la mayor parte de los deportes, pero también conocer qué perjuicios existen para aquellos deportistas que la utilizan para incrementar el rendimiento deportivo\", explica Del Coso.\n\nY es que los datos son muy llamativos: tres de cada cuatro deportistas de alto rendimiento utilizan la cafeína antes de la competición y muchas veces se utiliza sin saber los perjuicios que puede tener para el deportista. \"Lo que estamos investigando es cómo una sustancia, que te puede hacer rendir mejor, cómo puede afectar negativamente cuando termina una competición\".\n\nPero esta investigación no sólo puede extraer conclusiones interesantes y útiles para los deportistas profesionales, \"investigando el deporte de alto rendimiento podemos conocer los beneficios y perjuicios de la cafeína que también afectarían al deportista amateur y hacer un deporte en general más seguro\".\n\nPara el desarrollo de este proyecto de investigación están colaborando el Laboratorio de Fisiología del Ejercicio de la Universidad Camilo José Cela y la Agencia para la Protección de la Salud en el Deporte.\nPara el desarrollo de este proyecto de investigación están colaborando el Laboratorio de Fisiología del Ejercicio de la Universidad Camilo José Cela y la Agencia para la Protección de la Salud en el Deporte.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2017-10-06","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Universidad Camilo José Cela, Vicerrectorado de Innovación","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/800"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2017-10-06","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055869498,"id":"50|od______3989::2f4f3c820c450bd08dac08d07cc82dcf","originalId":["od______3989::2f4f3c820c450bd08dac08d07cc82dcf"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.806Z","dateoftransformation":"2020-03-03T13:06:54.229Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.806Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/797","datestamp":"2018-11-14T12:12:23Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Cuesta Cano, Laura","name":"Laura","surname":"Cuesta Cano","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Comunicación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Marketing digital","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Medios sociales","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Marcas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Empresas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Emprendedores","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"II Liga de Debate CICAE - UCJC","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"6114.13 Marketing","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Laura Cuesta - Liga de Debate CICAE - UCJC","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Laura Cuesta, profesora en la Universidad Camilo José Cela (UCJC) y Especialista en Marketing Digital y Medios Sociales para marcas y emprendedores. \n\nEn esta ocasión, Laura Cuesta, nos explica el significado de redes sociales y nos habla sobre el uso que le dan los usuarios.\n\nII Liga de Debate CICAE - UCJC http://ligacicae.ucjc.edu/\n\nLa Universidad Camilo José Cela (UCJC) y la Asociación de Colegios Privados e Independientes –Círculo de Calidad Educativa (CICAE) – organizan la segunda edición de esta gran liga de debate académico. El formato es al estilo inglés, similar al de la Oxford Union, donde algunos jóvenes se reunían a debatir sobre temas de la más candente actualidad y polemizaban con la intención de formarse en tan valoradas habilidades.\n\nLa Liga de Debate CICAE-UCJC es una ocasión única para los alumnos de ponerse en contacto con herramientas tan útiles como la oratoria y la comunicación efectiva. Pero, sobre ","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-10-23","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Universidad Camilo José Cela, Vicerrectorado de Innovación","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/797"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-10-23","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055870582,"id":"50|od______3989::752fd0b2bbac1ea1cc50e52fd46eb663","originalId":["od______3989::752fd0b2bbac1ea1cc50e52fd46eb663"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.561Z","dateoftransformation":"2020-03-03T13:06:55.716Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.561Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/798","datestamp":"2018-11-14T12:57:26Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Psicología","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Alcohol","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Dependencia","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Proyectos de investigación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Alcoholismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Realidad virtual","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Psicología conductual","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Investigación - Intervención basada en realidad virtual sobre la dependencia del alcohol","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Proyecto de investigación que se están desarrollando en la Universidad Camilo José Cela (UCJC) con tecnología aplicada al abandono de la dependencia del alcohol. \nRosa Jurado, profesora e investigadora de UCJC https://www.ucjc.edu/, explica cómo se conjuga el alcoholismo con la realidad virtual. \"Mi investigación consiste en desarrollar una técnica de intervención o rehabilitación que tiene como propósito fomentar la capacidad de inhibición para que las personas que tienen dependencia del alcohol sean capaces de controlar su conducta de aproximación hacia aquello que tiene que ver con el consumo de alcohol\".\n\nEn esta investigación, liderada por la Universidad Camilo José Cela, están participando, el Instituto de Investigación Biomédica del Hospital 12 de Octubre y la Universidad Politécnica de Madrid, con el CeDint. \"En nuestro equipo de investigación pensamos que, además del deseo de consumo, existen una carencia clara de capacidad de control inhibitorio en este tipo de personas\", destaca Rosa Jurado.\n\nLa realidad virtual sirve, según explica Gabriel Rubio, jefe de Psiquiatría del Hospital 12 de Octubre, \"para dar un paso más, para ver cómo reacciona un sujeto en un ambiente que no es el real, pero se parece mucho al real\". El objetivo final de estas acciones es \"mejorar la capacidad del individuo para poner en marcha mecanismos de inhibición conductual, para que el sujeto sea capaz de decir que no\".\n\nY no acaba ahí el proceso. La fase final del tratamiento se enfoca en que el paciente \"sea capaz de reconstruir su vida\" a través de programas de entrenamiento de habilidades sociales, asertividad, familias... \"Juntando todas las áreas de intervención, la abstinencia se mantiene mucho más tiempo y las recaídas disminuyen\".","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2017-10-05","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Universidad Camilo José Cela, Vicerrectorado de Innovación","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/798"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2017-10-05","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055870750,"id":"50|od______3989::7fcbe3a03280663cddebfd3cb9203177","originalId":["od______3989::7fcbe3a03280663cddebfd3cb9203177"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:25.652Z","dateoftransformation":"2020-03-03T13:06:55.95Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:25.652Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/796","datestamp":"2018-11-05T13:36:54Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Comunicación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Redes sociales","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Comunicación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Información","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Desinformación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"III Liga de Debate CICAE","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Instantaneidad","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Información periodística","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Fake news","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"5910.02 Medios de Comunicación de Masas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Jorge Gallardo - Liga de Debate CICAE","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Jorge Gallardo, profesor de Derecho en la Universidad Camilo José Cela (UCJC), Doctor en Comunicación Audiovisual y MBA en Empresas Audiovisuales. Subdirector de Espejo Público en Antena 3 Noticias. \n\nEn esta ocasión, Jorge Gallardo habla sobre las redes sociales, cómo han transformado la manera en la que nos comunicamos y nos informamos. \n\nIII Liga de Debate CICAE - UCJC http://ligacicae.ucjc.edu/\n\nLa Universidad Camilo José Cela (UCJC) y la Asociación de Colegios Privados e Independientes –Círculo de Calidad Educativa (CICAE) – organizan la tercera edición de esta gran liga de debate académico. El formato es al estilo inglés, similar al de la Oxford Union, donde algunos jóvenes se reunían a debatir sobre temas de la más candente actualidad y polemizaban con la intención de formarse en tan valoradas habilidades.\n\nLa Liga de Debate CICAE-UCJC es una ocasión única para los alumnos de ponerse en contacto con herramientas tan útiles como la oratoria y la comunicación efectiva. Pero, sobre","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-10-23","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/796"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-10-23","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055872001,"id":"50|od______3989::d791339867bec6d3eb2104deeb4e4961","originalId":["od______3989::d791339867bec6d3eb2104deeb4e4961"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.586Z","dateoftransformation":"2020-03-03T13:06:57.721Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.586Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/645","datestamp":"2018-04-12T07:59:27Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Transferencia de Conocimiento e Innovación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Innovación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Investigación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Ciencia","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Tecnología","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"UCJC Open Science Day 2018","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"El UCJC Open Science Day tiene como objetivo mostrar las actividades de investigación llevadas a cabo por investigadores de la Universidad Camilo José Cela (UCJC) a todos los miembros de la UCJC.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-04-10","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/645"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-04-10","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055872022,"id":"50|od______3989::d90d3a1f64ad264b5ebed8a35b280343","originalId":["od______3989::d90d3a1f64ad264b5ebed8a35b280343"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.912Z","dateoftransformation":"2020-03-03T13:06:57.747Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.912Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/795","datestamp":"2018-11-06T15:43:10Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Comunicación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Unión Europea","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Desinformación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Política","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Periodismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Ciudadanos europeos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Democracias","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Estado de derecho","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Derechos humanos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"5910.02 Medios de Comunicación de Masas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Ramón Luis Varcárcel - La Unión Europea ante el reto de la desinformación","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Conferencia de Ramón Luis Valcárcel, vicepresidente del Parlamento Europeo, en el Campus de Almagro de la Universidad Camilo José Cela (UCJC) bajo el título \"La Unión Europea ante el reto de la desinformación\", en la que ha alertado sobre el alcance de la desinformación en las democracias y sobre cómo pone en peligro los derechos de los ciudadanos europeos. También ha participado el rector de la UCJC, Samuel Martín-Barbero.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-10-11","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/795"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-10-11","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/dataset_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/dataset_10.json.gz deleted file mode 100644 index bd29d59ae948f260084f492e11d2439e22d96d1e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6736 zcmV-W8n5LaiwFp=uZ>;+17u-zVRL14UokK)YIARH0PS7flG`?xzMrQ+scNcbs?=(d z67_f2X*-!t*@??`rYbw0N`WLKp+f=+NOq!RUA1|JeF9(l!WUDy%dKwn6#FE54giXj z`ajbiGB$Qy8MOou2j@FD0C7GX{M1WmLC?fO_U1qJxVYdsk5bl~FKC*xK76xaO0i`3 zMZ!{6uq4{(UzmvNvV|)1qBrk(gIUkk#NXFSUNAv*$sYVr|4rk9OMwl?Da~^Zt@QFu z&a<^t#XMFr&%dYXw{ykTGUtU<8>p^}3YxLI6wc!n%b0#}B3Lo6+dk24=esiYiPw~R z{kq?Sc8hWi>S6`YSYFU<-JAR2H1xu7I-btF8FU(33cdf9$v67{hO_BxKKFto36mfW zg212okr&OPWV9F#<0+es!vM7<6<=~e(=&L$KYI79{!h?~Oj8yEFbgF!c*IY=Z`lT9 zk!vOrNfkbAKF8-lV;3|nv9piZdix9NiadhW3z`yk^wsI-r%ub8rMeYW7ch_>L2Eb5L(_t3BmpNtE2263>iH9> zvnq=9pNA)Z4nBo{JN0onDC6^Zz>*Ta23hov-<8l7qMI;LSOY~t=U3I62i_p`yn&BW z9f#wqeqbV@TCuVkNm4i8|^rY%tY znV0ICD)1v9`YkvuJDPE5Lwkz_Pr1cdkM{mv%P&=V&eK$qPniIwm>r(E4|{p)L-LP}A9>(K)^FQaqt6nh^3l!H&?G$#ooDJ7>eD+P~f%#96P zG8?|43C}a>U}zJD3$8%-2~8Xft-`Q;mjM7Gk^hJ`?v4l z&H9;4x9iRZ2~hM`rg1Fl9z7z)atf%!3$JPV-xCeO1MZkI4?v4fFZGj|FFDJ zv|vkq!}~@?xgh3=`<>eR;jjaU)+sO8H~R&e(jOFY5dWSbch*FefG?zifE-YZ=PV%> z-y7Bp4&Khet9Q~x6Jz60o1*JYG%!hi;j7>X(#l|!{h7!QU)k>`pp zS4CY&)G)yZjCiKc*-nA@H5a)ZNdQ;MWAJEuqsm5&W5s9zZVWtP29oKZu`Qh+N1G!N zABp66H1Licw)Yr(d!RLC{_@%5FAgO8at+hUiLP*k80JaeWHPxX#=2MFbq)OG;Iiih zql30T0m`glClBF zb9{+afpd$U+EK_<#ueG5cZt;W;G()&?crxNCDGiuSjH=u%^1y)RZuy`!(Zc4VfQ!H z?@IefmefRy9E1dS+nf;-cERI%e&j#yKI_$9MdL7py^4@yhpE{`vY(KRNs#iX$10uH{IE$uypX{%9O5W>GXvg2)@s!f~>Q;xL{>y<0evw^jcS zawP9Tx0}rFNw*7zv-e52GxD@ew+n(v?-Jc^a^(9*e)!TK&x6@K4Bs!^?m50rKHI)k zNOpTlRnZ`sRe#nrGgF%V_Ooh`+HPanIhxs>XlCeMF7!M6r|_B&iy5+kUs3`C6Py}z z`?76r+C7F}3yDx++x$YhxJ9dYJZFW2pj`-@U+Pr6=!~37vBY;@@|uLkfde!u;uGu=g|CBuMft_1T$fk1-?S*4#ZNx>$t*x-BXAGdBKQc7_8a;LP~)` zlnKW^TGd}jvO+GyE-nL_fHwk?H%BB+y>4AhI+yyZ+sMZPN&@n^|&y_hio`5OwoZb z1@1<5IZWX^vI|qVFvSCgDQ4bma&4GmF`bRZqj*H)Q4C)#m={m{(Qp{~laT6T;x~pV zZmZtSFva-3VTy3{KBE-lPDClD-jzuKFiJ7PqyR&2K6=kd0d9O9?UQ@&q}#{t>OI-3 z#DDu)*ZO(@Rhrf6K*-esVNL+cxpScr7aI9+XO`wZ@D3bSZ=$$k9W8UrGjP8!M=tbm zac4SO1>>3B7|k#5j5=W21SS~yJ}k0e!iTpGb23d&E052aih0c4xamr4s~$8`Fug&4 zzZC*GJw5sKQB4kafv-Q1J={I;^$Dyp0s7}MV@mdGUz zOeliqOZ(iMq}z9b(JMTE6(m|wj6~&BDR?=x@i&Iypf!@ObjZjWFFXRRcaA^v{rSCW zE#Vq`cu^zt%2gS33&Tg*_H7}plo)YZGi9a9wr^l~Eil^^RK%zfHpeQ5ST#6!C9JMN z#)xm<>P{3Cz5qQL3}cT| ze>RI|Vd5v@csvb#x|k%xI1FzIVcuH3gCWdtFb;?J3}H^j?=ObAlc)U{rhj7$)1S?U zQy0T@F-#Z3{8*-z{SXb(z+FhDbs+x669rmxNVJZG^Glrp0p4aEp`zvUzqsmFGUe{r zX(xsRv$3lj#)e*IIZzyv!ohPZlj6yKbk@9@tQw0h43ow(TeE^&vPEN%^$AoZpO-pj zosv_T-vF#5Q*_ZJ?oOuIgL$7!0k5v``~}P5qY3bTR-GNbXHXvZy}+u##_$L)x%n6| zQYKc^p2yYg-At5#_Sf}`9exjeW#5%N<&ols{y^c49%N#T`MsiMBve#Krjx5D9Rh7r zUA!)&7i&hw7;5KbO~t_4rJy9Ef1L^cX#De&DJeEV#kschjj`YwK(u40v4o2lpa9j0S-} zeP5)9TAubv4>noGm86H^d^DT~eygO1zk>&YAs?f6(rLuVnGo_~7yq+QIVe_@xpeos zV>CEM!$&-CxbSVqKscofwtXjDaH`$d?Lwcvb|AExB_f5TAM`EZI$Ng;2)65(nh`z+ zMVnB3!UUeT<<3Dl3WI~^4itv7{h;s>vGY68GUx=(8zR_}n$L(~nO8^8W4j`&kJ;b6 zN_HfME*iz%Nes_!W)tKLQxI(5G;=VByCp#Hz=rIHgBX6d&a}zW+*SHpZhJ8zbRnJ4BoJ6B^I8vP$G|On zGC~g;o5~{I{$ehj-o7C@$NfE>;Nh|+YyDxm^*t^GvMdf(f0}WD1@kh~VpYmBspWfL zl=`0Q!-fzpkx><6$3&q_;`x|@>Dj(54*FhauFx&Y=$z@C?I>~qGjFDoP&87to3s=b zJ6%d=ls|AC2Xqf)uy7IcHBB>}VI-#!Q=l+>HZnfwrc9usT7qP61#XoX3(M#RFhm9m zt5UmHRFghthL?4IjVg)#cO@VNKb@C)(LmS_nvG(pC=?`454k9spF&Ala5O(#62w(F z+6w40D5iyjk~e`U0> zn+i~{0p3?&Ohj*-%7%St=aAcBfyly8H(a{DwJH1X6-BUg&lJhnxm0~y%luB!#6b6U zqN2ubtg|SX$~Zh68))|TRZpw`85#zPb~ru+_?hi5Y1M|_-3BSZbmFwk4V{k*+?39Z zN;hQtD<}*;X*C>Ljv;wLJ#$;ZtKPmIcJt?zNUmj~hu7}*1^6887Q09xt3f%KoV6w^ z!?&7(5e=W4I1#ioRw5hCXuA|UfnY!^5niZJ)2w8rza8Y(XKC*N9b-4K&+J4I%T&8t ztAU5~y_lVcl9OeFTLyo_6jrdfzO#ao7dn5AS;5rla6ZIdxtQ?-Qqd+yzIWt}old(( zMf*2{WDZS7Mf=#XbyT#DP|>_jQ_y z)LSN}RLDNO;FhX!(|Pr0O(koT?m6DrgYw2sWme)Is7vzEdeHMJRo{YN7fX^DSLonq z6%MA8VdoM?OX`Rjt&fe^rCn*6x|iY6aEC$%P!I`Aq;gKFT`11Z$P>QQi8~&#h~mvA zXQwBRZ@t;XVe|g68TC=>ruYTL3yCncnK0bF9-jacSrbr=Wan=5bQSnm0rKqGkOn79>(Fsi@dPNcj-Rx=Q%2L9OVe!UDF# ziRmx}eQw`oc(oCEoD#EVr>Q=;J(w5RC%`$~zRmjp0rZh7!$!fTzWf0yB&Y`o@Fh}c zJ`6MuJv!h10;X%oNs5)i@K{W{Tv?C`mhHAjG+!mABGHT>7?Gvz^M|!Ex*LlL*nM1km26}xC8P`U*u(#Bop|o z1w#)5K!ajtgdPz0!0vY0^n$yWCS?wHX&^0EQQ4_mEK|CsyJ* z1((jz7V`+6R%eg0dE9EH_L~~FR=;kN4~@ZAG(_+?Och2(TN?LcxBrp|qP9_J6scW2 zF{Pl(;CsPK8;Y6Gm-W$9TiYXA>{woBzuVoou-yZ)fu=IKsg3%obM$hn0a&tb( z#a!CW@)FCHbS$E_Q-9&+rMnfSZ6K=T1Nr51mkyW?xGA1zE~`tcKs;wQwspZ3#LRV-C?m@%6oK%VYZ7Z`U@{{m}Y|slV_$esOl6oNxd@A0ZDq>a?P<` zx@a3+N`Hib^Mb{q@*7kWRlpFz?vSUDNKs{pxW@XB1@m0ylzd*|OY|N&XYbZ2THWk) zb>gHtSB=TD0-Xcl&da5y&Cq(vuc#A;;{@}*JmrUa!X!nQVaEi*D=jI_LnyXyW6g}v z>TQljOod`77ezzgCYO+AKtLy}eWx9LL8m}jLwU<<;9|uiOJ1pOnxZhq;D2Vg}ujm6SiVw2+gc`!*rwfx)<{*WWZP4aesaQ?> z+Uyh1I73E?<{sRhjLMoPk`y@RR}yF-(C(RlhZbdFw`TSb`-IDE`!+GSq~vchU+aB6 zKfuCY!lZ`oaC6xDGraw)0q9ejD4z7Io}RKt05RGKbWP~|R4`D!EBX4W(Pd~ThR@L_ zG+~-Ck(l%>Oj;mNb^ToiC!OtwTD}Ol7h~{y7Gc4&mSOU4OTg2nRkoE5rf@CW7r(j*u(J)t>SnoC1dn<5JaIy{WU zOeceq=y~v0Fhrzs?NWX%aBI0Df#t+B$8`FyS3RAf>GBXnO(&S-yp%>@O?N`UkkPlI zBx5mdaAnmyUOiyz1ysKtyvW}(eM629w)CSRp~YO;{@L;KHFg!Ci(nA0w(soCcrb>g zW+W{r^t$R%N(#Kc(XyU28EYX;jxpOP_DJjSRd26hP2mAcF1dCGn9coFk5GKx#rCZs zcqzG5)Y8*5X-d#tqOc~dY)3)_d(WA&x#9aDYKGgGn8*&A|GWfOr*m9Mq@|%Z+r9y~ z@=HFnk6_RU29+>F0y>t#tf$?OL0S(g_yY64ZS=FCLv;HNpPT?HT$CGzS+iV-@4@p3 zg-)lAIrLGY>6k+wF1C(2)Dh;;wA0KXHw8N8kYf%#By(sIde<_C7IDOe)Efp^_rr4p< zuCv1ZEcnN8{pzsj%EWJzx5RA~hH&P4123qu_vqX- z&5R_P58@ILc7_c3xl~I=KBGv*O5`<3y{-q#@fy1b^WAxk`|#J4L1O`mAvxskB*s2@ zB=u#AP-TDEpOoMZ0JGAh_WI-mi&RNTo4;Z*uNP(1xRy~T3pcS9=@?Bn_OjZfQ?3)aH zVI=zfIeI#s60Be<*6MI6!CXo(mlDjS1pCM)SeFmwk3{TqJhKnka2(I96FjqVr+H@1 zBRig%c%5W?%gMvd7~pgccjfr+)&}UrT6(4QmaLP+q8&N!;!1P=p-66`BCyPwR3e&qg zAX)|DF}xy~H;}{}S#)r;J1#T`Ia!q%x+{lHhXZ&b`u zJeaQ|){R$1Lh<;bKKoeISHs-#1;()_x@Zn}XHT3;(PY8yTTN=MrcW?TfQRh4Iq{4q zWuH-%(oEB3jUnO9DduJtnypLiB2LnbE|t6wQ6i$}5+72)I~yJ-f7Sav28(yC)MxBh zc>57kdUiMGl+ArSsJN@^o66XS-P(+7Y+ZA>9}iVysWnSEb}`jX(au=*fNp$wsXZVs z@&1(UTfEVyzD6gJrax7#0!s_&lNU5A;g_dSaQjXieL_{llyr2Fdku%X9P09Yr&C84 mX+%97UBuBvK2*90Mg_Y~7jYih(M4YW>Hh(z`8FgaPyqm7df$-% diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/datasource b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/datasource new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/otherresearchproduct b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/otherresearchproduct new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/publication b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/publication new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/software b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/software new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/dataset_10.json b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/dataset similarity index 100% rename from dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/dataset_10.json rename to dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/dataset diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/datasource b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/datasource new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/otherresearchproduct b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/otherresearchproduct new file mode 100644 index 000000000..ae580eab9 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/otherresearchproduct @@ -0,0 +1,10 @@ +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055422336,"id":"50|od______2017::0750a4d0782265873d669520f5e33c07","originalId":["od______2017::0750a4d0782265873d669520f5e33c07"],"collectedfrom":[{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null}],"pid":[],"dateofcollection":"2018-05-08T17:28:57.843Z","dateoftransformation":"2019-11-30T04:19:48.897Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2018-05-08T17:28:57.843Z","altered":true,"baseURL":"http://repositorio.ipv.pt/oai/openaire","identifier":"oai:repositorio.ipv.pt:10400.19/4572","datestamp":"2017-05-05T02:01:39Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Godinho, Maria do Céu","name":"Maria Do Céu","surname":"Godinho","rank":1,"pid":null,"affiliation":null},{"fullname":"Costa, Cristina Amaro","name":"Cristina Amaro","surname":"Costa","rank":2,"pid":null,"affiliation":null}],"resulttype":{"classid":"other","classname":"other","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"por","classname":"Portuguese","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Pesticidas","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Proteção integrada","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"O uso dos pesticidas em Portugal. Que caminhos?","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"A agricultura está inevitavelmente associada a um conjunto de variáveis no ecossistema, onde estão incluídos as pragas e doenças nas culturas e presença de outras plantas que competem pelo alimento, água e luz. Estes agentes são responsáveis por perdas no rendimento que, normalmente, não são aceites pelo agricultor. E é, por isso, que se recorre aos pesticidas: é preciso proteger as culturas do ataque dos seus inimigos.\nTambém é verdade que a agricultura é a atividade desenvolvida pelo Homem que mais se aproxima dos sistemas naturais e tem, assim, um papel central na produção de diversos serviços do ecossistema, como a biodiversidade agrícola, qualidade do solo e água, estabilidade climática e manutenção da paisagem. Ora, o uso dos pesticidas introduz efeitos negativos em todo este sistema: degradam o solo, reduzem a qualidade e disponibilidade de água, causam poluição e perda de habitats selvagens e de biodiversidade. Aqui reside a encruzilhada na proteção das culturas.\ninfo:eu-repo/semantics/publishedVersion","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2016-07-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":{"value":"Quercus","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[{"id":"http://zenodo.org/communities/chicago-covid-19"}],"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0020","classname":"Other ORP type","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"url":["http://hdl.handle.net/10400.19/4572"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"dateofacceptance":{"value":"2016-07-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"contactperson":[],"contactgroup":[],"tool":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055423621,"id":"50|od______2017::1bd97baef19dbd2db3203b112bb83bc5","originalId":["od______2017::1bd97baef19dbd2db3203b112bb83bc5"],"collectedfrom":[{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null}],"pid":[],"dateofcollection":"2019-02-25T04:30:56.994Z","dateoftransformation":"2019-11-30T04:19:58.463Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2019-02-25T04:30:56.994Z","altered":true,"baseURL":"http://repositorio.ipv.pt/oai/openaire","identifier":"oai:repositorio.ipv.pt:10400.19/5415","datestamp":"2019-02-23T03:01:23Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Brandão, Daniela","name":"Daniela","surname":"Brandão","rank":1,"pid":null,"affiliation":null},{"fullname":"Duarte, Natália","name":"Natália","surname":"Duarte","rank":2,"pid":null,"affiliation":null},{"fullname":"Araújo, Lia","name":"Lia","surname":"Araújo","rank":3,"pid":null,"affiliation":null},{"fullname":"Alves, Sara","name":"Sara","surname":"Alves","rank":4,"pid":null,"affiliation":null},{"fullname":"Teixeira, Laetitia","name":"Laetitia","surname":"Teixeira","rank":5,"pid":null,"affiliation":null},{"fullname":"Ribeiro, Oscar","name":"Oscar","surname":"Ribeiro","rank":6,"pid":null,"affiliation":null}],"resulttype":{"classid":"other","classname":"other","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Quality of life","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Centenarians","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Centenarians´ quality of life and the role of health and living context","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Introduction: The interest of the scienti􏰀c community to study the centenarian population has been growing due to the signi􏰀cant expansion of life expectancy and human longevity. Evidences about their quality of life (QoL) are, nevertheless, still limited. Different conceptualizations of QoL, the overvaluation of the negative aspects of advanced age and the heterogeneity of the oldest old are some issues that could explain the dif􏰀culties in this assessment. Objectives: This study aims to analyze the perceived QoL in a sample of centenarians and analyze if there are significant life context (community vs institutional) and health (objective and subjective perception) differences.\nMethods: A subsample of 59 Portuguese centenarians who participated in the PT100_Oporto Centenarian Study, conducted in 2013 was considered. Descriptive and chi-square analysis were conducted.\nResults: Most centenarians perceived their QoL in a very positive manner (50.8% rated it as “good”, 6.8% as “very good” and 3.4% as “excellent”); 28.8% perceived it as “acceptable” and 10.2% as “bad”. No significant statistical differences were found for the considered variables.\nConclusions: Centenarians seem to present high levels of perceived QoL, which can be associated with a great ability to adapt to adverse situations that occur through their lives. Neither contextual and health aspects seem to in􏰂uence the perception of QoL in this particular sample study. Nonetheless, more research is \nneeded to understand the speci􏰀cities of the QoL in the centenarian population, namely the role of psychological and social variables.\ninfo:eu-repo/semantics/publishedVersion","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2014-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[{"id":"http://zenodo.org/communities/edenis"},{"id":"http://zenodo.org/communities/aginfra"}, {"id":"http://zenodo.org/communities/primefish"}],"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0020","classname":"Other ORP type","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"url":["http://hdl.handle.net/10400.19/5415"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"dateofacceptance":{"value":"2014-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"contactperson":[],"contactgroup":[],"tool":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055423836,"id":"50|od______2017::1e400f1747487fd15998735c41a55c72","originalId":["od______2017::1e400f1747487fd15998735c41a55c72"],"collectedfrom":[{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null}],"pid":[],"dateofcollection":"2019-02-25T04:30:56.996Z","dateoftransformation":"2019-11-30T04:19:59.556Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2019-02-25T04:30:56.996Z","altered":true,"baseURL":"http://repositorio.ipv.pt/oai/openaire","identifier":"oai:repositorio.ipv.pt:10400.19/5422","datestamp":"2019-02-23T03:01:29Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Brandão, Daniela","name":"Daniela","surname":"Brandão","rank":1,"pid":null,"affiliation":null},{"fullname":"Ribeiro, Oscar","name":"Oscar","surname":"Ribeiro","rank":2,"pid":null,"affiliation":null},{"fullname":"Araújo, Lia","name":"Lia","surname":"Araújo","rank":3,"pid":null,"affiliation":null},{"fullname":"Paul, Constança","name":"Constança","surname":"Paul","rank":4,"pid":null,"affiliation":null}],"resulttype":{"classid":"other","classname":"other","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Oldest old","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Census","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"What about the Portuguese oldest old? A global overview using census data","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Background\nThe older population is itself aging, and achieving an advanced age is becoming more common worldwide. In Portugal, individuals aged 80+ represent 5.6 % of the total population and 26.5 % of the popu- lation were aged 65 and over in 2011. Having a national profile on this population will give important information to develop interven- tion programs and identify the areas requiring most attention. Ob- jective: This study aims to provide a profile of the Portuguese oldest old, as given by the last national census data.\nMethods\nThe characteristics of all residents aged 80+ (N = 532,219) were ana- lysed considering socio-demographic information (gender, marital status, education, type of residence, place of birth, income) and the existence of difficulties in functional, sensorial and cognitive activities due to health problems or ageing.\nResults\nThe majority of the most aged are females (64.5 %), widowed (53.9 %), and present low educational levels (46.1 % never attended school and 31.6 % do not know how to read/write). Own pensions constitute the main source of income (96.3 %) and the majority live in private households (88.8 %), with 43.2 % currently living in the place where they were born. The majority (73.0 %) reported major difficulties in at least one functional activity – bathing/dressing, walk- ing/climbing stairs, seeing, hearing, memory/concentration, under- standing others/being understood.\nConclusions\nThe high percentage of oldest old living in private households and the presence of functional limitations point to the importance of in- formal care and community care services to support this population. Further studies paying attention to their needs and utilisation of ser- vices are required.\ninfo:eu-repo/semantics/publishedVersion","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2016-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[{"dataInfo":[{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"propagation:community:productsthroughsemrel","classname":" Propagation of result belonging to community through semantic relation","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"}],"id":"ee"},{"dataInfo":[{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"propagation:community:productsthroughsemrel","classname":" Propagation of result belonging to community through semantic relation","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"}],"id":"fam"},{"dataInfo":[{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"propagation:community:productsthroughsemrel","classname":" Propagation of result belonging to community through semantic relation","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"}],"id":"covid-19"},{"id":"http://zenodo.org/communities/c2smart"}],"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0020","classname":"Other ORP type","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"url":["http://hdl.handle.net/10400.19/5422"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"dateofacceptance":{"value":"2016-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"contactperson":[],"contactgroup":[],"tool":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055424072,"id":"50|od______2017::210281c5bc1c739a11ccceeeca806396","originalId":["od______2017::210281c5bc1c739a11ccceeeca806396"],"collectedfrom":[{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null}],"pid":[],"dateofcollection":"2018-11-29T04:22:08.265Z","dateoftransformation":"2019-11-30T04:20:00.603Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2018-11-29T04:22:08.265Z","altered":true,"baseURL":"http://repositorio.ipv.pt/oai/openaire","identifier":"oai:repositorio.ipv.pt:10400.19/1089","datestamp":"2018-11-26T15:31:28Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Avelar-Rosa, Bruno","name":"Bruno","surname":"Avelar-Rosa","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"other","classname":"other","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"por","classname":"Portuguese","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Treino de Jovens","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Iniciação Desportiva","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Metodologia","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Didática","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Treino de Jovens: Princípios Orientadores","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Documento que recolhe os traços gerais da intervenção do treinador de jovens, considerando as possíveis orientações pedagógicas, os eixos de evolução do praticante, os diferentes modelos didáticos de iniciação, os momentos idóneos para a intervenção nas diferentes qualidades motoras e a relação entre o treinador de jovens e os restantes agentes implicados no processo (destacando-se, naturalmente, os pais).","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2010-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":{"value":"Confederação do Desporto de Portugal","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[{"dataInfo":[{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"propagation:community:productsthroughsemrel","classname":" Propagation of result belonging to community through semantic relation","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"}],"id":"beopen"},{"dataInfo":[{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"propagation:community:productsthroughsemrel","classname":" Propagation of result belonging to community through semantic relation","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"}],"id":"fam"},{"dataInfo":[{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"propagation:community:productsthroughsemrel","classname":" Propagation of result belonging to community through semantic relation","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"}],"id":"covid-19"},{"id":"http://zenodo.org/communities/primefish"}, {"id":"http://zenodo.org/communities/c2smart"}] ,"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0020","classname":"Other ORP type","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"url":["http://hdl.handle.net/10400.19/1089"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"dateofacceptance":{"value":"2010-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"contactperson":[],"contactgroup":[],"tool":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055424163,"id":"50|od______2017::225a8382b6a3db76fba97c2dc146f937","originalId":["od______2017::225a8382b6a3db76fba97c2dc146f937"],"collectedfrom":[{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null}],"pid":[],"dateofcollection":"2018-05-08T17:26:56.251Z","dateoftransformation":"2019-11-30T04:20:01.127Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2018-05-08T17:26:56.251Z","altered":true,"baseURL":"http://repositorio.ipv.pt/oai/openaire","identifier":"oai:repositorio.ipv.pt:10400.19/3290","datestamp":"2016-07-14T02:01:33Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Millenium","name":null,"surname":null,"rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"other","classname":"other","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"por","classname":"Portuguese","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[],"title":[{"value":"Referências bibliográficas ao estilo APA","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[],"dateofacceptance":{"value":"2016-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[{"id":"primefish"}],"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0020","classname":"Other ORP type","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"url":["http://hdl.handle.net/10400.19/3290"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"dateofacceptance":{"value":"2016-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"contactperson":[],"contactgroup":[],"tool":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055424299,"id":"50|od______2017::241d59362b075000e89d276e7bb6badc","originalId":["od______2017::241d59362b075000e89d276e7bb6badc"],"collectedfrom":[{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null}],"pid":[],"dateofcollection":"2019-09-06T04:39:29.25Z","dateoftransformation":"2019-11-30T04:20:01.631Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2019-09-06T04:39:29.25Z","altered":true,"baseURL":"http://repositorio.ipv.pt/oai/openaire","identifier":"oai:repositorio.ipv.pt:10400.19/1190","datestamp":"2019-09-04T02:00:41Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Menezes, Luís","name":"Luís","surname":"Menezes","rank":1,"pid":null,"affiliation":null},{"fullname":"Ribeiro, António","name":"António","surname":"Ribeiro","rank":2,"pid":null,"affiliation":null},{"fullname":"Martins, Ana Patrícia","name":"Ana Patrícia","surname":"Martins","rank":3,"pid":null,"affiliation":null},{"fullname":"Rodrigues, Cátia","name":"Cátia","surname":"Rodrigues","rank":4,"pid":null,"affiliation":null},{"fullname":"Gomes, Helena","name":"Helena","surname":"Gomes","rank":5,"pid":null,"affiliation":null},{"fullname":"Novo, Sónia","name":"Sónia","surname":"Novo","rank":6,"pid":null,"affiliation":null}],"resulttype":{"classid":"other","classname":"other","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"por","classname":"Portuguese","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Matemática","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Capacidades tranversais","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Penso, comunico, resolvo","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Esta brochura destina-se a alunos do ensino básico (1.º a 9.º anos), tendo como objetivo fornecer informação sobre as capacidades transversais que o novo Programa de Matemática português (2007) veio destacar.\nEscola Superior de Educação de Viseu","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2010-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":{"value":"ESEV","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[{"id":"http://zenodo.org/communities/fake"}],"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0020","classname":"Other ORP type","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"url":["http://hdl.handle.net/10400.19/1190"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"dateofacceptance":{"value":"2010-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"contactperson":[],"contactgroup":[],"tool":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055424626,"id":"50|od______2017::28fe553056801d4a764edac6bec334ec","originalId":["od______2017::28fe553056801d4a764edac6bec334ec"],"collectedfrom":[{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null}],"pid":[],"dateofcollection":"2019-02-25T04:30:56.998Z","dateoftransformation":"2019-11-30T04:20:03.799Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2019-02-25T04:30:56.998Z","altered":true,"baseURL":"http://repositorio.ipv.pt/oai/openaire","identifier":"oai:repositorio.ipv.pt:10400.19/5432","datestamp":"2019-02-23T03:01:24Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"da Costa Teixeira, Laetitia","name":"Laetitia","surname":"Da Costa Teixeira","rank":1,"pid":null,"affiliation":null},{"fullname":"Araújo, Lia","name":"Lia","surname":"Araújo","rank":2,"pid":null,"affiliation":null},{"fullname":"Ribeiro, Oscar","name":"Oscar","surname":"Ribeiro","rank":3,"pid":null,"affiliation":null},{"fullname":"Jopp, Daniela","name":"Daniela","surname":"Jopp","rank":4,"pid":null,"affiliation":null},{"fullname":"Paul, Constança","name":"Constança","surname":"Paul","rank":5,"pid":null,"affiliation":null}],"resulttype":{"classid":"other","classname":"other","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Europe","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Centenarians","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Centenarians in Europe","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"In the last decade, the number of centenarians world- wide has increased. Following this tendency, the number of centenarian studies has also exponentially augmented. Given the complexity to conduct research with centenarians, elementary information on this specific age group remains unknown at a European level. This paper compares basic characteristics of centenarians from 32 European coun- tries based on Census 2011. Results revealed that France is the country with higher ratio of centenarians, followed by Italy and Greece; on the other hand Croatia, Romania and Bulgaria present the lower ratio. The distribution accord- ing to gender is similar for all countries, with higher ratio of women reaching 100 years old. Different patterns of education level were found on countries where this infor- mation was reliable: Portugal and Greece are the countries with lower levels of education, with more than half without formal education; Finland, UK and Iceland are the countries with higher levels of education, with all centenarians having at least lower secondary education. Analysing the residence situation (living in the community or living in the institu- tion), Romania, Bulgaria, Lithuania, Greece and Latvia are the countries that present higher percentages of centenarians living in the community; Iceland is the country with lower percentage of centenarians living in the community. A posi- tive correlation between the number of centenarians and the percentage of Total Health Expenditure of GDP is observed. This study provides important information about the current profile of European centenarians, which is compared to gen- eral information on the centenarian population from EUA and Asia.\ninfo:eu-repo/semantics/publishedVersion","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2017-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0020","classname":"Other ORP type","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"url":["http://hdl.handle.net/10400.19/5432"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"dateofacceptance":{"value":"2017-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"contactperson":[],"contactgroup":[],"tool":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055424730,"id":"50|od______2017::2a777b5c11f070ece546d75162047179","originalId":["od______2017::2a777b5c11f070ece546d75162047179"],"collectedfrom":[{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null}],"pid":[],"dateofcollection":"2019-02-25T04:30:56.998Z","dateoftransformation":"2019-11-30T04:20:04.285Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2019-02-25T04:30:56.998Z","altered":true,"baseURL":"http://repositorio.ipv.pt/oai/openaire","identifier":"oai:repositorio.ipv.pt:10400.19/5428","datestamp":"2019-02-23T03:01:30Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"da Costa Teixeira, Laetitia","name":"Laetitia","surname":"Da Costa Teixeira","rank":1,"pid":null,"affiliation":null},{"fullname":"Araújo, Lia","name":"Lia","surname":"Araújo","rank":2,"pid":null,"affiliation":null},{"fullname":"Paul, Constança","name":"Constança","surname":"Paul","rank":3,"pid":null,"affiliation":null},{"fullname":"Ribeiro, Oscar","name":"Oscar","surname":"Ribeiro","rank":4,"pid":null,"affiliation":null}],"resulttype":{"classid":"other","classname":"other","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Survival","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"100 years old","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Survival after 100 years of age in Portuguese centenarians","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"The group of individuals aged 80 and over is growing faster than other segment of the population, and within this group the number of centenarians has risen exponentially worldwide. This phenomena lead to an increasing number of centenarian studies, improving the knowledge about this population, validating the idea of heterogeneity that char- acterizes this group. However, the factors influencing their aging survival remains understudied. The present study aims to identify factors associated with survival after 100 years old in a sample of Portuguese centenarians. After being included in a population-based study on centenarians (PT100–Oporto Centenarian Study), each participant was assessed by a tel- ephone interview every 6 months in order to assess his/her health condition. Follow-up was considered as the time (in months) between 100th birthday and death or the last tel- ephone contact. Survival analysis was performed to iden- tify factors associated with survival after 100 years old. Sociological and medical factors were considered. 140 cen- tenarians were assessed and 115 (82.1%) died. The median survival time after 100 years old was 38 months. Survival after age 100 was associated with acute disease (last month), self-perception of physical condition and drinking status. This study intends do identify factors that predict longevity in long-lived individuals. Findings recognize the importance of factors related with health status, self-perception of physi- cal condition and life style (no alcohol consume) for survival. Further studies should explore the predictive value of these factors, as well as psychosocial aspects, considering living longer but also the quality of time beyond one hundred.\ninfo:eu-repo/semantics/publishedVersion","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0020","classname":"Other ORP type","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"url":["http://hdl.handle.net/10400.19/5428"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"dateofacceptance":{"value":"2018-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"contactperson":[],"contactgroup":[],"tool":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055426968,"id":"50|od______2017::48eb86b38e6e2cfb94bf0a4663848c10","originalId":["od______2017::48eb86b38e6e2cfb94bf0a4663848c10"],"collectedfrom":[{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null}],"pid":[],"dateofcollection":"2019-02-25T04:30:56.995Z","dateoftransformation":"2019-11-30T04:20:13.611Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2019-02-25T04:30:56.995Z","altered":true,"baseURL":"http://repositorio.ipv.pt/oai/openaire","identifier":"oai:repositorio.ipv.pt:10400.19/5416","datestamp":"2019-02-23T03:01:21Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Teixeira, Laetitia","name":"Laetitia","surname":"Teixeira","rank":1,"pid":null,"affiliation":null},{"fullname":"Araújo, Lia","name":"Lia","surname":"Araújo","rank":2,"pid":null,"affiliation":null},{"fullname":"Duarte, Natália","name":"Natália","surname":"Duarte","rank":3,"pid":null,"affiliation":null},{"fullname":"Brandão, Daniela","name":"Daniela","surname":"Brandão","rank":4,"pid":null,"affiliation":null},{"fullname":"Azevedo, Maria João","name":"Maria João","surname":"Azevedo","rank":5,"pid":null,"affiliation":null},{"fullname":"Ribeiro, Oscar","name":"Oscar","surname":"Ribeiro","rank":6,"pid":null,"affiliation":null}],"resulttype":{"classid":"other","classname":"other","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Cognition","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Survival","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Functional capacity","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Cognition and functionality: how do they related to time lived after 100 years of age?","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Introduction: The number of centenarians has greatly increased in Portugal in the last decades. Therefore, the analysis of survival time after becoming a centenarian and related factors constitute an import- ant issue for the quality of care provision. The objective of this study is to identify health related factors associated with the time lived after being 100 years of age. Methods: Data come from the population-based study PT100 (Oporto Centenarian Study) and considers information gathered through face-to-face interviews with centenarians and their proxies during 2013. Study eligibility criteria included being 100 and more years old and living in the Oporto Metropolitan Area. Survival analyses were performed in order to identify factors associated with survival after the 100. Functional status (e.g., walking, being bedrid- den) and specific health conditions (e.g. presence/absence of cognitive impairment) were considered as potential factors. Results: The sam- ple comprises 140 centenarians with approximately 14.5 months as median survival time after their 100th anniversary. Centenarians who were bedridden presented a higher probability to live after 100 years when compared with functionally independent centenarians. Presence of cognitive impairment was not associated with the time lived after 100 years old. Conclusions: Given the increasing number of centenarians in Portugal, different studies based on this population need to be consid- ered. The present study, focused on the life after 100 years old, provides new information about this topic contributing to the formulation of new scientific questions for this population.\ninfo:eu-repo/semantics/publishedVersion","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2014-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0020","classname":"Other ORP type","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"url":["http://hdl.handle.net/10400.19/5416"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"dateofacceptance":{"value":"2014-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"contactperson":[],"contactgroup":[],"tool":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055427049,"id":"50|od______2017::4a1bcdd965b879fbac86a7d3b947a6cc","originalId":["od______2017::4a1bcdd965b879fbac86a7d3b947a6cc"],"collectedfrom":[{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null}],"pid":[],"dateofcollection":"2018-05-08T17:24:51.637Z","dateoftransformation":"2019-11-30T04:20:13.727Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2018-05-08T17:24:51.637Z","altered":true,"baseURL":"http://repositorio.ipv.pt/oai/openaire","identifier":"oai:repositorio.ipv.pt:10400.19/995","datestamp":"2014-07-15T01:00:31Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Oliveira, Ana Maria","name":"Ana Maria","surname":"Oliveira","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"other","classname":"other","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"por","classname":"Portuguese","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Cognição e Linguagem","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Processamento da Linguagem","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:result_subject","schemename":"dnet:result_subject"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Processar a Linguagem através dos Sentidos: entre a doçura e a amargura","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[],"dateofacceptance":{"value":"2010-07-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":{"value":"Instituto de Literatura Comparada Margarida Losa – Faculdade de Letras da Universidade do Porto","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0010","classname":"Lecture","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"url":["http://hdl.handle.net/10400.19/995"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::2017","value":"Repositório Científico do Instituto Politécnico de Viseu","dataInfo":null},"dateofacceptance":{"value":"2010-07-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"contactperson":[],"contactgroup":[],"tool":[]} diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/publication b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/publication new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/software b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_datasourcewithconstraints/software new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/dataset b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/dataset new file mode 100644 index 000000000..d03739195 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/dataset @@ -0,0 +1,10 @@ +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055868909,"id":"50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529","originalId":["od______3989::02dd5d2c222191b0b9bd4f33c8e96529"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[{"value":"10.4185/RLCS-2018-1243","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofcollection":"2020-03-03T13:05:26.091Z","dateoftransformation":"2020-03-03T13:06:53.161Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:26.091Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/562","datestamp":"2018-01-23T15:06:07Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Gallardo-Camacho, Jorge","name":"Jorge","surname":"Gallardo-Camacho","rank":1,"pid":null,"affiliation":null},{"fullname":"Trujillo Fernández, José Ramón","name":"José Ramón","surname":"Trujillo Fernández","rank":2,"pid":null,"affiliation":null},{"fullname":"Jorge Alonso, Ana","name":"Ana","surname":"Jorge Alonso","rank":3,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"COVID-19","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"yihadismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"viralidad","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"vídeo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"propaganda","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"terrorismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"terrorism","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"jihadism","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"virality","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"video","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"5903 Ideologías Políticas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"6310.13 Terrorismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"El individualismo como estrategia","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Individualism as an improvised strategy","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Se analiza como el Yihadismo distribuye su mensaje propaganístico en You Tube.\nUniversidad Camilo José Cela","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"MediaLab - UCJC","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[{"dataInfo":[{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"propagation:community:productsthroughsemrel","classname":" Propagation of result belonging to community through semantic relation","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"}],"id":"ee"},{"dataInfo":[{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"propagation:community:productsthroughsemrel","classname":" Propagation of result belonging to community through semantic relation","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"}],"id":"fam"},{"dataInfo":[{"deletedbyinference":false,"inferenceprovenance":"propagation","inferred":true,"invisible":false,"provenanceaction":{"classid":"propagation:community:productsthroughsemrel","classname":" Propagation of result belonging to community through semantic relation","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"}],"id":"covid-19"}],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/562"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055868954,"id":"50|od______3989::05d8c751462f9bb8d2b06956dfbc5c7b","originalId":["od______3989::05d8c751462f9bb8d2b06956dfbc5c7b"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:26.079Z","dateoftransformation":"2020-03-03T13:06:53.239Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:26.079Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/227","datestamp":"2017-11-15T16:29:55Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Rico Pérez, Marta","name":"Marta","surname":"Rico Pérez","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"coronavirus disease-19","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"agriculture","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Retos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"I Congreso Nuevos Retos en los Eventos Deportivos","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"I Congreso Nuevos Retos en los Eventos Deportivos, celebrado en la Universidad Camilo José Cela.\n\n\n\nDerivado del proyecto de investigación \"La situación de los eventos deportivos en España a través de su estructuras y de sus herramientas\", tiene como finalidad conocer la situación de los eventos deportivos en España. Se realizó un estudio descriptivo holístico en el que se investigaron elementos culturales, el perfil de los recursos humanos, la estructura de los comités, entre otras.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2015-06-20","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/227"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2015-06-20","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055869097,"id":"50|od______3989::0f89464c4ac4c398fe0c71433b175a62","originalId":["od______3989::0f89464c4ac4c398fe0c71433b175a62"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:26.069Z","dateoftransformation":"2020-03-03T13:06:53.54Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:26.069Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/802","datestamp":"2018-11-14T15:30:47Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Ciencias de la Actividad Física y del Deporte","name":"Ciencias La Actividad Física Y. Del Deporte","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"fishery","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"sea","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Impacto físico","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Proyectos de investigación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"2411.06 Fisiología del Ejercicio","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Investigaciones en Ciencias de la Actividad Física y del Deporte CCAFD - UCJC","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Blanca Romero, investigadora y docente de Ciencias de la Actividad Física y del Deporte en la Universidad Camilo José Cela, describe dos de las líneas de investigación que se han desarrollado en el departamento CCAFD de UCJC: el impacto físico de las diferentes actividades deportivas y los métodos de recuperación más eficaces tras el ejercicio físico.\nBlanca también destaca la importancia en el ámbito de la salud los estudios en Ciencias de la Actividad Física y del Deporte.\nCiencias de la Actividad Física y del Deporte en la Universidad Camilo José Cela","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2016-03-21","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Universidad Camilo José Cela, Ciencias de la Actividad Física y del Deporte","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/802"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2016-03-21","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055869215,"id":"50|od______3989::1729c3988199b95d1d566851af7d3c55","originalId":["od______3989::1729c3988199b95d1d566851af7d3c55"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:25.653Z","dateoftransformation":"2020-03-03T13:06:53.764Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:25.653Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/801","datestamp":"2018-11-14T15:19:38Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Ciencias de la Actividad Física y del Deporte","name":"Ciencias La Actividad Física Y. Del Deporte","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Actividad Física y del Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Prevención","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Hidratación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deportistas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Electrolitos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deportes de resistencia","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"2411.06 Fisiología del Ejercicio","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"La importancia de la hidratación en competiciones resistencia, por Juan del Coso","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Juan del Coso, investigador y profesor en el Grado en Ciencias de la Actividad Física y del Deporte UCJC, presenta el proyecto de investigación que ha realizado el departamento CCAFD de UCJC: la prevención de desequilibrios de agua y electrolitos que se dan en los deportes de resistencia para que los deportistas realicen su actividad de una manera más segura.\nMás información: http://www.ucjc.edu/2016/03/investiga...\nUniversidad Camilo José Cela, Ciencias de la Actividad Física y del Deporte","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2016-03-21","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Grado en Ciencias de la Actividad Física y del Deporte UCJC","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/801"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2016-03-21","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055869487,"id":"50|od______3989::2e3f34ce90520fae350a7e1148d7dcea","originalId":["od______3989::2e3f34ce90520fae350a7e1148d7dcea"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.912Z","dateoftransformation":"2020-03-03T13:06:54.218Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.912Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/800","datestamp":"2018-11-14T13:43:21Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Actividad Física y del Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Cafeína","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Efectos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Alto rendimiento","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Competiciones","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Beneficios","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"2411.06 Fisiología del Ejercicio","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Investigación - Efectos positivos y negativos del uso de la cafeína en el deporte","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Proyecto de investigación que se están desarrollando en la Universidad Camilo José Cela (UCJC) sobre la cafeína y sus efectos cuando se realiza deporte. https://www.ucjc.edu/2016/08/la-bbc-s...\n\nJuan Del Coso, director del Laboratorio de Fisiología del Ejercicio de la Universidad Camilo José Cela, nos describe cómo está siendo la investigación que desarrolla con deportistas de alto rendimiento y el consumo de cafeína.\n\n\"Nuestra investigación se enmarca dentro del campo de la fisiología del ejercicio y el objetivo que tenemos es investigar los efectos que tiene la cafeína en los deportistas, principalmente saber si es una ayuda ergogénica en la mayor parte de los deportes, pero también conocer qué perjuicios existen para aquellos deportistas que la utilizan para incrementar el rendimiento deportivo\", explica Del Coso.\n\nY es que los datos son muy llamativos: tres de cada cuatro deportistas de alto rendimiento utilizan la cafeína antes de la competición y muchas veces se utiliza sin saber los perjuicios que puede tener para el deportista. \"Lo que estamos investigando es cómo una sustancia, que te puede hacer rendir mejor, cómo puede afectar negativamente cuando termina una competición\".\n\nPero esta investigación no sólo puede extraer conclusiones interesantes y útiles para los deportistas profesionales, \"investigando el deporte de alto rendimiento podemos conocer los beneficios y perjuicios de la cafeína que también afectarían al deportista amateur y hacer un deporte en general más seguro\".\n\nPara el desarrollo de este proyecto de investigación están colaborando el Laboratorio de Fisiología del Ejercicio de la Universidad Camilo José Cela y la Agencia para la Protección de la Salud en el Deporte.\nPara el desarrollo de este proyecto de investigación están colaborando el Laboratorio de Fisiología del Ejercicio de la Universidad Camilo José Cela y la Agencia para la Protección de la Salud en el Deporte.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2017-10-06","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Universidad Camilo José Cela, Vicerrectorado de Innovación","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/800"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2017-10-06","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055869498,"id":"50|od______3989::2f4f3c820c450bd08dac08d07cc82dcf","originalId":["od______3989::2f4f3c820c450bd08dac08d07cc82dcf"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.806Z","dateoftransformation":"2020-03-03T13:06:54.229Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.806Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/797","datestamp":"2018-11-14T12:12:23Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Cuesta Cano, Laura","name":"Laura","surname":"Cuesta Cano","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Comunicación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Marketing digital","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Medios sociales","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Marcas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Empresas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Emprendedores","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"II Liga de Debate CICAE - UCJC","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"6114.13 Marketing","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Laura Cuesta - Liga de Debate CICAE - UCJC","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Laura Cuesta, profesora en la Universidad Camilo José Cela (UCJC) y Especialista en Marketing Digital y Medios Sociales para marcas y emprendedores. \n\nEn esta ocasión, Laura Cuesta, nos explica el significado de redes sociales y nos habla sobre el uso que le dan los usuarios.\n\nII Liga de Debate CICAE - UCJC http://ligacicae.ucjc.edu/\n\nLa Universidad Camilo José Cela (UCJC) y la Asociación de Colegios Privados e Independientes –Círculo de Calidad Educativa (CICAE) – organizan la segunda edición de esta gran liga de debate académico. El formato es al estilo inglés, similar al de la Oxford Union, donde algunos jóvenes se reunían a debatir sobre temas de la más candente actualidad y polemizaban con la intención de formarse en tan valoradas habilidades.\n\nLa Liga de Debate CICAE-UCJC es una ocasión única para los alumnos de ponerse en contacto con herramientas tan útiles como la oratoria y la comunicación efectiva. Pero, sobretodo, una oportunidad de fomentar el pensamiento crítico y la socialización entre alumnos con espíritu emprendedor. La Liga de Debate CICAE - Universidad Camilo José Cela se diferencia del resto de torneos en que potenciamos que sea eminentemente formativa. Alumnos y profesores reciben formación en comunicación, oratoria y debate por parte de alguno de los expertos en debate españoles más reconocidos en la escena internacional.\n\nDatos de la III Liga de Debate CICAE - UCJC http://ligacicae.ucjc.edu/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-10-23","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Universidad Camilo José Cela, Vicerrectorado de Innovación","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/797"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-10-23","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055870582,"id":"50|od______3989::752fd0b2bbac1ea1cc50e52fd46eb663","originalId":["od______3989::752fd0b2bbac1ea1cc50e52fd46eb663"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.561Z","dateoftransformation":"2020-03-03T13:06:55.716Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.561Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/798","datestamp":"2018-11-14T12:57:26Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Psicología","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Alcohol","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Dependencia","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Proyectos de investigación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Alcoholismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Realidad virtual","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Psicología conductual","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Investigación - Intervención basada en realidad virtual sobre la dependencia del alcohol","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Proyecto de investigación que se están desarrollando en la Universidad Camilo José Cela (UCJC) con tecnología aplicada al abandono de la dependencia del alcohol. \nRosa Jurado, profesora e investigadora de UCJC https://www.ucjc.edu/, explica cómo se conjuga el alcoholismo con la realidad virtual. \"Mi investigación consiste en desarrollar una técnica de intervención o rehabilitación que tiene como propósito fomentar la capacidad de inhibición para que las personas que tienen dependencia del alcohol sean capaces de controlar su conducta de aproximación hacia aquello que tiene que ver con el consumo de alcohol\".\n\nEn esta investigación, liderada por la Universidad Camilo José Cela, están participando, el Instituto de Investigación Biomédica del Hospital 12 de Octubre y la Universidad Politécnica de Madrid, con el CeDint. \"En nuestro equipo de investigación pensamos que, además del deseo de consumo, existen una carencia clara de capacidad de control inhibitorio en este tipo de personas\", destaca Rosa Jurado.\n\nLa realidad virtual sirve, según explica Gabriel Rubio, jefe de Psiquiatría del Hospital 12 de Octubre, \"para dar un paso más, para ver cómo reacciona un sujeto en un ambiente que no es el real, pero se parece mucho al real\". El objetivo final de estas acciones es \"mejorar la capacidad del individuo para poner en marcha mecanismos de inhibición conductual, para que el sujeto sea capaz de decir que no\".\n\nY no acaba ahí el proceso. La fase final del tratamiento se enfoca en que el paciente \"sea capaz de reconstruir su vida\" a través de programas de entrenamiento de habilidades sociales, asertividad, familias... \"Juntando todas las áreas de intervención, la abstinencia se mantiene mucho más tiempo y las recaídas disminuyen\".","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2017-10-05","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Universidad Camilo José Cela, Vicerrectorado de Innovación","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/798"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2017-10-05","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055870750,"id":"50|od______3989::7fcbe3a03280663cddebfd3cb9203177","originalId":["od______3989::7fcbe3a03280663cddebfd3cb9203177"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:25.652Z","dateoftransformation":"2020-03-03T13:06:55.95Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:25.652Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/796","datestamp":"2018-11-05T13:36:54Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Comunicación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Redes sociales","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Comunicación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Información","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Desinformación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"III Liga de Debate CICAE","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Instantaneidad","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Información periodística","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Fake news","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"5910.02 Medios de Comunicación de Masas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Jorge Gallardo - Liga de Debate CICAE","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Jorge Gallardo, profesor de Derecho en la Universidad Camilo José Cela (UCJC), Doctor en Comunicación Audiovisual y MBA en Empresas Audiovisuales. Subdirector de Espejo Público en Antena 3 Noticias. \n\nEn esta ocasión, Jorge Gallardo habla sobre las redes sociales, cómo han transformado la manera en la que nos comunicamos y nos informamos. \n\nIII Liga de Debate CICAE - UCJC http://ligacicae.ucjc.edu/\n\nLa Universidad Camilo José Cela (UCJC) y la Asociación de Colegios Privados e Independientes –Círculo de Calidad Educativa (CICAE) – organizan la tercera edición de esta gran liga de debate académico. El formato es al estilo inglés, similar al de la Oxford Union, donde algunos jóvenes se reunían a debatir sobre temas de la más candente actualidad y polemizaban con la intención de formarse en tan valoradas habilidades.\n\nLa Liga de Debate CICAE-UCJC es una ocasión única para los alumnos de ponerse en contacto con herramientas tan útiles como la oratoria y la comunicación efectiva. Pero, sobretodo, una oportunidad de fomentar el pensamiento crítico y la socialización entre alumnos con espíritu emprendedor. La Liga de Debate CICAE - Universidad Camilo José Cela se diferencia del resto de torneos en que potenciamos que sea eminentemente formativa. Alumnos y profesores reciben formación en comunicación, oratoria y debate por parte de alguno de los expertos en debate españoles más reconocidos en la escena internacional.\n\nDatos de la III Liga de Debate CICAE - UCJC http://ligacicae.ucjc.edu/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-10-23","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/796"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-10-23","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055872001,"id":"50|od______3989::d791339867bec6d3eb2104deeb4e4961","originalId":["od______3989::d791339867bec6d3eb2104deeb4e4961"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.586Z","dateoftransformation":"2020-03-03T13:06:57.721Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.586Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/645","datestamp":"2018-04-12T07:59:27Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Transferencia de Conocimiento e Innovación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Innovación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Investigación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Ciencia","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Tecnología","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"UCJC Open Science Day 2018","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"El UCJC Open Science Day tiene como objetivo mostrar las actividades de investigación llevadas a cabo por investigadores de la Universidad Camilo José Cela (UCJC) a todos los miembros de la UCJC.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-04-10","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/645"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-04-10","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055872022,"id":"50|od______3989::d90d3a1f64ad264b5ebed8a35b280343","originalId":["od______3989::d90d3a1f64ad264b5ebed8a35b280343"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.912Z","dateoftransformation":"2020-03-03T13:06:57.747Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.912Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/795","datestamp":"2018-11-06T15:43:10Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Comunicación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Unión Europea","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Desinformación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Política","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Periodismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Ciudadanos europeos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Democracias","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Estado de derecho","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Derechos humanos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"5910.02 Medios de Comunicación de Masas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Ramón Luis Varcárcel - La Unión Europea ante el reto de la desinformación","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Conferencia de Ramón Luis Valcárcel, vicepresidente del Parlamento Europeo, en el Campus de Almagro de la Universidad Camilo José Cela (UCJC) bajo el título \"La Unión Europea ante el reto de la desinformación\", en la que ha alertado sobre el alcance de la desinformación en las democracias y sobre cómo pone en peligro los derechos de los ciudadanos europeos. También ha participado el rector de la UCJC, Samuel Martín-Barbero.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-10-11","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/795"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-10-11","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/dataset_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/dataset_10.json.gz deleted file mode 100644 index ee62cd7913e67677f1d81e880f137d1545656608..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6888 zcmVxLlHK z`R}e$UCC6?oiw_82FOki0!pH7RwUA)q)MD@1Dj{qC+N$*=!-$_a_ieX#XiZNLsFLP z`p;BHO2XYnrfl=)@O+2A=i}k`y@VF@Tr6a7{(X;&OP=#6Wxe@=ra9}wHw&f|OLku* zEM)~tqK*EAiMUQ%s4_2l^PV@D^=wZ3eXZmr6I7?{!4LIk8W&s&tT;|-o^zt*3ZAjNpxL@N_rq!Eh2eBOoq028G}aV)|1Fa*_5TfL)7gCP1xXSnK^z2u zKl39mnnlTIF&xHIHXDZlYD+4<P{2QX56u2A(27h`76UK~B{L}E``%Y<1G309 z6N#h>OPkNJJgDrFrX@D^30rS}L|u_5P1ZkvD{Yul(WM3+KUj;LZI1>Q$)=MMb`lDx=ru7|+Auz#r=z z>|01}u=pobCn<00?t5$BPgou+zOGcaqUsVl(i5odc6n%8FpVVO1ZYJxXMcb36!NT! zV*Tgg$)AId;onYuTn@_kA|9}$gs(vs{o~gq)P?9KOcd5YQPBBy`R0K)2t9A$qg2P? zIIvo=-M*&{-t&(F5H>_rcoP&fV+BR4{0tz;*HqgL$~Zqh{^px+2G|>^idP((J&AtprzGu<>gr+G~i99-`8I4!6Pdcl#P+R7*mqcYzI{ht&}{po zXv&bYpMSuw<MJkO+qp-mVrxdPoMG;uJr3d8n&!dzEq6$AKvO_x+8)Zx%34zNa6u!wb2 zLsxKcFtiE7goA&sF;MdiM$aI4&2p6_&e z|ADW!E+ZJ}Ryc|orxVK}s+N+8#Pk*OZ7xd{+n$5|u7Z7A*iYIEYyOU<(A}f3PQnVC zXDs3LDUHYxx#Glk0Npus8RZ+7pMfKWq)L0>Q0)==exXz3x#G)JQ6~}>rtcg2#t%1Z z^LqW+0Z!|!iH`AkU&ZfsWj9Kj!w{R50yg2;{G^PFyjUq&E>}6rV9Y+yJb7McC2~QG zcalUbm14=ok`#I!W=bTsBmzaj$j@U)o>pymG>Do8I1PIUnDK_<4DIy(tLuG1GpG1A z6c0lZPtXOA-@vegT#EDs+vytzDlT$6Zv!YPk750^H-}q3N~{=&eaWz8P^YG&*wp!P zv^f&-kw}h51Mk?)BoCPC_oj%Z%%49$`|Lon&)2YsJ<%Di5yO=9WhRpw*c*EZ-qgTf zrLA~gFgmFFW1eQ$wH;R~v9{+gt2y!d=3kc(gjmU3BaZ?+Dh09MD^-f*NNc~VBps}% zNK!Tcd5*8}wQwbCHxejhD&vaA*1JNNd~{LW8v60q&`P4Y!+0F8VC~FkhJ2-J8Fu(< zTq1KN{5dpB9iJ|aF(L{SHQ%Y|2P!3)T^Z!qn;+L2Sa6y;n z2G%h6fdG`SX5~v7^X-p8Iho>eBe%K9?^3g9(m)Vz;in7hUfTkyq?Yv z(M`i~euxhEA@I1V%YFzqB0E2X^FutMA7bXsCO7&a7Sq{iJc>s&9>wt0f_d@89}S0* zKMARhFmbCN;;!u7^h1mv>W2tNAJPpm?t~j+>RlU81lJZ(o^Lu}+5{#T z`93U_VA_VggSoD%XZy1YreYp**KfMwp{hHL6igqpKWyPfPESuhep25Uaqg?%kUiYr zef1Hn6#@DeGGj{i>#;VGN*jcXVGG01WFBZf_i%0Xt!F2K@}(^|C+YURVDy@izjF1g zDEgCfsuXN7trv}4vGr4 z&Dp+%RE3q3T?~b^^YHS|+NW{VWYhJ%} zj>s-z>l~4ta75yPbC(^FZccQLNau)rOh;tkhc`JQ{YfxG58~9H&7xVD_(?b(PeY$B zCdn`k!#f<2cV_RPBQhL}!{GxRk(2R<^F;3CY2Op+-|C6BppT60LWj)WVRIs*diW?k^yc7Zs z^fJqV;+PZ;o;#ToPxrmEW^1x)EV|H58p~|W3U0|3jZW4lkd=H|YM*sVPGx=zu#Qa8 zMU%KcnPLy-eKG}XU7`F1%i&Ms;Qyq$0sKIxJf3@jRe_D+MOx~_L4{c>%mptW>;)eb}VMh-#vBvyfQ8N-Ms*BIb^^*>Px~WcHC(?^GBV!D? z^RlL5VD3^-l2O4RpXOTRY*{M27yS%Bfx+QvotAe_8s|>csn#(%5stm#@lG{^!ChgY zolDoTAbvwS{gqe{|H+8u&~&5+hpi($bcOWbJJQ4NtYbOS!yg#`J4p|t>5aFq12$X? zM=_guVc;z&8-^a8Fy9|dlSvZmdwI8!9`4NEJ<`K)G?@9pgGdjfLEuk66zQRsr+w0c z4S8`b>0vk@4d;R1D(T@LFaklBk8yO;!NAD55b|>8|FfkWB&*WMpMTJ=5jaMJV>GyV z!?|xe2f`^`u6#Jxd(;eeBUX2MC|-dvTxf?)f$8G}LGFO)g?3m#wnmE_ASXdp6}@Z4_7r=>kosb?{OiJWpS|j(~JvDn3tIrt5Ti`5a0U@553ft ziV!Z5Q59puM4?Rh_?Uv}*|seX+FnPl&^5~Fg6YWXC~^TaZ>ED#Gy<-huoM3D zlRn0GmvwxNDirT^R+WQ4bT_Zt{v@#qN;KLJu5c^3?MTj@0=9MFlJ&3jfZ|<0K45V zc2YJqPKD*(XzQ{|S2TzX)ETd!s7r?NO29sBe$c(JD4>iDeWBRvk|hRzWwf!I3Q({C zzB*t`L?4{WhJEP4A-CNEk%g{qxO9DMQ?~FGMX+?s6v^0yRDGMv{7%utK=*c{qQ-9Q zU{NrYaVQ)sXtwutPpkhq8U~7X*gph}Gut22st$d+4N`#V#A%ruIv*E!D4iRXZpij0 zP#FBB)o^GzhU5kL%$ozRd;5CW&7apIxt59UUb{CG;O}U+*hLCi4a&jftTkB~zSR`; zX!yH{6G2O3C9=_swo9=S2s*?P;e`w}%}Q4K+d*!}EIoQa!`N-?GdoelGS$Pa)xcwp zy%?Q`l9OeFTLzzD3M*J#zdS+7OC3MQtYGSJIKRhMIiK+(Qqd+yzIWt}old((Mf(q( zWDZS7Mf;;;>!@fQp`v-6rlPqi&{5GG741=}XtU{!RJ6rtF^s2y7mq?OO1x=8WB89Z ziQ&s6Ui9vuqTQ9flT@^+H-0D;E%;y$lGpOIO+}l`u6dArM@0+f@GlsCSSs3SsgF!f zsgQlx;FhZKAbIt%rjj*M_Z)BRQF&vhGAr>6)CKuy-Rb#^s;^*N7fX_uq0qt6DjZA) z!_Fo2medh5TJIaN3%k-XbuYuK;SPlkpdb>KNaY5lcA+>wCr|lO2kv;nB8o4YoS&YY z-TAVK!{)-4VN^JNjiE$PvN1i1q!6U`-q7@1obxhcLZ1 z%ylWNXDOj*rU!^}_btvzG`XhC;g|5KT=C2H%r;+GU3BtqY2!tnHtn|F-1Y4z&YK%%liNUw2>;qM!~8+{}wVNs0Rt~B~oZU3{(#- zy4d~*rfbMaij~6fSWLZKS&#~r?T$w@UnQm@(TpJIk)>_(hO9wf8MH5=nh$2Ects<= zk(OI&nj9%05m-Mujj1*Xo|^Zd1{SvKFxLm66>F`sNV5G&bJlhoA?QZLopNju|Aq=YkNYA9n0(d7Y{eiZTE<5peedSgQ3%@V*~vm(R6H}-!Ha~4b%xX z5ITLk%m#8(pko6$Hqhg-fhJx!4Q^rsO~PQ2cu^2VH1-+w<2dvfem5GkXgtPVcpDq& z&g`9J1BGM%L2RIKF!9G9kPT$zX`c-=?cK%(3Mcbm>;h9cg3&z*M$hq0a&te)`CQtK z@)FCHbS$E_Lx17nrF#{nZ6K=P1Nqf^mkyW?3{yPMTvV4`2 zafXZ(%{{m~7?m|oBq?x=uO!ewpx$!pO zA~E4vn6yBk>iW9~PCD8TwR{m6UW~!-S%d}8T84?w!h{L(QdmhUCNK3=fZ2K1A%CC%} zeFTF>(5Zy!640;=W<5O&8KiZmf-f-s+eViK4WirkSaJfWa8Yg;M$K|AzDJKE6gr(c z=FlG!O~)Mi{bK8wLmgobO*_pTa#NsV4msw~V={*(p?4#5Xc0$jNWEb&^~U32oFpt- zB*QqG1>Vq~U|^YBnL~GF?;dj~2nXZvVNcqHgIV~Yh(onF?GuM=?6xb!AumK>hZs@G z5r-Uc$VDdl4NemKp5f+^lEb1+EL^wf3>2uY>WZgBqFt69Y*ri?-2mgJJH-x_b_Xjw z&w_sp*RKwXu1p;0HsvzezRwGo)g3Nv;_@;5iV?xSaZB7*VF+iwH}HZgdXJ7x(~L-> z`5-P3VQ0vYpGvi4K{YeSK0bo{|&|V*%V3H~*Y4cYM=Jm3S8qYH7U||=MJm3C=jy5gN2?lGTWJo@f zNN1xrMgTi7?`AwN!*bc@+ZYPfiV9Mn6q}bCGm4L(SzQ={QND`{l*uY!nBB?17e=CA z9;2tzA;Ai!VyzAr63m4Jb0NW8NU%TH1nc6V{FaD)j%W6JHXO$@>jcki+-aVf8<8E) z%<;?~k!Kcoo_`b1ESb#wA^bU>L@XXBLlydgEo|@adOk}DxgPgMbeCPwiA?5&i7SB-Cu~dvc1=J)lA-|Umc#H2k<^ZuX7n3Y zIZN*4TBX`GmTSKaEHbl5?VK%`zy(BjT)zqK3q7Y(=QnYF6X!Si1Ncqwc&yug6E`9| zzlrmkJfhzuxbbC=WacG9>MzD4ngruf6tajVQ#uTzVCoG=cfIU!XZB9|O=kYX?&nVi zlhNb@`c14nZTn3ovuph(ctL;s$`9wG;oSF}-^BS%+y(vL;4INRjs(NUV~ev=L0@$z zlI||D1H;DLmZ|2LyFSq_4Em0%d*Y*Q98P%&Ya@#H6PTVWq`O4f@???8SYdc~2Slqt zoWT~!>_8H8WzoUWZokkVCRJ zv2JV?3B~J+`tD;WKZM2U!ANGzm)JsVyqf8F~f28(a4)OYMx`1%o3 zdUiMWl+AlQsJQF%o6OjT-P(+7Y+iG@A1_s7sx?bFHZj$MqMfnq1>N}SQhPvN;rl7u zcle@D{ftf`O?#?X1(p`lCogGM!Y|Jt;r6{a`iQECDe34U59$v0xzy$RPN$A8((rmX ix`?BT{9frI=oRcXUBr#ZjxO@%PyY{%{0-$MQ~?0JZc1qY diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/datasource b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/datasource new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/otherresearchproduct b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/otherresearchproduct new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/publication b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/publication new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/software b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/contextnoprovenance/software new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/nocontext/dataset b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/nocontext/dataset new file mode 100644 index 000000000..54552697b --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/nocontext/dataset @@ -0,0 +1,10 @@ +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055868909,"id":"50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529","originalId":["od______3989::02dd5d2c222191b0b9bd4f33c8e96529"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[{"value":"10.4185/RLCS-2018-1243","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofcollection":"2020-03-03T13:05:26.091Z","dateoftransformation":"2020-03-03T13:06:53.161Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:26.091Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/562","datestamp":"2018-01-23T15:06:07Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Gallardo-Camacho, Jorge","name":"Jorge","surname":"Gallardo-Camacho","rank":1,"pid":null,"affiliation":null},{"fullname":"Trujillo Fernández, José Ramón","name":"José Ramón","surname":"Trujillo Fernández","rank":2,"pid":null,"affiliation":null},{"fullname":"Jorge Alonso, Ana","name":"Ana","surname":"Jorge Alonso","rank":3,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"COVID-19","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"yihadismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"viralidad","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"vídeo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"propaganda","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"terrorismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"terrorism","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"jihadism","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"virality","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"video","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"5903 Ideologías Políticas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"6310.13 Terrorismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"El individualismo como estrategia","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Individualism as an improvised strategy","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Se analiza como el Yihadismo distribuye su mensaje propaganístico en You Tube.\nUniversidad Camilo José Cela","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"MediaLab - UCJC","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/562"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055868954,"id":"50|od______3989::05d8c751462f9bb8d2b06956dfbc5c7b","originalId":["od______3989::05d8c751462f9bb8d2b06956dfbc5c7b"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:26.079Z","dateoftransformation":"2020-03-03T13:06:53.239Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:26.079Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/227","datestamp":"2017-11-15T16:29:55Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Rico Pérez, Marta","name":"Marta","surname":"Rico Pérez","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"coronavirus disease-19","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"agriculture","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Retos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"I Congreso Nuevos Retos en los Eventos Deportivos","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"I Congreso Nuevos Retos en los Eventos Deportivos, celebrado en la Universidad Camilo José Cela.\n\n\n\nDerivado del proyecto de investigación \"La situación de los eventos deportivos en España a través de su estructuras y de sus herramientas\", tiene como finalidad conocer la situación de los eventos deportivos en España. Se realizó un estudio descriptivo holístico en el que se investigaron elementos culturales, el perfil de los recursos humanos, la estructura de los comités, entre otras.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2015-06-20","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/227"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2015-06-20","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055869097,"id":"50|od______3989::0f89464c4ac4c398fe0c71433b175a62","originalId":["od______3989::0f89464c4ac4c398fe0c71433b175a62"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:26.069Z","dateoftransformation":"2020-03-03T13:06:53.54Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:26.069Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/802","datestamp":"2018-11-14T15:30:47Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Ciencias de la Actividad Física y del Deporte","name":"Ciencias La Actividad Física Y. Del Deporte","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"fishery","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"sea","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Impacto físico","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Proyectos de investigación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"2411.06 Fisiología del Ejercicio","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Investigaciones en Ciencias de la Actividad Física y del Deporte CCAFD - UCJC","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Blanca Romero, investigadora y docente de Ciencias de la Actividad Física y del Deporte en la Universidad Camilo José Cela, describe dos de las líneas de investigación que se han desarrollado en el departamento CCAFD de UCJC: el impacto físico de las diferentes actividades deportivas y los métodos de recuperación más eficaces tras el ejercicio físico.\nBlanca también destaca la importancia en el ámbito de la salud los estudios en Ciencias de la Actividad Física y del Deporte.\nCiencias de la Actividad Física y del Deporte en la Universidad Camilo José Cela","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2016-03-21","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Universidad Camilo José Cela, Ciencias de la Actividad Física y del Deporte","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/802"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2016-03-21","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055869215,"id":"50|od______3989::1729c3988199b95d1d566851af7d3c55","originalId":["od______3989::1729c3988199b95d1d566851af7d3c55"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:25.653Z","dateoftransformation":"2020-03-03T13:06:53.764Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:25.653Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/801","datestamp":"2018-11-14T15:19:38Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Ciencias de la Actividad Física y del Deporte","name":"Ciencias La Actividad Física Y. Del Deporte","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Actividad Física y del Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Prevención","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Hidratación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deportistas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Electrolitos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deportes de resistencia","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"2411.06 Fisiología del Ejercicio","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"La importancia de la hidratación en competiciones resistencia, por Juan del Coso","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Juan del Coso, investigador y profesor en el Grado en Ciencias de la Actividad Física y del Deporte UCJC, presenta el proyecto de investigación que ha realizado el departamento CCAFD de UCJC: la prevención de desequilibrios de agua y electrolitos que se dan en los deportes de resistencia para que los deportistas realicen su actividad de una manera más segura.\nMás información: http://www.ucjc.edu/2016/03/investiga...\nUniversidad Camilo José Cela, Ciencias de la Actividad Física y del Deporte","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2016-03-21","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Grado en Ciencias de la Actividad Física y del Deporte UCJC","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/801"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2016-03-21","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055869487,"id":"50|od______3989::2e3f34ce90520fae350a7e1148d7dcea","originalId":["od______3989::2e3f34ce90520fae350a7e1148d7dcea"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.912Z","dateoftransformation":"2020-03-03T13:06:54.218Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.912Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/800","datestamp":"2018-11-14T13:43:21Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Actividad Física y del Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Cafeína","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Efectos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Alto rendimiento","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Competiciones","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Beneficios","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"2411.06 Fisiología del Ejercicio","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Investigación - Efectos positivos y negativos del uso de la cafeína en el deporte","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Proyecto de investigación que se están desarrollando en la Universidad Camilo José Cela (UCJC) sobre la cafeína y sus efectos cuando se realiza deporte. https://www.ucjc.edu/2016/08/la-bbc-s...\n\nJuan Del Coso, director del Laboratorio de Fisiología del Ejercicio de la Universidad Camilo José Cela, nos describe cómo está siendo la investigación que desarrolla con deportistas de alto rendimiento y el consumo de cafeína.\n\n\"Nuestra investigación se enmarca dentro del campo de la fisiología del ejercicio y el objetivo que tenemos es investigar los efectos que tiene la cafeína en los deportistas, principalmente saber si es una ayuda ergogénica en la mayor parte de los deportes, pero también conocer qué perjuicios existen para aquellos deportistas que la utilizan para incrementar el rendimiento deportivo\", explica Del Coso.\n\nY es que los datos son muy llamativos: tres de cada cuatro deportistas de alto rendimiento utilizan la cafeína antes de la competición y muchas veces se utiliza sin saber los perjuicios que puede tener para el deportista. \"Lo que estamos investigando es cómo una sustancia, que te puede hacer rendir mejor, cómo puede afectar negativamente cuando termina una competición\".\n\nPero esta investigación no sólo puede extraer conclusiones interesantes y útiles para los deportistas profesionales, \"investigando el deporte de alto rendimiento podemos conocer los beneficios y perjuicios de la cafeína que también afectarían al deportista amateur y hacer un deporte en general más seguro\".\n\nPara el desarrollo de este proyecto de investigación están colaborando el Laboratorio de Fisiología del Ejercicio de la Universidad Camilo José Cela y la Agencia para la Protección de la Salud en el Deporte.\nPara el desarrollo de este proyecto de investigación están colaborando el Laboratorio de Fisiología del Ejercicio de la Universidad Camilo José Cela y la Agencia para la Protección de la Salud en el Deporte.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2017-10-06","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Universidad Camilo José Cela, Vicerrectorado de Innovación","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/800"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2017-10-06","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055869498,"id":"50|od______3989::2f4f3c820c450bd08dac08d07cc82dcf","originalId":["od______3989::2f4f3c820c450bd08dac08d07cc82dcf"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.806Z","dateoftransformation":"2020-03-03T13:06:54.229Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.806Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/797","datestamp":"2018-11-14T12:12:23Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Cuesta Cano, Laura","name":"Laura","surname":"Cuesta Cano","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Comunicación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Marketing digital","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Medios sociales","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Marcas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Empresas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Emprendedores","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"II Liga de Debate CICAE - UCJC","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"6114.13 Marketing","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Laura Cuesta - Liga de Debate CICAE - UCJC","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Laura Cuesta, profesora en la Universidad Camilo José Cela (UCJC) y Especialista en Marketing Digital y Medios Sociales para marcas y emprendedores. \n\nEn esta ocasión, Laura Cuesta, nos explica el significado de redes sociales y nos habla sobre el uso que le dan los usuarios.\n\nII Liga de Debate CICAE - UCJC http://ligacicae.ucjc.edu/\n\nLa Universidad Camilo José Cela (UCJC) y la Asociación de Colegios Privados e Independientes –Círculo de Calidad Educativa (CICAE) – organizan la segunda edición de esta gran liga de debate académico. El formato es al estilo inglés, similar al de la Oxford Union, donde algunos jóvenes se reunían a debatir sobre temas de la más candente actualidad y polemizaban con la intención de formarse en tan valoradas habilidades.\n\nLa Liga de Debate CICAE-UCJC es una ocasión única para los alumnos de ponerse en contacto con herramientas tan útiles como la oratoria y la comunicación efectiva. Pero, sobretodo, una oportunidad de fomentar el pensamiento crítico y la socialización entre alumnos con espíritu emprendedor. La Liga de Debate CICAE - Universidad Camilo José Cela se diferencia del resto de torneos en que potenciamos que sea eminentemente formativa. Alumnos y profesores reciben formación en comunicación, oratoria y debate por parte de alguno de los expertos en debate españoles más reconocidos en la escena internacional.\n\nDatos de la III Liga de Debate CICAE - UCJC http://ligacicae.ucjc.edu/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-10-23","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Universidad Camilo José Cela, Vicerrectorado de Innovación","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/797"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-10-23","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055870582,"id":"50|od______3989::752fd0b2bbac1ea1cc50e52fd46eb663","originalId":["od______3989::752fd0b2bbac1ea1cc50e52fd46eb663"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.561Z","dateoftransformation":"2020-03-03T13:06:55.716Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.561Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/798","datestamp":"2018-11-14T12:57:26Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Psicología","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Alcohol","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Dependencia","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Proyectos de investigación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Alcoholismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Realidad virtual","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Psicología conductual","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Investigación - Intervención basada en realidad virtual sobre la dependencia del alcohol","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Proyecto de investigación que se están desarrollando en la Universidad Camilo José Cela (UCJC) con tecnología aplicada al abandono de la dependencia del alcohol. \nRosa Jurado, profesora e investigadora de UCJC https://www.ucjc.edu/, explica cómo se conjuga el alcoholismo con la realidad virtual. \"Mi investigación consiste en desarrollar una técnica de intervención o rehabilitación que tiene como propósito fomentar la capacidad de inhibición para que las personas que tienen dependencia del alcohol sean capaces de controlar su conducta de aproximación hacia aquello que tiene que ver con el consumo de alcohol\".\n\nEn esta investigación, liderada por la Universidad Camilo José Cela, están participando, el Instituto de Investigación Biomédica del Hospital 12 de Octubre y la Universidad Politécnica de Madrid, con el CeDint. \"En nuestro equipo de investigación pensamos que, además del deseo de consumo, existen una carencia clara de capacidad de control inhibitorio en este tipo de personas\", destaca Rosa Jurado.\n\nLa realidad virtual sirve, según explica Gabriel Rubio, jefe de Psiquiatría del Hospital 12 de Octubre, \"para dar un paso más, para ver cómo reacciona un sujeto en un ambiente que no es el real, pero se parece mucho al real\". El objetivo final de estas acciones es \"mejorar la capacidad del individuo para poner en marcha mecanismos de inhibición conductual, para que el sujeto sea capaz de decir que no\".\n\nY no acaba ahí el proceso. La fase final del tratamiento se enfoca en que el paciente \"sea capaz de reconstruir su vida\" a través de programas de entrenamiento de habilidades sociales, asertividad, familias... \"Juntando todas las áreas de intervención, la abstinencia se mantiene mucho más tiempo y las recaídas disminuyen\".","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2017-10-05","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Universidad Camilo José Cela, Vicerrectorado de Innovación","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/798"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2017-10-05","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055870750,"id":"50|od______3989::7fcbe3a03280663cddebfd3cb9203177","originalId":["od______3989::7fcbe3a03280663cddebfd3cb9203177"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:25.652Z","dateoftransformation":"2020-03-03T13:06:55.95Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:25.652Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/796","datestamp":"2018-11-05T13:36:54Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Comunicación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Redes sociales","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Comunicación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Información","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Desinformación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"III Liga de Debate CICAE","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Instantaneidad","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Información periodística","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Fake news","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"5910.02 Medios de Comunicación de Masas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Jorge Gallardo - Liga de Debate CICAE","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Jorge Gallardo, profesor de Derecho en la Universidad Camilo José Cela (UCJC), Doctor en Comunicación Audiovisual y MBA en Empresas Audiovisuales. Subdirector de Espejo Público en Antena 3 Noticias. \n\nEn esta ocasión, Jorge Gallardo habla sobre las redes sociales, cómo han transformado la manera en la que nos comunicamos y nos informamos. \n\nIII Liga de Debate CICAE - UCJC http://ligacicae.ucjc.edu/\n\nLa Universidad Camilo José Cela (UCJC) y la Asociación de Colegios Privados e Independientes –Círculo de Calidad Educativa (CICAE) – organizan la tercera edición de esta gran liga de debate académico. El formato es al estilo inglés, similar al de la Oxford Union, donde algunos jóvenes se reunían a debatir sobre temas de la más candente actualidad y polemizaban con la intención de formarse en tan valoradas habilidades.\n\nLa Liga de Debate CICAE-UCJC es una ocasión única para los alumnos de ponerse en contacto con herramientas tan útiles como la oratoria y la comunicación efectiva. Pero, sobretodo, una oportunidad de fomentar el pensamiento crítico y la socialización entre alumnos con espíritu emprendedor. La Liga de Debate CICAE - Universidad Camilo José Cela se diferencia del resto de torneos en que potenciamos que sea eminentemente formativa. Alumnos y profesores reciben formación en comunicación, oratoria y debate por parte de alguno de los expertos en debate españoles más reconocidos en la escena internacional.\n\nDatos de la III Liga de Debate CICAE - UCJC http://ligacicae.ucjc.edu/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-10-23","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/796"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-10-23","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055872001,"id":"50|od______3989::d791339867bec6d3eb2104deeb4e4961","originalId":["od______3989::d791339867bec6d3eb2104deeb4e4961"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.586Z","dateoftransformation":"2020-03-03T13:06:57.721Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.586Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/645","datestamp":"2018-04-12T07:59:27Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Transferencia de Conocimiento e Innovación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Innovación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Investigación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Ciencia","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Tecnología","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"UCJC Open Science Day 2018","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"El UCJC Open Science Day tiene como objetivo mostrar las actividades de investigación llevadas a cabo por investigadores de la Universidad Camilo José Cela (UCJC) a todos los miembros de la UCJC.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-04-10","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/645"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-04-10","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055872022,"id":"50|od______3989::d90d3a1f64ad264b5ebed8a35b280343","originalId":["od______3989::d90d3a1f64ad264b5ebed8a35b280343"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.912Z","dateoftransformation":"2020-03-03T13:06:57.747Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.912Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/795","datestamp":"2018-11-06T15:43:10Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Comunicación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Unión Europea","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Desinformación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Política","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Periodismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Ciudadanos europeos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Democracias","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Estado de derecho","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Derechos humanos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"5910.02 Medios de Comunicación de Masas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Ramón Luis Varcárcel - La Unión Europea ante el reto de la desinformación","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Conferencia de Ramón Luis Valcárcel, vicepresidente del Parlamento Europeo, en el Campus de Almagro de la Universidad Camilo José Cela (UCJC) bajo el título \"La Unión Europea ante el reto de la desinformación\", en la que ha alertado sobre el alcance de la desinformación en las democracias y sobre cómo pone en peligro los derechos de los ciudadanos europeos. También ha participado el rector de la UCJC, Samuel Martín-Barbero.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-10-11","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/795"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-10-11","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/nocontext/dataset_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/dataset/update_subject/nocontext/dataset_10.json.gz deleted file mode 100644 index cf3c3aa7b6412355f6172a0b70a3929ebdcc2735..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6755 zcmV-p8l2@HiwFqS2%ug717u-zVRL14UokK)YIARH0PS7Pj^j4A-p^AI1_qb`TrNqr z{I{#PDwztplSX$ZKyG>vP!eskB8d(qRpR6}aPtiJ33}O!UJQDdt+#oK`y}@qlCor% z>n~F^Qo1=)=-3u19-i;;_k2A3r4!Mdo~Lusnf}t@=_Sv25VOv7PUDPq;g>m+l11Ae zA{MipMZsD=RXXwoPN+culI; zue%*6H!oJ8E|&9zWjRe&ovG`OeaH95!|}wKK&7!H-}!G5eW(7{pNuEdspCbFA9NK#ovTEy{dC}D+lgZGx1 zU8gs2$Nus2C$kqvp5u;>TyM~atUnYq=5x;EF0hf{d+6z3u0X`2@ACDET?5Vb>FcgF z-Mg7CqK9TH=Eh2toPoFk_Xe=pts)bXd@u;)zN|LT;fb4k-|E@VOv=NL}?e$O4M z9PB4Zt+DtPiW8LAb@z+0?^BkAlCMhDEvdYOhI9(0-7F7{bEc34oB*wWX6&oyPaw}S z&sTr#pZwMP6#lmA|tx9GF)<~c2CH~ z@s}s($6n7l1~I;i!eiCX&>~+7soL5c+MYr8GaAQKM&f8j6B;f>mwXN{U{Iz>P=3t{ zc}*602#9_UotA0M*tDU%`JBhxV60kucc=3#s0N%B!3FdbJ3apF%dgK*kKD=ief*wSyYlQyV0-Yp z_u$B zXlM%#7KSEaxG)nXi$e4Bqk!K>_yvun=GYd?U zzzqAY({uaerJX)Cx<#Gy992`>a5{?#Pa_VU7oq{ckWj!sEXyU$*@EA&-bg4O#=Vsk_W|_k*pw4bVe^2DR&t*?=l!P zAxlcW5e0c!1g!Te{VL^`OlG(tM`kqPu`n)*8H=%-uqfWsRY*gqQ(n67O594fm*ZNU z-o4}Ntcn17x+V5v+UdlSfXannX{6hV{xuVY3{A^He^<_a%FQ?Bh1H)!k*emA8z*6j z%~KX}`h*7Lh+J{vTYzruy0r4O%TL1*LQ<(baH#eGeZSBsvP|;DGOrS)6-@8}y_@lK zwpAeh%+t)YB!DaCA@pcFt;%?aL&<0k-5B(U2}q_K#3pri9ITJh@F{iV(2%0mx$;Zf9g(x*A?(r9m$U44fe|Zl*h@Tx?|rF zD*yC%rIjA8{-J~*#8PAmc@*G5NkQFCA!DQ~8T(yEac@b}C}us7=lBYP0>>6Jw4;!* z2uor~=L&)8!9}&B+QW}(QlPo>bP+CLG-EVD20_^|?mrC+iPc}1&!y@kSyYiOG7u8F z+xmzYu}dCS<0Joh`&rHQO3H>N=au9eP0}l$l)st`X^{lgIK05^n9>Zt{pahy{9^Hc zC_bb=xRwv;N8@nhyMv)Op9H};@&ac#@rTho2>oyrbZ+58-j@A8$cMZKwQe-IC$-M& zPacz6r{!syTIYGA&J}9i=*V@CT>qszoO+X~?>{cJ?l~?epKabsB(^=JGOrQJ%C8lr zOs8hQ|0-()w%ZtWmQr?@QWgp+QhLdur^IDHgF0J+S;L!ibRl^N##2aUfoKwl=PVc2 zH`FKuFe896Z98a*L+G5$M7qGmE%~Bgmm(vojo@Bz4F5TU*Wg!;8o#`$^;i!=8zH~D z2jSkP&?OI0DUgjmBI2EP!166UOYK|{WIpOROd zCzOC%3iBOOpdic$5^V*+($8SRwfPq%l)&8na`P6Sqq75Dn?(piFU`nW|CW&@*awIQ z@1fbNP8W=mrA+%x=D0>u6$nKN>urhkx~C95@`4e`&>ORPPl^XnW0aJdG)sz79IQSW1#)9Pznj$hFbXD^*8B~zx zpc;4sOd=T(psK8AcW55am*XEf!z0hJI&Ft9$NC{YWW%w3h!*%EaJ#9^ehAwmTR(*L zLp-1#V&Y6j*ZLvm4q4#!VNKY4xaEwH^cx>`1hUZ;E_-G-?$v@k{PV}LQd665!OrC0xnhBPhnvOgR z#-!S@GOyj{d$N;>|Ng7W^-T}5)U#Edj>7`s0e?fPv%Zgi$M-Q;s|b5ArqK=FjM+o0 z%~HV==tNT9=&K3jAv$hGrPLKX@O}H7;oUZFjazK8`ci`0GS%?&n}n@dB|Q7`3cXwZX0&_)x{<%n)6y2(?jf7MNAwpQf zFf^Han$I0vTV3PX3E=(0yf-89=3UC@fu6r~^(-m+lQJqLEHRB2jiFO#g&Zm6_%X%{ zuR!aa;$yx$zL%v%T!9bEEkes0vC2WE++XMfN;YqEVWdRA(TYhURkC>l&1()C2q7c- zhOjtRc8Fzxy;rHxHOLt7&0AH8oWc*FCv6-Q6>O@rc>}3(BPYoKu1LGkv{UR4v}lmv z;W4xugb(Nj8{(#fXUX3!af4OZJoJ``h1VX4`q;6xj>uLxB5}jH&5lSrCR#_Nbwoa- zBhqvI>l~5p$eW-CaqLbe!NiZ;$R7^JzDwt$s2}?NEsn@rv$xO@>Gy_y|DKM>(eUv+ zkz0A1_e8okdLrG)v_H0DeN zK!DY(4QJUb5N9~+mLlf%@MtTV1T(&?4BCd8W?4`ylfuGtE0f~Mu6I^1O_sGu7rIG9 zk*rwG4cVgB$hriwlFtj}vyRD3WH$h7$rNogiMx|2c3|EmQ^3*{-als(*ccwfB{v@iM#{vJnp3r^yqk#=nyEIs^nE7HCfX#hLAfeDk=u%E>cPonli|zxE2{(6cW!xKgDlg zaJXBiEQ!wSeEqg5#xU=>0vOw_Vl&K`t$xEWE02toH=EE-=QPsx`S~vib8cR?&6W6;J>0!|G-0@?P9x8d7Cq0;u7Y9iX{pp}T_1s2D4_`wM1Wi7~-bn=m zBj;%1zu@lOUYL zF!h7FMO?+)v<|^$JEppa&qC286rZpZ&)ag_L0Jleh35_whVxyg@DVZNJ5e&IFwAR` zvIW(j5yK)YkDiBSMphlOzj>ByNepc?io25-p54qQ$QXto*u1I7U`X#4%AEWSx3B(A zG7-pK(^{+e;>=iOSg1E!|k&Lb%d|$`C7-=2C}`4=I?QsoQ+7?p5RpRiX$k zn2NlPBIhvjCMpO;E#SHiOJT56sZ?C}y{==A?tu&zo`!ry<3vRm$!NeNC=B0?j1Rgl zBFHEgAlcg#w@S2yC3FoKB7=pM0oqHdNf+b0iz>cG8H)ag0+5177loQM5cZQ|qi8A$ z1&QN*E{fu(P*UU^&CjF+aV5649I6b8sbjN^XOCg=*-tCR z%QDpNHb?=c6UIfR>3lrJP3cUlbWJwDfx_UEQp3LGXp$G?({~QM>g?)aJANKSawQ_w zyta2Jz~^8)*+mLj1{&tR%> zI3HrItk3uXsc54k*Ew>AR;O*FqWzmrGK;3AqJ8YxS}IygsAx{Bsc3cxv{W=pMSD;x z+GKn!6>UD4_rtO0gah9RB4-@Y5dLvSA^aGH^Uf_)wA-?`l8QEVhWDkSd5?A>c_mNN zRJ76Lz=h;nDw;QiKd=9|RJ2*4woGO;6Vd1Et?6u1X! zgM2jZ^n6O?_t3AW3lizB(8AFu983kn&IGiU*b*}uZyTWvyV5XqFTtbX7KIj|kVY&L z(soMCLUDdhp74bV+;PeRiZ`2_&rZ&6z1hTK^Z3{dU3buP`(!(*vT(G6F{V~`#H{F! z{#XODM6fnueRm>Q-3Geb=r!~{Os_RL%T^4h=8Am)IjiMUxs$XeW)At0BN5`~5B1VxF6hq)BKE%y~N*RVGR-A#5vWlyQnV;NQd&mM%f^NP3GZC`| zYU`QgmlQ7~Lf>Y@aQBYudyM|?fBtv2c_+g{(-~%>(iWb>QitZg|$(54bi69kb`W(q_fDMgkEinfGKiYg`$6po^lM59n7ibH7lC59dn z*7t-o{|>=$eL6KRu0!!n0QE#O^a*d|2b=+w87(q(k$E72!A;~V&92}^4%Ji*QijF39IH_bC zdkU;J;VBw`=7pI(WYoxsk+=%MrDC*&Jb<@V(W6Wpw@Rtqy2g#wuj=GOZLlQ`5qcaZ zb1kDujr*~ie+dLp*(el>)Gi+BR8VE;d!d)s6th%a)<sy^#Hqb|jreyCfBv$v8B9PB8AvQ@*b! z3{r#=0afXZ(#XYz^7?m+kBq?x=uau&JK)L5ByfiOz zvo*7W*e6^ho41k1B_{t6*-Gu}xgI9|5(YKw4%hpoKf~LpXgtm%xM0>o$| zP_AwXg)`m(17|O3rMNUSHX*X0p~K5q$W$;WfjtjC1w%wC*EZzW9JiKB5?DrbaZH8(deu=8 znyxlM)O3VF&I_RhR&*y63>ke(N)i_023Jz9@u~(}O`y6}<3;|St{ZZEu%w?g2`%K( z)X%h^Z?LKWT>y=6xp`-9#)CF26eDR)q1I)MVv^(ijfVB4$XIisV~m+Xu|`UVuR1#g zD+&)-a=}%1fYIDvbqK}xU2fiLf|nEvNew+sk)~3*4HQX3n{D0zT-lXAw2NTS02)=Qn*>xWfl*I&LmFw-sNe^T|F%}| zf(p^iJA88lsPME{GmM&LU3?E7N65E2walT95>3k-`f#ze%%PSrhsLdD4%s2lGKVa4 z=pmUyBj34}IW!Lg)~8P28#}{cKa3(4%%gr7OgyLWjxeyyjm)9jvbT>pYe6T9sSameve*gi&7vcw@v9I}y#{)B_Xu4lNurDU;a5)0cbS_1{D zt32Xqk!Y4>2a6R8MmxZ`?o6>orP+~G2jNYSS)6^r9C_ab{MA#ZK#YtW|n96fIKtLaop>8X3=Qk_Tl4j6tHj@^;zJ# z&LCne7_h-)=yq=AncbSbeLORa2sOUneRZSW$a5bvLe;nx_tgythTOYLG;;<=uJ_Uz zP5sH#8(E^6C7RulX!cUItZL;?W@?>mb|v<)vo3}P{iGI%mH=VnCA9?$LzA^2Gzr4X zD$0fhq8(mnwK=UeXI|j>`jC-Rx+XYNw65nr7q)kIJ)gyd9LBvC-DMkeA`w}x<4Rz} z2@?~69S8_WViW+~au}W=5Xuo#kAA~SXUUyht5mthGUc~{NoEqrt+NFa*nkKR>o>u5 zp<{Jw{U+9LV*Mr`f!_qT$J*^Tu|2Z&n^?ce1Nu$8Yj5_5CQj6+?tD0)kvAL!J_}eh zrhPx~#!i25+sz)gW^bk6Wa8fMeEz668jK#%Z(`(W+HW$N9Q2#u0sY}i*PjmhQ`fP6 z6YDpz2lW4hqeOi>5)2==EzSxFZPo5b+OxPdx)~(!6Ff%#PIGGh(>`pgC&w)fkgVqqJ^W`cA-GX$g)VVyRvB51Nj!E zyHX*Z(*$98Qt*s?O=Y-wBSRMB!F=Vht}PV_#p8?W>|(H!p1o|uWW z&Vt>wn%G!Pm!O*f57~2l;u%lMKBF?GiK5GDL&BR=^vx_ZSrw{_IEfRwkm5c>iGW@R zd`S*#Hat-Ns`Ez-7VlcA&e$*U_9G_M=&sKx>-%_6afkEk%vgu*+>ESEUVXS94^?BT z6^l7mF;<lg(5phuEO?76{gUNyGj1TgX_&w2T zDPm4J58y-fn+K^>F*Y1T+!zV19GI1ni$v*k5@==28IQi2Xptx*Q>9l>oU+DzA+D0y z1aq+vRB;%KbW*i_LT#HePhR6CxxW3jbpY+ASpw`5sayz?@?!T(=3;IvT^@fv)JM6o>7rMhyr!(k}hTU*5823Bwv@g2dz#Re+O3Rsyd2|~p z_`|_9i~kY0qM}FyAegCE3#j6ogRjI2XiAj=?|Szulp|tMebDd*i4mP@T$bP?Ro94`^fE0yz!(raz^7J zmJp92w{07nQ(V@S?3km6Q7jmUS z;)X67xZb2YM#2^%<;eAWpsOa~G@x>t&d$ypB0vXo3Egy*o*mymxqaMmT%_)c zFgPYH=2<#dnlxkzV$0S3V-?DHu3GFa*OH_BZ=C;#Le9V2xQLB+q56HjxV4R6{(pN+wI52Tx34tS;l7@jf3&pX!;b)SvJdr z5nI|XD%sv{_4Qk{2w9wh5u_1fGXF~$iX`@8^q+#d$v}o2EBiX*QLeCc^1z6kDIIQ* zSfyUVpQ5?vFsI1!ZL67qt~LYV$SGx~RVvPZIR7ujWSokb2D{KmaWA9@qY)-nWKC@> z-e;M?Y1go%9F>Ko83X#-H@sHVm(8> zFB@4JfnP`98v4HEaT?OvgH}lOwMk78P8gqQ8Nk@eG-sd-=TBjTi=|BYQbf>NDj0M_ z1rkO)<1AaSK*e8zsi(Es66Wdir%NFXixm?A^bICZv4AO7#qb>~B%2XL+4)bXZ=h$1 z<}*|`&_5BM{{Vl+@V}e}Oiy66(-&+)Whtg6FxjN=@7Ji8YLS3!7ofb6VxX5uCoCq; zpTcwm<)iG0!)^f`(A*5WV3*vqhL!;>#$qf^ zb-*e}VuiNpoOen?d+hwj7F8;NUo9fk2&gwupqGrVjkO76#%=}@hGBxnbtW{ca{-n_ zpzl}-at&6%2Dm5}a;A*)dHjek{PQ0dSj!T-0DQu%d`+705fW0ll$0SvrY=>4KP~y6 zs0A<)3&Lo{Mi>WL9?J!{japHnc(6W+qJe}KDujf8Ko=4Y6|n%gqIbx1nYT@d;|g84uA+?fdf>Kpj)V2+?V;+ z0My|8DexCHV8P=oMHM1KfGnUMRAJ$;`x zCz&5fGZ*3Su?esijp4n9;k8>!9zdVcVsluSTG(U_+Ai#~yoRS&UxOYL3aRV7gw=@- zGSe{f8jKVRpX(V+xuIPa*sZHknGW(r0`8Mj@i@(Yk#EHQ4Rd~~rJtp=mA9K)DrmR2 zwyFS{)wJWp96nlS7zoB_Ih&_drZ{in8|%Hw9SToG3>KE$2rwlqvRJ0Dkv$IPV9;l( z4Ry<~?Trs`ltFm`+w$mvC}oH(%Gl(C4(g61%u1%DZH-O2T4l*5~!!Z~7JdB(O>T!hAzC4;1yR#{?G zQl(HQ|M>0Sf76KS<$x32LC3ocobdc`JoLGkdgIU!JE7n0xLx1#I=(;Z`aypOII*+% zo`4hMw%ciU`Uo|2-ARAojK{s}fD`?Z(;IYO9yoCsJ+B#@==Z$-MR39r*L~!65uE6B zUoANCAuvCL9aD?lzB~AJN1U{Y%~G6vS;6aS@o!Uy+$HndW${ey4FVitJVnSDqHAX ztA$5GJ{A(U@^?5`pmeP>RZ7)ZQJ`$Ef35Z&NFPcA+wPly>#D8dXVp-ivutOur$CX2 zXB$o5if6bHd&MZx@6dCSu9%v#NKQoqGVH^8pbiz?K$mHZ_-)((efuH3tq@UNj(__f z>(Q*nC4)JSfh}TMMFJD&0#*uhN{-|b*66I*={MNXUTMPBv7_78VNRtd>=7sfV#6>D zQ&b344in%yTK?a^|MzzRE8IFO2J2|pJ%lUD6I6P}5|v~T&5mr&4fBPF8SK=~P@Td| zU_q)@&SG0B(8^QHX>xqZR_;rwqE5 z&N)()!6)EHL_=zhXvCB_2c{C{e@3YiVL1bRL%7Rf_sM|b!WpHCzz_+G%Y{J+aM+ox zo(QT9cGiJ_oFVoSE91<-s@K z*74}b%GH{jLsigAsk!~fpF|z$?Gg4nAF2p`+!e6!521f2n7<;qKW5c=nDq{v%@s>v zH;KBWjk{_@>_%#;d$`R+ekY=9qa8Wqn!5TRYCS9}wX0|6V{IE8OVAF$2{i&cb_ z_2%v}u)aklp6hgqT6~UmJM5lXQw!J$rg$zZ;89(UmMK$MQHJpU+FFy6uY^rwLn!YX z3K3{D62P(g%*Ko$a4*N4Q#O~gIRgtQA{vTo3f=-Gkinb*QDB=2Tpam!CUNcqTP0x7 z5hiKP9PCh&)c%2x9cp`;<)(oKs0xA%Dc354cCEZ3+Ox8IFwoH%oR0^vDs`34)a1`{{0Ehgg-@AiOl(LJY-jog1= zz?s~W-FF7;H-#@YRVip`jW#5uGjRG<7W~*|`Db#iQ>BAG?4?h%S9};N~k}Mpwy}B2tM35J4<$Mqi-PndS?N9_4FI z>n-M5a08wFI+NP+3x*_5Q_P}b#IV{dhk>?&SSWpaW2Pj0dig-dD^JvenEAGS+#^+; zuMPJKT(U}m+gTU_f-aT^7TyDP*b6&xT9e%2J@${R-|e(I-uNgNK+HiRD+4(SU;{gg z4sq2E#R9pL%w@2~&cm5N_k4>jxRxk&gikWlVj|3f-0zM{EF>i-BM>dTU>8uDFLgRc z?E>`^M4E4<6e<|J`$pe@+E|U? z7Ja%Pa+n??fulghq9*@B%q#ZGcYkI+2V)S=jzevoJ3uR#?>56Uys+n!I;XIkL|k16 z$^vSk>5m0yxLzF-td2Jft?m-|)Mf-fVR70Bm}4Ic7s!-S;X3MP^@zYdLdN10EiOo) zhM-pr+rsTaGFFMK^lH7P(*QZJmnbk}#aV^I|$+F9ze6gFI@JbYk~iMAz9P#>C7f|eU_ z?Ty?FY&^>E9=KKs6MtMTs_nvo9|@mXA4y&G)?If|Ii4@cT5Xfk=1dZMgv;bd5Ma2& z0(e8L?6fMHuNq(;fW<1FsY1I0)~0s4D?iLoSPsY@6@7SV@~><6s87hDb=C-iDI31O zX1-7PqG9}-V0?8k_G;*S&@r$tNE@vE{lMBg@4`LWW@Xf1?eCxy8 z+bHX+AJVv~f~@z{Y8w+6F%U0$^SJ)VE@%L{%J5y)n2uPWn?bR|kbn|oSli%jYI;t#cw3j~ zmFCE4JUlNEPjnlv*{{J9YbQAs^b?1BIbQQMIn67@VXTJFd;Jh;gIc|2)XEp=S85Q$ z2C99J7=K-;wstZ4idU&CaQE*TboZS{cge@U=jd*;PHoWLcU4q7(OqxQy%gQ;^!cbe z>i7fR4gKL@>htk1=!AjS8%)RDT_GHI7T+6mw+fj!z?;jRzIQEj*K@qiD+=LQr{^`J zyWP&%y@>A8i}PNu2;umOUrD|z@x3pZElN-~-^Ue*Zf9J2^dR{Mh?5JzT|)-CP`aPcIJSyW-|$toLX?N3;5# z@8=qG)a>WpRZ;ES&vnM*OZRgyMX(Kzf32V&^^1D}V%uKnDu;(M~6 z8@FBf?|@#S8c#Z7*gsu!J2&WhFZE%pZREV}v%224-JGS(?&jP{&wItYIT4E|7_{Y1 zcK(w&sGg9@_9SvwO9DeBqxHtA(OjCB_uJH}%hCoyNI zJjJlG^(V55WawqE@mf}1^x(xF%X*+fP{9mYdIJw{)L+Zu&%WP`Pg7<;c(H&uAFBm6 z^$D2fxYktpyK0GY9=uo>oO@~sl6#19t4H=BsjatyIS=+d2F@Ei#(fuz!J;{VxIc@7 zwb#Em*yj9j6;5jS^gEZ$QT+W7PYo`O4?X8^e}xQV`UmnHwHQ7lq->ab z3{-*vkg0DQb@(}BbeIxJicXMRnFPN3^s_;9_ze?uOa0J+_tUm8?|aqW4;xqF>gJ* z_4|g4dwtgO3dfkH{43G8yx#|xH@QCc+~wUE_r~S@u8L}>%X`qdG#Er@G!^}R*X<8R zt{3+BaL^MW4+g#ny4{}G>GIxHe1BZto52@G*YbIHo#A-=N`pa+-bgTr?(Sd^y;tk> z4mrD_aGQ7SaN;*NOTM5t`~~AST+avo;z1~`=koqkCCL@zYGBqE^mPBqC=|OK-fycf z`b|ZlxRq&@G-25HVzzF=uwmP2Ig4N2H?HZ4b+xcyj>ve*vSMdcx5%5sh;Nfw$ z@P)n*R-0l&wDG}vK_Sy+9AIPw3@iN+EJSVSbb3CVp0kHd)N`sUpA*NLKr#v=hpQM@ zbv%kzjAt{tv=5bkOu-Or#AS>j{5TJIoQJ5cA_vpk^mUS340wUnaJ^cKTTn!fplHId zAg7oZw0(oxPvpOdaaL@zR7+ch2$47&(B4f~ZioOC8je*~L zRmgF6xE>Osx)!VSMzjcN_rW#{%TcLjty0Hu$CA!7n^-U#I(~_^y7suo!q<&Dcc-tVe+*uFsaFtqtdH9Zt#QrWZT^IL)nJ>~ak(;_pY;YMGVy)dr z6mjPh8{*{FWAsP>M>T!>_~t3nuY3&lfabIop$4ZA8QRxWwgk(mLQ>_CKI#E%Dg(A= zU2N2PX$rd-=3~9hga#Tf7~#v}wOd1Z>$4N;@`;hWi5Brj@Wf%`a(?$NeKdI2yN`E~ z*K^PDuEzK@c-Olss-1Y(uo$1TOTfE+B|>z~dmb+gt~K${uZKiE%=A(&>8-^E&3Nl) z(mEEk<_&Dbw-@5u3Pu$}knt%@o^>zp)JCw5>+zF=D?yMAVD&c2{_21gVhR?eq4>5J zLXy{_ha1dDh}0BT*^ojmb_;75LD*6~7p6rXmr-C@5UCYcYFUY)KDpr$v4s zgJacm&@ruMA{O?6N}dv4)n*p4dP2VQqSp@lM4e%LYK!8*CLN?tum}hv4)VICwgKmv zDaLCBpiy{a>_BMYpu#1;9v;1nwO+IUil_ouo(L<7s^$xpAPOluG&3A=LG0lrCc$AR zRMrM=mKdrT<84#nif?U;zghCsx@Vgmo<4$>fBEju_w9?!HL#pL#1cm>hL9A#h@Q$o zCg@E^)Cjr*8*YoYah8gx%|tTCNY?hzbScglvU$Y@YylhVIbrWG4WZJiyf_1~91|pG zo&1Q&NQ=c-?6nPQimeq-#$&v4i(25`k{v>ow(_GAW`OQMO%#GXpD_pov zZ@%q!SY?|n4rHk5L}=V?S(iUb6(Nonlqf#-33800V;R7`&t)e(eOb=P5D&qa~7 zSq-wL$o7Q*uolnjv%^ux@qTl}D$T^Yakv!}5zn@%8}_q1%2`=w1m(qf3G?S-U+XOo zFfMxtrUI0V9TIb?nj?%x9!=W>=_D2znS#9lzG8PvNm~JF8Gn@rGEXzJhUTC;#Ub4l zyD+8-NDNqrz+OBP6gM5T0)MnI^m|yhs1Cx781mdu9D`X5#$ij0@uGTyu!^4MDrzab zymM`Kv3oFS3n;jP!F(8l?ni-|E6QZDg*YOJxDe9C$mC|OGLR7%;z;G7VBTd!+GrBV zZN4e(Sqq2BnTR5auV7ZeT#?-HcLM9US`-Fkdk*Rm2f>_x?WMB|*c7cf5a66z_3_0@ z5xgwe9Bg3|@BB?6Sfj>?+#pyDg0+tbR%f*D2v%c!8U*WI71b^TYcL*+E<><-BjJw* zes?4Wq7zL0anGN+yf+whN4-(txjPW7oyGST!MfHfWnR}AcrW*rvaR&Ib_B~C>_V_Q zulSj=YXevNfLGnjxW-6m_jr2#G^#fQDg4>@(Iyy2`CkyP`Va%xQRuiUUDTrN{Mnzb z4qI(7@QHw3EVOo)P9}eb^?uqb#O$^wiNyQR%7?;$8@G`whRshLX2+S_&5f{UloWNjP}l;IK&L%z)&v2q#N9Td8!pCVf{=s_uYG_w zi3nyj1gv3q71s*F~POX`m5-r*c-bC>J{v&dg~s9JxSeEANmmLM_8K=zDB= zU*0#}gmLp4a)>+-$K0oXNiVI@UK02QyJrMlcz7d$T*B})QOxnPt`v{YIgMEid8{^Z!r0=SzcEO}Ww|8kQrylqGAPmQY zzCRj{r#=rx13nD9{ zL9jd07F{&1`F)rV>iEj3ouxVY9Kb~KMwWXf%dId=Ayv(40%aU#p5^(EctGE<&!gnSO5_Vz@2ZbAMPf4eXsi?Ee9U`?Xtd`~U#n?eH=H diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/otherresearchproduct/update_zenodocommunity/publication b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/otherresearchproduct/update_zenodocommunity/publication new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/otherresearchproduct/update_zenodocommunity/software b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/otherresearchproduct/update_zenodocommunity/software new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/dataset b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/dataset new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/datasource b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/datasource new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/otherresearchproduct b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/otherresearchproduct new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/publication b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/publication new file mode 100644 index 000000000..12b385b22 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/publication @@ -0,0 +1,10 @@ +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055113942,"id":"50|ec_fp7health::000085c89f4b96dc2269bd37edb35306","originalId":["ec_fp7health::000085c89f4b96dc2269bd37edb35306"],"collectedfrom":[{"key":"openaire____::ec_fp7_health","value":"HEALTH FP7 Publications Database","dataInfo":null}],"pid":[{"value":"10.1016/j.fgb.2012.05.007","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofcollection":"2016-02-11T18:10:02.893Z","dateoftransformation":"2016-03-08T16:42:14.571Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2016-02-11T18:10:02.893Z","altered":true,"baseURL":"file:///var/lib/dnet/health_crossref","identifier":"","datestamp":"","metadataNamespace":""}},"author":[{"fullname":"Lewis, Leanne E.","name":"Leanne E.","surname":"Lewis","rank":1,"pid":null,"affiliation":null},{"fullname":"Bain, Judith M.","name":"Judith M.","surname":"Bain","rank":2,"pid":null,"affiliation":null},{"fullname":"Lowes, Christina","name":"Christina","surname":"Lowes","rank":3,"pid":null,"affiliation":null},{"fullname":"Gow, Neil A.R.","name":"Neil A. R.","surname":"Gow","rank":4,"pid":null,"affiliation":null},{"fullname":"Erwig, Lars-Peter","name":"Lars-Peter","surname":"Erwig","rank":5,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Candida albicans infection inhibits macrophage cell division and proliferation","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[],"dateofacceptance":{"value":"2012-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":{"value":"Academic Press","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"embargoenddate":null,"source":[{"value":"Fungal Genetics and Biology; Vol 49","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by/3.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0001","classname":"peerReviewed","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"openaire____::1256f046-bf1f-4afc-8b47-d0b147148b18","value":"Unknown Repository","dataInfo":null},"url":["http://dx.doi.org/10.1016/j.fgb.2012.05.007"],"distributionlocation":"","collectedfrom":{"key":"10|doajarticles::8cec81178926caaca531afbd8eb5d64c","value":"HEALTH FP7 Publications Database","dataInfo":null},"dateofacceptance":{"value":"2012-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"journal":{"name":"Fungal Genetics and Biology","issnPrinted":"1087-1845","issnOnline":"","issnLinking":"","ep":"680","iss":"9","sp":"679","vol":"49","edition":"","conferenceplace":null,"conferencedate":null,"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055114128,"id":"50|ec_fp7health::000b9e61f83f5a4b0c35777b7bccdf38","originalId":["ec_fp7health::000b9e61f83f5a4b0c35777b7bccdf38"],"collectedfrom":[{"key":"openaire____::ec_fp7_health","value":"HEALTH FP7 Publications Database","dataInfo":null}],"pid":[{"value":"10.1096/fj.09-145573","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofcollection":"2016-02-11T18:10:11.811Z","dateoftransformation":"2016-03-08T16:42:14.579Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2016-02-11T18:10:11.811Z","altered":true,"baseURL":"file:///var/lib/dnet/health_crossref","identifier":"","datestamp":"","metadataNamespace":""}},"author":[{"fullname":"Tamassia, N.","name":"N.","surname":"Tamassia","rank":1,"pid":null,"affiliation":null},{"fullname":"Castellucci, M.","name":"M.","surname":"Castellucci","rank":2,"pid":null,"affiliation":null},{"fullname":"Rossato, M.","name":"M.","surname":"Rossato","rank":3,"pid":null,"affiliation":null},{"fullname":"Gasperini, S.","name":"S.","surname":"Gasperini","rank":4,"pid":null,"affiliation":null},{"fullname":"Bosisio, D.","name":"D.","surname":"Bosisio","rank":5,"pid":null,"affiliation":null},{"fullname":"Giacomelli, M.","name":"M.","surname":"Giacomelli","rank":6,"pid":null,"affiliation":null},{"fullname":"Badolato, R.","name":"R.","surname":"Badolato","rank":7,"pid":null,"affiliation":null},{"fullname":"Cassatella, M. A.","name":"M. A.","surname":"Cassatella","rank":8,"pid":null,"affiliation":null},{"fullname":"Bazzoni, F.","name":"F.","surname":"Bazzoni","rank":9,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Uncovering an IL-10-dependent NF-kappa B recruitment to the IL-1ra promoter that is impaired in STAT3 functionally defective patients","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[],"dateofacceptance":{"value":"2010-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":{"value":"FEDERATION AMER SOC EXP BIOL","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"embargoenddate":null,"source":[{"value":"The FASEB Journal; Vol 24","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0001","classname":"peerReviewed","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"10|doajarticles::8cec81178926caaca531afbd8eb5d64c","value":"Unknown Repository","dataInfo":null},"url":["http://dx.doi.org/10.1096/fj.09-145573"],"distributionlocation":"","collectedfrom":{"key":"openaire____::ec_fp7_health","value":"HEALTH FP7 Publications Database","dataInfo":null},"dateofacceptance":{"value":"2010-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"journal":{"name":"The FASEB Journal","issnPrinted":"1530-6860","issnOnline":"","issnLinking":"","ep":"1375","iss":"5","sp":"1365","vol":"24","edition":"","conferenceplace":null,"conferencedate":null,"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055114168,"id":"50|ec_fp7health::000c8195edd542e4e64ebb32172cbf89","originalId":["ec_fp7health::000c8195edd542e4e64ebb32172cbf89"],"collectedfrom":[{"key":"openaire____::ec_fp7_health","value":"HEALTH FP7 Publications Database","dataInfo":null}],"pid":[{"value":"10.1097/FJC.0b013e31828780eb","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofcollection":"2016-02-11T18:09:48.436Z","dateoftransformation":"2016-03-08T16:42:14.58Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2016-02-11T18:09:48.436Z","altered":true,"baseURL":"file:///var/lib/dnet/health_crossref","identifier":"","datestamp":"","metadataNamespace":""}},"author":[{"fullname":"Ford, John","name":"John","surname":"Ford","rank":1,"pid":null,"affiliation":null},{"fullname":"Milnes, James","name":"James","surname":"Milnes","rank":2,"pid":null,"affiliation":null},{"fullname":"Wettwer, Erich","name":"Erich","surname":"Wettwer","rank":3,"pid":null,"affiliation":null},{"fullname":"Christ, Torsten","name":"Torsten","surname":"Christ","rank":4,"pid":null,"affiliation":null},{"fullname":"Rogers, Marc","name":"Marc","surname":"Rogers","rank":5,"pid":null,"affiliation":null},{"fullname":"Sutton, Kathy","name":"Kathy","surname":"Sutton","rank":6,"pid":null,"affiliation":null},{"fullname":"Madge, David","name":"David","surname":"Madge","rank":7,"pid":null,"affiliation":null},{"fullname":"Virag, Laszlo","name":"Laszlo","surname":"Virag","rank":8,"pid":null,"affiliation":null},{"fullname":"Jost, Norbert","name":"Norbert","surname":"Jost","rank":9,"pid":null,"affiliation":null},{"fullname":"Horvath, Zoltan","name":"Zoltan","surname":"Horvath","rank":10,"pid":null,"affiliation":null},{"fullname":"Matschke, Klaus","name":"Klaus","surname":"Matschke","rank":11,"pid":null,"affiliation":null},{"fullname":"Varro, Andras","name":"Andras","surname":"Varro","rank":12,"pid":null,"affiliation":null},{"fullname":"Ravens, Ursula","name":"Ursula","surname":"Ravens","rank":13,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Human Electrophysiological and Pharmacological Properties of XEN-D0101: A Novel Atrial-Selective Kv1.5/I-Kur Inhibitor","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[],"dateofacceptance":{"value":"2013-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":{"value":"LIPPINCOTT WILLIAMS & WILKINS","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"embargoenddate":null,"source":[{"value":"Journal of Cardiovascular Pharmacology; Vol 61","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0001","classname":"peerReviewed","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"10|opendoar____::96da2f590cd7246bbde0051047b0d6f7","value":"Unknown Repository","dataInfo":null},"url":["http://dx.doi.org/10.1097/FJC.0b013e31828780eb"],"distributionlocation":"","collectedfrom":{"key":"10|opendoar____::1a551829d50f1400b0dab21fdd969c04","value":"HEALTH FP7 Publications Database","dataInfo":null},"dateofacceptance":{"value":"2013-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"journal":{"name":"Journal of Cardiovascular Pharmacology","issnPrinted":"0160-2446","issnOnline":"","issnLinking":"","ep":"415","iss":"5","sp":"408","vol":"61","edition":"","conferenceplace":null,"conferencedate":null,"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055114204,"id":"50|ec_fp7health::0010eb63e181e3e91b8b6dc6b3e1c798","originalId":["ec_fp7health::0010eb63e181e3e91b8b6dc6b3e1c798"],"collectedfrom":[{"key":"openaire____::ec_fp7_health","value":"HEALTH FP7 Publications Database","dataInfo":null}],"pid":[{"value":"10.1016/j.ajpath.2013.06.019","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofcollection":"2016-02-11T18:10:16.68Z","dateoftransformation":"2016-03-08T16:42:14.582Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2016-02-11T18:10:16.68Z","altered":true,"baseURL":"file:///var/lib/dnet/health_crossref","identifier":"","datestamp":"","metadataNamespace":""}},"author":[{"fullname":"Pizzolla, Angela","name":"Angela","surname":"Pizzolla","rank":1,"pid":null,"affiliation":null},{"fullname":"Wing, Kajsa","name":"Kajsa","surname":"Wing","rank":2,"pid":null,"affiliation":null},{"fullname":"Holmdahl, Rikard","name":"Rikard","surname":"Holmdahl","rank":3,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"A Glucose-6-Phosphate Isomerase Peptide Induces T and B Cell-Dependent Chronic Arthritis in C57BL/10 Mice Arthritis without Reactive Oxygen Species and Complement","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[],"dateofacceptance":{"value":"2013-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":{"value":"ELSEVIER SCIENCE INC","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"embargoenddate":null,"source":[{"value":"The American Journal of Pathology; Vol 183","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://www.elsevier.com/open-access/userlicense/1.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0001","classname":"peerReviewed","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"10|doajarticles::8cec81178926caaca531afbd8eb5d64c","value":"Unknown Repository","dataInfo":null},"url":["http://dx.doi.org/10.1016/j.ajpath.2013.06.019"],"distributionlocation":"","collectedfrom":{"key":"10|opendoar____::1a551829d50f1400b0dab21fdd969c04","value":"HEALTH FP7 Publications Database","dataInfo":null},"dateofacceptance":{"value":"2013-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"journal":{"name":"The American Journal of Pathology","issnPrinted":"0002-9440","issnOnline":"","issnLinking":"","ep":"1155","iss":"4","sp":"1144","vol":"183","edition":"","conferenceplace":null,"conferencedate":null,"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055114228,"id":"50|ec_fp7health::00110d3f9d05812aa683d5117964bdcd","originalId":["ec_fp7health::00110d3f9d05812aa683d5117964bdcd"],"collectedfrom":[{"key":"openaire____::ec_fp7_health","value":"HEALTH FP7 Publications Database","dataInfo":null}],"pid":[{"value":"10.1128/MCB.00231-12","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofcollection":"2016-02-11T18:10:21.541Z","dateoftransformation":"2016-03-08T16:42:14.583Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2016-02-11T18:10:21.541Z","altered":true,"baseURL":"file:///var/lib/dnet/health_crossref","identifier":"","datestamp":"","metadataNamespace":""}},"author":[{"fullname":"Gronroos, E.","name":"E.","surname":"Gronroos","rank":1,"pid":null,"affiliation":null},{"fullname":"Kingston, I. J.","name":"I. J.","surname":"Kingston","rank":2,"pid":null,"affiliation":null},{"fullname":"Ramachandran, A.","name":"A.","surname":"Ramachandran","rank":3,"pid":null,"affiliation":null},{"fullname":"Randall, R. A.","name":"R. A.","surname":"Randall","rank":4,"pid":null,"affiliation":null},{"fullname":"Vizan, P.","name":"P.","surname":"Vizan","rank":5,"pid":null,"affiliation":null},{"fullname":"Hill, C. S.","name":"C. S.","surname":"Hill","rank":6,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Transforming Growth Factor beta Inhibits Bone Morphogenetic Protein-Induced Transcription through Novel Phosphorylated Smad1/5-Smad3 Complexes","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[],"dateofacceptance":{"value":"2012-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":{"value":"AMER SOC MICROBIOLOGY","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"embargoenddate":null,"source":[{"value":"Molecular and Cellular Biology; Vol 32","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0001","classname":"peerReviewed","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"openaire____::1256f046-bf1f-4afc-8b47-d0b147148b18","value":"Unknown Repository","dataInfo":null},"url":["http://dx.doi.org/10.1128/MCB.00231-12"],"distributionlocation":"","collectedfrom":{"key":"openaire____::ec_fp7_health","value":"HEALTH FP7 Publications Database","dataInfo":null},"dateofacceptance":{"value":"2012-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"journal":{"name":"Molecular and Cellular Biology","issnPrinted":"0270-7306","issnOnline":"","issnLinking":"","ep":"2916","iss":"14","sp":"2904","vol":"32","edition":"","conferenceplace":null,"conferencedate":null,"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055114253,"id":"50|ec_fp7health::0013c229505d753b29370b1029f196d3","originalId":["ec_fp7health::0013c229505d753b29370b1029f196d3"],"collectedfrom":[{"key":"openaire____::ec_fp7_health","value":"HEALTH FP7 Publications Database","dataInfo":null}],"pid":[{"value":"10.1111/j.1469-0691.2011.03696.x","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofcollection":"2016-02-11T18:10:16.003Z","dateoftransformation":"2016-03-08T16:42:14.586Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2016-02-11T18:10:16.003Z","altered":true,"baseURL":"file:///var/lib/dnet/health_crossref","identifier":"","datestamp":"","metadataNamespace":""}},"author":[{"fullname":"Espinal, P.","name":"P.","surname":"Espinal","rank":1,"pid":null,"affiliation":null},{"fullname":"Seifert, H.","name":"H.","surname":"Seifert","rank":2,"pid":null,"affiliation":null},{"fullname":"Dijkshoorn, L.","name":"L.","surname":"Dijkshoorn","rank":3,"pid":null,"affiliation":null},{"fullname":"Vila, J.","name":"J.","surname":"Vila","rank":4,"pid":null,"affiliation":null},{"fullname":"Roca, I.","name":"I.","surname":"Roca","rank":5,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Rapid and accurate identification of genomic species from the Acinetobacter baumannii (Ab) group by MALDI-TOF MS.","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[],"dateofacceptance":{"value":"2012-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":{"value":"Blackwell Publishing","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"embargoenddate":null,"source":[{"value":"Clinical Microbiology and Infection; Vol 18","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://www.elsevier.com/open-access/userlicense/1.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0001","classname":"peerReviewed","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"openaire____::1256f046-bf1f-4afc-8b47-d0b147148b18","value":"Unknown Repository","dataInfo":null},"url":["http://dx.doi.org/10.1111/j.1469-0691.2011.03696.x"],"distributionlocation":"","collectedfrom":{"key":"openaire____::ec_fp7_health","value":"HEALTH FP7 Publications Database","dataInfo":null},"dateofacceptance":{"value":"2012-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"journal":{"name":"Clinical Microbiology and Infection","issnPrinted":"1198-743X","issnOnline":"","issnLinking":"","ep":"1103","iss":"11","sp":"1097","vol":"18","edition":"","conferenceplace":null,"conferencedate":null,"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055114274,"id":"50|ec_fp7health::00161e52ec870cc48b5a0a40c5d88e58","originalId":["ec_fp7health::00161e52ec870cc48b5a0a40c5d88e58"],"collectedfrom":[{"key":"openaire____::ec_fp7_health","value":"HEALTH FP7 Publications Database","dataInfo":null}],"pid":[{"value":"10.1021/ac200237j","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofcollection":"2016-02-11T18:09:49.972Z","dateoftransformation":"2016-03-08T16:42:14.588Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2016-02-11T18:09:49.972Z","altered":true,"baseURL":"file:///var/lib/dnet/health_crossref","identifier":"","datestamp":"","metadataNamespace":""}},"author":[{"fullname":"Xu, Shoujiang","name":"Shoujiang","surname":"Xu","rank":1,"pid":null,"affiliation":null},{"fullname":"Liu, Yang","name":"Yang","surname":"Liu","rank":2,"pid":null,"affiliation":null},{"fullname":"Wang, Taihong","name":"Taihong","surname":"Wang","rank":3,"pid":null,"affiliation":null},{"fullname":"Li, Jinghong","name":"Jinghong","surname":"Li","rank":4,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Positive Potential Operation of a Cathodic Electrogenerated Chemiluminescence Immunosensor Based on Luminol and Graphene for Cancer Biomarker Detection","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[],"dateofacceptance":{"value":"2011-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":{"value":"AMER CHEMICAL SOC","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"embargoenddate":null,"source":[{"value":"Analytical Chemistry; Vol 83","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0001","classname":"peerReviewed","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"openaire____::1256f046-bf1f-4afc-8b47-d0b147148b18","value":"Unknown Repository","dataInfo":null},"url":["http://dx.doi.org/10.1021/ac200237j"],"distributionlocation":"","collectedfrom":{"key":"openaire____::ec_fp7_health","value":"HEALTH FP7 Publications Database","dataInfo":null},"dateofacceptance":{"value":"2011-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"journal":{"name":"Analytical Chemistry","issnPrinted":"0003-2700","issnOnline":"","issnLinking":"","ep":"3823","iss":"10","sp":"3817","vol":"83","edition":"","conferenceplace":null,"conferencedate":null,"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055114298,"id":"50|ec_fp7health::001b3ce526a763539e684a5c8bec1223","originalId":["ec_fp7health::001b3ce526a763539e684a5c8bec1223"],"collectedfrom":[{"key":"openaire____::ec_fp7_health","value":"HEALTH FP7 Publications Database","dataInfo":null}],"pid":[{"value":"10.1038/nrmicro3067","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofcollection":"2016-02-11T18:10:14.775Z","dateoftransformation":"2016-03-08T16:42:14.589Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2016-02-11T18:10:14.775Z","altered":true,"baseURL":"file:///var/lib/dnet/health_crossref","identifier":"","datestamp":"","metadataNamespace":""}},"author":[{"fullname":"Chandler, Michael","name":"Michael","surname":"Chandler","rank":1,"pid":null,"affiliation":null},{"fullname":"de la Cruz, Fernando","name":"Fernando","surname":"La Cruz","rank":2,"pid":null,"affiliation":null},{"fullname":"Dyda, Fred","name":"Fred","surname":"Dyda","rank":3,"pid":null,"affiliation":null},{"fullname":"Hickman, Alison B.","name":"Alison B.","surname":"Hickman","rank":4,"pid":null,"affiliation":null},{"fullname":"Moncalian, Gabriel","name":"Gabriel","surname":"Moncalian","rank":5,"pid":null,"affiliation":null},{"fullname":"Ton-Hoang, Bao","name":"Bao","surname":"Ton-Hoang","rank":6,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Breaking and joining single-stranded DNA: the HUH endonuclease superfamily","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[],"dateofacceptance":{"value":"2013-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":{"value":"NATURE PUBLISHING GROUP","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"embargoenddate":null,"source":[{"value":"Nature Reviews Microbiology; Vol 11","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0001","classname":"peerReviewed","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"openaire____::1256f046-bf1f-4afc-8b47-d0b147148b18","value":"Unknown Repository","dataInfo":null},"url":["http://dx.doi.org/10.1038/nrmicro3067"],"distributionlocation":"","collectedfrom":{"key":"openaire____::ec_fp7_health","value":"HEALTH FP7 Publications Database","dataInfo":null},"dateofacceptance":{"value":"2013-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"journal":{"name":"Nature Reviews Microbiology","issnPrinted":"1740-1526","issnOnline":"","issnLinking":"","ep":"538","iss":"8","sp":"525","vol":"11","edition":"","conferenceplace":null,"conferencedate":null,"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055114326,"id":"50|ec_fp7health::001bd73b6a5ada26a6021d608de6c6a4","originalId":["ec_fp7health::001bd73b6a5ada26a6021d608de6c6a4"],"collectedfrom":[{"key":"openaire____::ec_fp7_health","value":"HEALTH FP7 Publications Database","dataInfo":null}],"pid":[{"value":"10.1016/S0140-6736(12)62202-8","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofcollection":"2016-02-11T18:10:29.817Z","dateoftransformation":"2016-03-08T16:42:14.591Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2016-02-11T18:10:29.817Z","altered":true,"baseURL":"file:///var/lib/dnet/health_crossref","identifier":"","datestamp":"","metadataNamespace":""}},"author":[{"fullname":"Hansel, Trevor T","name":"Trevor T.","surname":"Hansel","rank":1,"pid":null,"affiliation":null},{"fullname":"Johnston, Sebastian L","name":"Sebastian L.","surname":"Johnston","rank":2,"pid":null,"affiliation":null},{"fullname":"Openshaw, Peter J","name":"Peter J.","surname":"Openshaw","rank":3,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Microbes and mucosal immune responses in asthma","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[],"dateofacceptance":{"value":"2013-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":{"value":"ELSEVIER SCIENCE INC","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"embargoenddate":null,"source":[{"value":"The Lancet; Vol 381","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0001","classname":"peerReviewed","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"openaire____::1256f046-bf1f-4afc-8b47-d0b147148b18","value":"Unknown Repository","dataInfo":null},"url":["http://dx.doi.org/10.1016/S0140-6736(12)62202-8"],"distributionlocation":"","collectedfrom":{"key":"openaire____::ec_fp7_health","value":"HEALTH FP7 Publications Database","dataInfo":null},"dateofacceptance":{"value":"2013-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"journal":{"name":"The Lancet","issnPrinted":"0140-6736","issnOnline":"","issnLinking":"","ep":"873","iss":"9869","sp":"861","vol":"381","edition":"","conferenceplace":null,"conferencedate":null,"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055114352,"id":"50|ec_fp7health::001d16a111fbca498017c8d39b0068bf","originalId":["ec_fp7health::001d16a111fbca498017c8d39b0068bf"],"collectedfrom":[{"key":"openaire____::ec_fp7_health","value":"HEALTH FP7 Publications Database","dataInfo":null}],"pid":[{"value":"10.1016/j.expneurol.2011.08.012","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofcollection":"2016-02-11T18:09:51.03Z","dateoftransformation":"2016-03-08T16:42:14.592Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2016-02-11T18:09:51.03Z","altered":true,"baseURL":"file:///var/lib/dnet/health_crossref","identifier":"","datestamp":"","metadataNamespace":""}},"author":[{"fullname":"Mazarati, Andréy","name":"Andréy","surname":"Mazarati","rank":1,"pid":null,"affiliation":null},{"fullname":"Maroso, Mattia","name":"Mattia","surname":"Maroso","rank":2,"pid":null,"affiliation":null},{"fullname":"Iori, Valentina","name":"Valentina","surname":"Iori","rank":3,"pid":null,"affiliation":null},{"fullname":"Vezzani, Annamaria","name":"Annamaria","surname":"Vezzani","rank":4,"pid":null,"affiliation":null},{"fullname":"Carli, Mirjana","name":"Mirjana","surname":"Carli","rank":5,"pid":null,"affiliation":null}],"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"High-mobility group box-1 impairs memory in mice through both toll-like receptor 4 and Receptor for Advanced Glycation End Products","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[],"dateofacceptance":{"value":"2011-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[{"value":"Experimental Neurology; Vol 232","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"CLOSED","classname":"Closed Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0001","classname":"peerReviewed","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"openaire____::1256f046-bf1f-4afc-8b47-d0b147148b18","value":"Unknown Repository","dataInfo":null},"url":["http://dx.doi.org/10.1016/j.expneurol.2011.08.012"],"distributionlocation":"","collectedfrom":{"key":"openaire____::ec_fp7_health","value":"HEALTH FP7 Publications Database","dataInfo":null},"dateofacceptance":{"value":"2011-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"journal":{"name":"Experimental Neurology","issnPrinted":"0014-4886","issnOnline":"","issnLinking":"","ep":"148","iss":"2","sp":"143","vol":"232","edition":"","conferenceplace":null,"conferencedate":null,"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/publication_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/publication/update_datasource/publication_10.json.gz deleted file mode 100644 index 99c4015e715ae7540f024ef729623d3e0484d739..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5257 zcmV;46n5($iwFocB%odZ18{X>Y-wX*bZKvHUokK)YIARH0PS7fa@#nTzTc-n>0)c9 zNA8|+J60mwD%(kSPuEroBq0knMQA|Ej+4p7KF&VJ-tI%~0V#@<6w8tlXXp%4 zUDYOZz=NN|1K|6BgD;Jcq@*8@c;nzpgT?nOVSYp#2O|g(5L1N5e| zDKr;N*@Px3nM@l8*m5n+vM@G0Q-?}RO_ugg8hjj0oiQa*IzBki;Gb&+t~WA$&kh4! zw>>{J9B4+vGBg_s!3DcvF^T%{fiDt6ar^nsF^(i_TI%qiaBNmXX+UN*u!zT#^(b7!fSrmN*l~Y++ zP*l7GYi+DyyK~zf-S}-?!+Kk@+M4D-?tjck#72yY<<1T{Tj|j}Wv$y-_%VH)(kHrW z@$1*7GUx5gS$3c_Wj>l)PFC^;^qSq$^cKd~*ge47fu^@z&-i6a`?6B0c@nx@t`&$c6q!3IIu;M?2D*xfq&0+( zUsU?4npWz{axAo2&u9Zv7c`kgsq{(9-e6if*OgCy`Ipu2Jb#~bay;TU>6f)V! zAOU$1sDfUkqOjN}dUXQ62j%LQTVUyj)!NFg?Xi@8To$NSbwxzk!!Gt9pzil1PQz@F zmEUSu9fbc}%ne8Y)lQ4?x-cF&wQFe@{%fda-?rJCxr#z;PT^P@l7y>k&=>$T%HvUcJ&=i{C}&YiU@nYHzt*5l6EAbL|^O zR@tIZ@1bDe$HUB^T&IOy)0+Qs5l1YZH?e#ku=oxNlP6G#!q~2s{|0}{9!j!ANB()w zp%{&9+@>L`8uOwTqYSfI3?}pB>Qq5l(sq{g^*_F-w)!zFfr)i@154n0)W##%7+J*h zwZO0($MGFM2*Q!!KF<oL%h;dK3ytWL;2qsXF=Nl9A1Y7uwt%YxMaum;HZI z(Q~p|p-g6>!BgCf;$~je7`x8RypJ=04Aw%hoA!VFE0N9g>dbw{RCf)0_O zftay$B9l{&(lO1X1(EE`gv00uX(UA~K~N((zY7tI(baYL+CZaOoRxz_(IX0J#{S$> zGzBw<_a#pogQ9fmXhz>p>1YaJ`7xBv$??(gW%s&&agMrY$Cv2pqKA%uAELwl#Xz<0 zOAtKQ;Q3CvSI39wZ4Oc8w2p47=KghPo%xjRDe9p&xVSn#T4}&u#1k0w{%z=o?*Z+9 z1pBax>+C~3yc8Ct2EbQIF|7Iabz;FXw3hAK+SbGZHXLigELa7zfDPMPPz%}Ep{NB# zE!3v1o~IVaOEz)K?Tk{VaMbib3-ZOt)(prVP61q;Tdn&PokGnd?xU&s$IG|@E|FpI7< zS(#A5vNPd5IfhGqLj|UEt%w=+`?Nf`~I8@pNh29 zQYMsYr0CZC!m^fAF7BZRn&|I5f)z)pXYpLBnah^nrnY0(q~LV#pm*LyWLEh+i^pp3 z>NO2oXMs*<6B46iIW3Z>aG*Gu$pj?IcG(SKk}@H|B5epl;JLu05Fer6kI!32 z8rJXu>VlWMrxEI=f|00oMWZ>>^X?wEtxmu7ZYEGaKl8)I)4i`~pE}yj_tQRxLcM+r z?K9{PhyC;3#q~A%tv?v_yJuJEKVdHaH~ILke}1K!_$9cXIVvclQX52>bESTh}83+<89_`!tG z6KLF?CsLN{f{A6pE4|RtM%dIqNl1JhkHXNiy+AX+3tv?OF-Wy$kmR8dyKTEW6G-}tolz-@ zQ!fH($UcAO+2;Okd_$|4>*Be@TnpjGeq_>bFq%rb;x?(MDIO~v4~t8Pe>eR2lt+`0 zjH4#HWOqbV2#dvY6@AZz8{3je3PM&8a=rZEI)vOsZ=zWM3*MI98p7IeItKTE`U#)F zvOJ+^2n!qVAgCXQGnjaz>-_vS>d`1_9W72A^u~h6EI?h6js;9-Ww0pJvz)^Ltnttp zEXON9JuvVrvlJGMB!`(VK0V&h7+pJ4s#Ra=bU_zfrVNyaD)_<>76=t^8M}sR|6d1PUW+32@bJ*4&@=Zi zR<#2@>Bx0yE7#eXB~;9_cCZR3rNT-58s|Blftf zoc_R83Ei6obdRxVF0lK2ELY&Y0`F_nR?ma?n<7hLtcAwN3pLBdIw7`egpkMaY|{^e z@OkilbN2dx_gHs3XT8HVI0XZ@u&zM+`dRX}puLXUmbr6eDc5)@)V?fFyNR4l(;L^tcpxJsfq#2Q$LIvA#Rc-ggBF|G6LiR9iq5#0j`>X< zsa9^6PbrIAIrR~u>|MoJfYPzxvzzgJ=X_3`K%F9R(;>Q=kPvsQmi)gl=Q2J~6)@?Q z*^{~hqc&8=B1+O(zjt{d!z5k2`9n49OK=%y946e^cF2r~l|ew|!_sXShOXNF*WoeL zF8SwS_J8%Q4Qr^ytBRYX#=$xw>6<nRm0$uk&GIT7%*FD35F+tP45%%oRcpgUIoV`XcI>w#bHa2aq zrP&^q+%IlxhV9wyPYP16pH=U$4_?!DzRlUSUvMM*ygc=S)Q^*?{E9b1wsV<10re{? zZ%<8|=(M8YbXmh(W=|mfh~3^LW6nj~M1$&m)XPdru{}Zb4@{mIsB!{tSF!ENDS=%> zCj~@O4pqKk8J7ovtzmNoYAaB?UOsyr)V?HO!r7D^)>|_nH|!U#EjL2B8|H?_T;3U* z%s17`$wC(XzZ<}$J>@>M0~N?8GKNgd82U@s|0}u?d^Sb?BRcC2j{2?ZixYH~aYPEY zenlpuaO>LO)`zfyy?c;XqYv|+^TTSYRev~a-HTY9g}FLoP%b~mFfuO*4V_PSi%s~d z1AZOsTJ6DCd+-%GunCJ%M8PlqUA{a~u*KNA#ui}gxh=;ue&3ocz*sX%wg4Awfs7E< zD0kf}wm`83b!w~U*#c)%#2gz_OQ(VBXh8rYgGDrAYJnBHF15CZn6o*1eb@p`#~l*r zlD&3r6+KWtGv4~51+VQn`p)#g%iXlFC{MlUf!}9MbT#I)TSl@Ke}UIml1hmB_t~Bl z!GM99KbAE7vHEh5*^?gljmQ8z*MyCERYRFj3Bp4vL3nl}2L`N(-oi{^SxF_OT9vIy z0tMtNAirMTbsfkbN>wrd&XA|l_rL@j*3Dvp;S|G4Fv|7R@Fw zr%nRdP^dqd%wnFb`vw%KPw#2x%~1U*ws zcVR*Fm}ZDX<_8lP?s9Nn1=#tO zzJ@ZZE6`=b#U;9|Lb@n) zxn6DcJi6>{3hCk-0kw3SIJRLK9<^PQSb^))0PFe|d(k&%uMxUzxSd!`BspbhwxjUn z`Wf=p+t1Cm<5)Z6%ifD^KVOumUiflPhBb-gCAen{##$Pc&}o@ay`44}-xFsJDT)Z{ ziP`5SI+64DIILW}T1ly3wYmJB(DTt_NI;dKm6+`Fr^>!`5ZV)TK4rn(B#Ryci+CQR z!)g$jN>Vj8N|s}50D9%_RUX5HhDnWYh%Z>RzsiKl1+0a9W50ke*F0{W@(jN}B$ckp zA4)w7xrPe+qJZ_+IjpP$>xY68IsMO2?_17d`L|q^M6{L25hbJ{Iy&zjWIN(d-=88G zbTpm?5tTvjli8Gt5t*>)QDNb)%NA7h`PyLN=iTe~m&a)M{&3L0I_;moL2oWE-Vas7 zz62tEPSTm6D2E=CrG0wyUGK`G6)>)V@t+BduYeM-3L00VU>%|H4PJf?A;*rXwXmgc zAFahQ+yy}H7WlZOTMKv`tLsV?JYK7|dKMlxbbAv#9y*3^6N`jIw~4LkIJ7l4q;_Bv za|?KUWA+-sMzD*EPN6Dxkc6CcSm&bFb}UXIG%y!!H*4d{Lfy z0p+LAEHrAOYeDb1K-VP}TBNMtyqWx-IQd&Xj`JAkR}|`*{cY?A8X^RriYi%zH%#8OAn`yX`9TRWPoCaepR^`{d$T0lL4m z_xM?NS?2zI4ZmHO+}8N5>y#UuJ=gXM2-mgo0>aG)5QX6?47WCI^*n}a=^HTI5ZeS} zJn{o#daj0@zzq%0*EHMpN6%xpo3qyo!@X_OPt%yr1drw+p4_&kF0iblQEwYg!;*2j zb_Q|vmx`vbC{DdF+%xi-$Yb$M)Qv;&zyI@C;+ryo{@;H;F5#Pn{GKT88G(xMCORW& z%1Ehe@m$p;MfZel`_M&A^npaOt>TKJYEr37if@hBt_0wwpFfj0Yd=UKqFSqJQY9jH zo};nfcIO@u5tCcf#Vx5;tW2nEO_#EbZMQiqBw8WS^>Xv;Nc1Tz87s$*o?aSg(hVSTx_p&G< zdJ{d)qkbQQ%uw)f7NqK`$ya7XDu&0u7~Rgto2`M_<4;p6V6hUXBtqxelzg$(R#&$p zDv(xzv_Bi9eQI_5s$g2>#p?*ue%srxL$(?=Tc+!7581-Euz+m!0Gst, and not Rst<>Gst).","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2012-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[{"id":"http://zenodo.org/communities/euromixproject"}],"externalReference":[],"instance":[{"license":{"value":"https://creativecommons.org/licenses/by-sa/3.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0029","classname":"Software","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"10|opendoar____::fd4c2dc64ccb8496e6f1f94c85f30d06","value":"ProdInra","dataInfo":null},"url":["http://prodinra.inra.fr/record/255707"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null},"dateofacceptance":{"value":"2012-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"documentationUrl":[],"license":[],"codeRepositoryUrl":null,"programmingLanguage":null} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055721330,"id":"50|od______1582::5aec1186054301b66c0c5dc35972a589","originalId":["od______1582::5aec1186054301b66c0c5dc35972a589"],"collectedfrom":[{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null}],"pid":[],"dateofcollection":"2019-01-24T16:45:07Z","dateoftransformation":"","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2019-01-23T18:54:28.567Z","altered":true,"baseURL":"http://oai.prodinra.inra.fr/ft","identifier":"oai:prodinra.inra.fr:402973","datestamp":"2018-03-19T00:00:00Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Muratorio, Sylvie","name":"Sylvie","surname":"Muratorio","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"software","classname":"software","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"modèle de simulation","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"modèle physiologique","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"approche génétique","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"castanea","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"fagus sylvatica","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"PDG Documentation, version 2","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[],"dateofacceptance":{"value":"2017-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"https://creativecommons.org/licenses/by-sa/3.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0029","classname":"Software","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null},"url":["http://prodinra.inra.fr/record/402973"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null},"dateofacceptance":{"value":"2017-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"documentationUrl":[],"license":[],"codeRepositoryUrl":null,"programmingLanguage":null} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055728345,"id":"50|od______1582::626bf0af9988f811e6290d694587edf5","originalId":["od______1582::626bf0af9988f811e6290d694587edf5"],"collectedfrom":[{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null}],"pid":[],"dateofcollection":"2019-01-24T16:45:07Z","dateoftransformation":"","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2019-01-23T18:57:53.59Z","altered":true,"baseURL":"http://oai.prodinra.inra.fr/ft","identifier":"oai:prodinra.inra.fr:396375","datestamp":"2018-12-20T00:00:00Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Chades, Iadine","name":"Iadine","surname":"Chades","rank":1,"pid":null,"affiliation":null},{"fullname":"Chapron, Guillaume","name":"Guillaume","surname":"Chapron","rank":2,"pid":null,"affiliation":null},{"fullname":"Cros, Marie-Josee","name":"Marie-Josee","surname":"Cros","rank":3,"pid":null,"affiliation":null},{"fullname":"Garcia, Frederick","name":"Frederick","surname":"Garcia","rank":4,"pid":null,"affiliation":null},{"fullname":"Sabbadin, Regis","name":"Regis","surname":"Sabbadin","rank":5,"pid":null,"affiliation":null}],"resulttype":{"classid":"software","classname":"software","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[],"title":[{"value":"Package MDPtoolbox R","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"The Markov Decision Processes (MDP) toolbox proposes functions related to the resolution of discrete-time Markov Decision Processes: finite horizon, value iteration, policy iteration, linear programming algorithms with some variants and also proposes some functions related to Reinforcement Learning.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2017-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0029","classname":"Software","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null},"url":["http://prodinra.inra.fr/record/396375"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null},"dateofacceptance":{"value":"2017-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"documentationUrl":[],"license":[],"codeRepositoryUrl":null,"programmingLanguage":null} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055729488,"id":"50|od______1582::639909adfad9d708308f2aedb733e4a0","originalId":["od______1582::639909adfad9d708308f2aedb733e4a0"],"collectedfrom":[{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null}],"pid":[],"dateofcollection":"2019-01-24T16:45:07Z","dateoftransformation":"","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2019-01-23T18:50:56.823Z","altered":true,"baseURL":"http://oai.prodinra.inra.fr/ft","identifier":"oai:prodinra.inra.fr:408837","datestamp":"2018-03-19T00:00:00Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Becheler, Ronan","name":"Ronan","surname":"Becheler","rank":1,"pid":null,"affiliation":null},{"fullname":"Masson, Jean-Pierre","name":"Jean-Pierre","surname":"Masson","rank":2,"pid":null,"affiliation":null},{"fullname":"Arnaud-Haond, Sophie","name":"Sophie","surname":"Arnaud-Haond","rank":3,"pid":null,"affiliation":null},{"fullname":"Halkett, Fabien","name":"Fabien","surname":"Halkett","rank":4,"pid":null,"affiliation":null},{"fullname":"Mariette, Stéphanie","name":"Stéphanie","surname":"Mariette","rank":5,"pid":null,"affiliation":null},{"fullname":"Guillemin, Marie-Laure","name":"Marie-Laure","surname":"Guillemin","rank":6,"pid":null,"affiliation":null},{"fullname":"Valero, Myriam","name":"Myriam","surname":"Valero","rank":7,"pid":null,"affiliation":null},{"fullname":"Destombe, Christophe","name":"Christophe","surname":"Destombe","rank":8,"pid":null,"affiliation":null},{"fullname":"Stoeckel, Solenn","name":"Solenn","surname":"Stoeckel","rank":9,"pid":null,"affiliation":null}],"resulttype":{"classid":"software","classname":"software","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"clonalite","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"eucaryote","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"reproduction clonale","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"approche bayésienne","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"écologie végétale","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"logiciel","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"endogamie","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"autoincompatibilité","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"ClonEstiMate 1.01","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"ClonEstiMate, a Bayesian method for quantifying rates of clonality of populations genotyped at two-time steps","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2016-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"https://creativecommons.org/licenses/by-sa/3.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0029","classname":"Software","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null},"url":["http://prodinra.inra.fr/record/408837"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null},"dateofacceptance":{"value":"2016-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"documentationUrl":[],"license":[],"codeRepositoryUrl":null,"programmingLanguage":null} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055738736,"id":"50|od______1582::6e7a9b21a2feef45673890432af34244","originalId":["od______1582::6e7a9b21a2feef45673890432af34244"],"collectedfrom":[{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null}],"pid":[],"dateofcollection":"2019-01-24T16:45:07Z","dateoftransformation":"","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2019-01-23T18:53:57.871Z","altered":true,"baseURL":"http://oai.prodinra.inra.fr/ft","identifier":"oai:prodinra.inra.fr:403174","datestamp":"2018-03-19T00:00:00Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Delenne, Jean-Yves","name":"Jean-Yves","surname":"Delenne","rank":1,"pid":null,"affiliation":null},{"fullname":"Richefeu, Vincent","name":"Vincent","surname":"Richefeu","rank":2,"pid":null,"affiliation":null},{"fullname":"Frank, Xavier","name":"Xavier","surname":"Frank","rank":3,"pid":null,"affiliation":null},{"fullname":"Radjaï, Farhang","name":"Farhang","surname":"Radjaï","rank":4,"pid":null,"affiliation":null}],"resulttype":{"classid":"software","classname":"software","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Modélisation et simulation","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Modeling and Simulation","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"structure granulaire","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"algorithme","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"simulation numérique","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"flux de gaz","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"flux de liquide","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Flowbox","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"The code FLOWbox is dedicated to the computation of flow through porous and granular materials.\nFLOWbox is based on an optimized 3D Lattice Boltzmann algorithm for the computation of liquid or gas flows directly at the scale of heterogeneities. FLOWbox intends to be a powerful and versatile software able to operate on highly detailed microstructures in a systematic fashion. These microstructures can be generated either from numerical simulation or from tomography.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2015-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"https://creativecommons.org/licenses/by-sa/3.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0029","classname":"Software","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null},"url":["http://prodinra.inra.fr/record/403174"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null},"dateofacceptance":{"value":"2015-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"documentationUrl":[],"license":[],"codeRepositoryUrl":null,"programmingLanguage":null} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055741711,"id":"50|od______1582::71dd00de0e70764d2800a766c0b165db","originalId":["od______1582::71dd00de0e70764d2800a766c0b165db"],"collectedfrom":[{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null}],"pid":[],"dateofcollection":"2019-01-24T16:45:07Z","dateoftransformation":"","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2019-01-23T18:35:31.489Z","altered":true,"baseURL":"http://oai.prodinra.inra.fr/ft","identifier":"oai:prodinra.inra.fr:442564","datestamp":"2018-11-12T00:00:00Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Bitteur, Sylvaine","name":"Sylvaine","surname":"Bitteur","rank":1,"pid":null,"affiliation":null},{"fullname":"Lortal, Sylvie","name":"Sylvie","surname":"Lortal","rank":2,"pid":null,"affiliation":null}],"resulttype":{"classid":"software","classname":"software","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[],"title":[{"value":"Charte graphique et site web MILK International Symposium","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[],"dateofacceptance":{"value":"2014-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"https://creativecommons.org/licenses/by-nd/3.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0029","classname":"Software","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null},"url":["http://prodinra.inra.fr/record/442564"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null},"dateofacceptance":{"value":"2014-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"documentationUrl":[],"license":[],"codeRepositoryUrl":null,"programmingLanguage":null} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055745351,"id":"50|od______1582::764d9ba73f2f3e794e00a80b075330ef","originalId":["od______1582::764d9ba73f2f3e794e00a80b075330ef"],"collectedfrom":[{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null}],"pid":[],"dateofcollection":"2019-01-24T16:45:07Z","dateoftransformation":"","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2019-01-23T18:41:37.345Z","altered":true,"baseURL":"http://oai.prodinra.inra.fr/ft","identifier":"oai:prodinra.inra.fr:407117","datestamp":"2018-05-28T00:00:00Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Favre, Colette","name":"Colette","surname":"Favre","rank":1,"pid":null,"affiliation":null},{"fullname":"Grel, Audrey","name":"Audrey","surname":"Grel","rank":2,"pid":null,"affiliation":null},{"fullname":"Granier, Evelyne","name":"Evelyne","surname":"Granier","rank":3,"pid":null,"affiliation":null},{"fullname":"Cosserat-Mangeot, Régine","name":"Régine","surname":"Cosserat-Mangeot","rank":4,"pid":null,"affiliation":null},{"fullname":"Bachacou, Jean","name":"Jean","surname":"Bachacou","rank":5,"pid":null,"affiliation":null},{"fullname":"LEROY, Nathalie","name":"Nathalie","surname":"Leroy","rank":6,"pid":null,"affiliation":null},{"fullname":"Dupouey, Jean-Luc","name":"Jean-Luc","surname":"Dupouey","rank":7,"pid":null,"affiliation":null}],"resulttype":{"classid":"software","classname":"software","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"fra/fre","classname":"French","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"digitalisation","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"carte des sols","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"géoréférencement","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"vectorisation","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"France","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"usage du sol","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Digitalisation des cartes anciennes : Manuel pour la vectorisation de l'usage des sols et le géo-référencement des minutes 1:40 000 de la carte d' Etat-Major","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[],"dateofacceptance":{"value":"2017-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"https://creativecommons.org/licenses/by-sa/3.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0029","classname":"Software","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null},"url":["http://prodinra.inra.fr/record/407117"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null},"dateofacceptance":{"value":"2017-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"documentationUrl":[],"license":[],"codeRepositoryUrl":null,"programmingLanguage":null} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055748883,"id":"50|od______1582::7a6ace3d7ad42ac813e4955f66a44435","originalId":["od______1582::7a6ace3d7ad42ac813e4955f66a44435"],"collectedfrom":[{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null}],"pid":[],"dateofcollection":"2019-01-24T16:45:07Z","dateoftransformation":"","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2019-01-23T18:52:22.609Z","altered":true,"baseURL":"http://oai.prodinra.inra.fr/ft","identifier":"oai:prodinra.inra.fr:407676","datestamp":"2018-03-19T00:00:00Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Desjeux, Yann","name":"Yann","surname":"Desjeux","rank":1,"pid":null,"affiliation":null},{"fullname":"Latruffe, Laure","name":"Laure","surname":"Latruffe","rank":2,"pid":null,"affiliation":null}],"resulttype":{"classid":"software","classname":"software","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Stochastic Frontier Analysis (SFA);R; R package","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"sfadv: Advanced Methods for Stochastic Frontier Analysis. R package version 1.0.1","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Stochastic frontier analysis with advanced methods.\nIn particular, it applies the approach proposed by Latruffe et al. (2017) to estimate a stochastic frontier with technical inefficiency effects when one input is endogenous.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2017-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"https://creativecommons.org/licenses/by-sa/3.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0029","classname":"Software","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null},"url":["http://prodinra.inra.fr/record/407676"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::1582","value":"ProdInra","dataInfo":null},"dateofacceptance":{"value":"2017-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"documentationUrl":[],"license":[],"codeRepositoryUrl":null,"programmingLanguage":null} diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/software/software_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/sample/software/software_10.json.gz deleted file mode 100644 index 3dcadf41d70ba3ac0cd5661d34778d279d35e352..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 6727 zcmV-N8o1>jiwFo3Y@l8M19NX?ba!ELWnVEcE^2dcZUF6FUvt|yl7GKXfm7Aq*j2Jj ziljto_V&uLoy2o?lCrbAH@7pDf=Ez8LJ?XLw4==CVc+hfte^Hn_;o+jbp!k-S&nQ^ z5-Vd&Wo#1!&_H*ifo}W={L%`kq(@04TEkyjJUQnXAH}RSjA)#(Hl#!>O<7nage+!~ zg`>H8!;+xRE7LratzpaR3|d7=d_POYIZLR@*@74KGYupc2@(!snq?fQw6b}|r!$et zVUUU}yP)yAVajGA<5Hw^D6X<5bjog%ISVFi%2dT+!sM`~y{BkrRi2{88*+X8*lqz` zna_Y-O!6tqB%RJ$L)-H`%kz2z-|_pcPKW6uHm{k@7K5XnX_}{H1lr;y(jdL>#jK1QVrV3RV2lddKF_7BOQRR?-$Fg_2?rXa@ zbiJX~|KC=bQ>HY@B9TsMksIEy_mHaV$0`NoO|AH)=>7rAf|SonrI^t49Qb?ywKi!w zuWf(mxI@S5xZVKi(pWO3qo9GHha;M?Kb{^#nTeFM;qESw>j1rwLti`UN0jbH5|uP$ ziR2MyDe{W);c~X2>vp|iW7|7+_qAmW)jw>goDOsK_=7%&8#ye2cLY^pGpbFc@Fh)%{i9WOP3 zCF3RPhsikR*<=OrvWC?}>$e|qgyx9^qof8vmXF@TXi{P>Mk8sRv2##kLW5i~0u7$= zl%gFY8L%9NH6wvYMMBRx7+Qh@`L8*R3+*M1E|At@2h?-j^u}ZLo_uWD0J1O@kZW@R z`NU*E7H0$Usp){Mo(~rLa+ew57P*^zxYe9+W44bUQIjQ?$fR13t$i?DAi;4;XA}O{ zoDn7okZ0tAjmS{~wklDhfyOYLr`WW7N)D&|m=iE8OLA>$<`l*`(exG-bm7Iaj-pui zf&81r3{6E{%CevfW>xTsRe?5TF*~P;4AnfPX3XY1R!^ZcfNslj$}6=6MUL~DWw#95 zZ_9RQcs@d#j6+0EGi*Ad=~%#WM+-uy!o-vYg;;Qglduvgp419O{~ZW0&3u$gHS_A} zNfp!+l&&g(cBE>yz#Jo741hqh43;hvS!YTL62IuhU@`xHV2KEYn)L%Qo#qLbFbVC3 z$gvh`5fnnZ5TCM2l1PRpS)nmdVax-TWDDAZ)n`zDkTS3-=L{b{Gfz0t=Y!4pGfqXaqcGvl#a?LxP%iA>ci*I04@z8xr_U z!knpwL6vp0iVoFn*E9wENXrsjW>6whY9aNipP-AVa;6Mbn7(Lo{~Kncy01}?-?0bZQ}O2J8mETxR^k?HPq8p^g-s<;g6U1bi@tf~N$7>H2XiW2je(TSMd{I-DK(UT$M^aN`uIMLf+U ziFtxmg&5HUEyov0J{>{-fOo3@=*(qgOO=Ce0xjF$ZZG9*aCsrC3puefN=q67RWjUy zJlRC#aYzy&Q7SRUu#g#4b2^)%azHa1p01^a5~xxn z;#gd07NA{KVp$&4TgWeDW2q`N3slyMW2jr#3Wp;3PNM3;h>6uE1dJgDK1dRP>$*K8 zz*^!$yNONo|Cah~lTN3z1@8h8=hGzHR;gX}2L5v32V_v0ef85Jy&?@W42rZqJb~Ab z9&=WX15lMj84DcnxvW%bt5h3ER9yZZN9+j422Yf^5-I-x)(6-+2NsfNEMX~FyTZhQ zMuKF<(8WY++EDQfC^Z<;u#C?$)Tw2A`CJzI^9&?cRoE_MrR^xpA{_LlCR~lxWy4Yk zdMA7yz6ONINd1G8P=YWjtLk;^-;eNQlrRy1$m0a#g$;3tD4ddh@#d#&s zr4p^@JOn<1`G6XgeJ{-_=1KA(W*`;J0Ih*(KgX~!c@BJ>@BEqZ1cuF+fb#R0k5Zmb zJLI{zKqXT#ZK&n~g(cMpN6}}9`2h7mALYNuflxL>(s!C!&~OmAT1hBn!Di~G{bsLK zzvimG6!{5FgE-!Q@LZ8EyfLBvoK6j2pzAtnZ+tmn`YC6C42z~#pDpW^ei z6^8EX3U*{lF~U56p|Q|nHM(%cfv6m9l!08yCR~nSEm_Cx|J-Ha`sqD2t7rDq9w{Os zz}4g%d+-Jp!Zeu6H*^%dnNjFJZDB}3RiQHN*n_WuoXDs_P8j6GXF^UaZaDgV%6ieL z=h#s{@M*8xXFlutecuZQgF!d^tjLLCt8Rmk6UNFL)KtX zOTC^w0^>1q2VQq%`y+p3F|W4{IkB$z<3&#Nov(tN=o{q3qeD)B%bcUfU#=B0<&rY6 zkf}(}5}PeCgQYM6F<6R?lm_nKTwP-4iVd9c)n#&ZDa|-}Xvr9q%R@_m&SqsKfIv6( zO4W>u2e()Td$fsiQJZa7{|wFW*boTVF*r;CXK_Yev2>bC{A@Xxot>TDEFc34+eFdc z0TlN9ki0s5`RsJ>guFUDJ^ABnviIVEG?VvH1IUxZ{g)?)${;K?9gE)@@WMowkMvTtnw!`)e;CLx>Oz1fyKEcR13~AJoDSgkU`E*D?8(go~_Z6N@@ueuklOWE+ z0$Zk#Cy4i`R(B#9mfG;NMxK>vFDe*SFqcKYUaUlyHFDOiLkJO^k6W6(t08=i>eXPS72X1v zKBXZmMN@)&3YZ?*`st@b{imK%e)>t_`;g0u$=6S1;b$5uB7A%5pP1Wn0M zy2vG3z_Ut@lPwJ|+eXY$`LYRLOT`8B{V4{?0OO#>n?szN|#8OW)CZ5PNqGbSqPqI^*L6mT@p4Lly|Sc9rx?Wj5nP&Vdx$29_+DZ_-!LZ<$w>Vj5}LP?5wTQ44lUv0u{PI7@l ztJbyx7F@u(dNxwfup8&i>`P6Yjdp%wek|lwPL3d6HDl^k!Qw=Ty4A?HV&43%&zMTd zDzM&tr!aDMZ;+^Uv};S#T?5?xu!4^*y;)F^xK}mPS;Lg%0>HC zGuY{8p21GP{tTwO01m0Po?}sbZv(2mgQm6crnY#8VZM^pQ0xziiAWmT2MNE&lx>3t zzEjM?oUvo@z$@{uQB5W@cZaRx(4vuwH?R05%5b{VB3Dr@?6KVN4;KP1zs39 z-k{&5o{!x6B;a>l@ka;v^%3yf^*i2|KPt_I*{FZhQEB$zD*=8_a`n|E7y2l9zI2d0 zUpz*>%H1-U-(PiFnh3A{5#vX*_+fVu!Z5gphlHP$BAHEqzET{6)OvXQ9IR$0%wS44 zMIr%3--=Nqg@=?vM8|nXGQ8(Sg5zm~u^D)|R|n6?LDQLP6SX5KlI6Qu3}kN8rTgpe z@aUV+#&0h|ZGz34s5U0pyeimy@0jf^FW%7Mi%p6q8!QiNbMwU;y>4$5 zSu`39d_VGSoAtT_E9?zi&+o$+_SQs`*A{<_(d7Qnb2?xCR#IosbNb$@V6xr)dZ$P1 zPiTnnup=5STuE9z*O6qM_10nJ_A(vd0QgtWavsMN3{Fj}nY2I-OSEo#Bk2;o6pv6m zp($rOKZuMq$So!|2ttvHnsX;=&w%@!w#ida7MAkhT}`c-RMXN0Dr)YXsGZT#2*rcv zeT;e5Y8?2XA8R^g){2s6g8#P;{;v)h(BK`EBPR#1q!95)yeH=Cx^HNRfhMgoA4-*$z(n zH&Ypsh$ndUY;mx0yTpP@mKHH97&ttbH&YPeOH(ypMXifP6_d1ym3;kJNwGRNv664x zB$!yqZ%U-=VkHCD_iv1qID>&TpkYMALD;u^$MT~t1t#_#hq=^R7b{s;{4vH#tfANI z_}wpl)TV3szH`%2n_mri_k@9x#w-QLTL9Ati{h3*+J*@Ig0aOXJ{N!TW5Vm4U-7M9O7wPt~PhE1M>rC?IF$x}MwtijLf zxh7ZSy=TZo#eAe>&@*{;Ih)XAp?8w~>+e-!lcBoQt)nMPQ6Y?Lu_*>>bj<0P=IOOS zRGHWa4wdC9y1hHm{SyVn6F_th&fT;@uY9ga>8ur<{+;N6p^#!a0;cXyQdlJkeY9g0 zziXt`6w5*tP5*AOFDY2?j>V{=F-wxBiWbj}=vR@sqBStluX{jWgBZ+P%%wr{Jt&5Z zPWnM6NCaJ)4p9tIP4#r8HyY z!H*W!Cqo)3Jm?gjgknsm+zf_?oiK7Kc!I~xz~X-dJ^=WHMk1R5$=`<+)nO+236+f4 z9To33o9o|@NN>Nn-V}A4Q1XP%S;lEXrc6$RzOEHSkDxcaeKWh$=1U3lrNnP__rKU~+N9loV{*;z|6>>F zn%#fL?>oI4cmG+R4n|#@b|c0jJonb|2bSw}Y2>(F*KMuY{a;u7G4B35L$BZQ`}UXL z{derX+j`*L{{w~_jI7v{|M8qPkoV1`=3Zx9#G>zm0XW5L3?r7e$)7lgNZl$~ZrfG~ zHL)^#>wf$HPq9#&{0}|n3bjxqSI;#em9urf9sg4rzNJ_H#2x>1Le+j|wdG$XG`fVj z`M>_Jb=ysUvwyp4|Mou9pU90j_P}}# ztP16+3D-YtxW4>y#mM1dM@D6#@DhZp%ha6i_0S@T;`}`xNjRn-j372a5O~`-Hy2`V z%5LJ*SX_+6`G8`y<5##mgJBqA6uJKHR^M1f%MWj(Slum> zihPXs(ZFy0{cEw_OUV>(H{rnR&QD3B-s+M{yb1=IGUXrO**PG`R7xH&@B7i1O$_uq%)qqDK(754&A`A3wTVje4 z(%EEgz9hcWU|F+q;F*mBvsY{OYRz8lZ+5R%Z4PYGUac{?X0P_Ki*(Iit!wvf`=-6x zz8!{^6*7zUt$xo9yS`=7KHhaQvU^@QYJGaIc5U%TxL50VL&xs8{ue*i&UL$9Z{=RC zZSUCKul1!imy+eFz9EG!oNU)fX?(7&^SAD|Q+o^}ipICPjzwo}-FA=GY-p}Ij%^>5 zUaGB{*#yJUdV3R6F4%~i93B5RIZD(z8JP->=h+))Que0AGi!WS>GC2 zea~?$7OiuH*A;(+j<9PFoqor0y)O@1?%Ve2Bg?#Fb0hA(gvzKcMnH zSOwg{+&KZe$AoN?)2qwzLLlLk{p;`JYlq}7m22Jays{@0%scqO8f@NLz?m-wE{cq; z`yEkse0ciu$2NIEQQ=WpHi zXtK?Yltd}rjjpxeDNb`=8vPxDft0j_f8QMTSZdp1tCrhgf){sK(lsyR#9&pZPFIgJ zBa4TW#rWz{q*s^G)urm-RDVxn1oB`4IS0!H%Q<7G9$pmqiC17itPd)KJi}{d!W^dx zBZdJJn zvos^AJX4Y$`pOngTUrSdo4iu7hZF`HVOsA1;LVza4b(4F6i+L%qP z2SInLDC#C{Y8jJjHnkqRNY`y@`M&SmxT!^ZG+<8Hr=i=Wfp0s^9e7^U>rvNroppB> zuPy!vH?_R(u-onQtS^u5wEDe%ul3+3@nmmV{=Q9qq|F;Zil>FUMlfINo~TX=42&Yy zCQJ9WT)(lU%-S-z&R?@taV7;!02zK}_%wypFeoC~12X`2EhAfJPxrRJKmDGZl9|4P z*F;4&BPxnE)L0PPD+D} zp*0$J$QG{Kx5@7hULFl?&@gM@?9#WC?NWMS_4|LokD_5d Date: Mon, 16 Oct 2023 08:57:47 +0200 Subject: [PATCH 05/60] [raw graph] WIP: mapping original resource types --- .../raw/AbstractMdRecordToOafMapper.java | 64 +- .../dhp/oa/graph/raw/OafToOafMapper.java | 28 +- .../dhp/oa/graph/raw/OdfToOafMapper.java | 22 +- .../dnetlib/dhp/oa/graph/raw/MappersTest.java | 60 +- .../dnetlib/dhp/oa/graph/clean/synonyms.txt | 855 +++++++++++++++++- .../eu/dnetlib/dhp/oa/graph/clean/terms.txt | 292 +++++- 6 files changed, 1244 insertions(+), 77 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java index 2fed7d627..5f0acbab7 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java @@ -42,10 +42,6 @@ public abstract class AbstractMdRecordToOafMapper { protected static final String DATACITE_SCHEMA_KERNEL_3 = "http://datacite.org/schema/kernel-3"; protected static final String DATACITE_SCHEMA_KERNEL_3_SLASH = "http://datacite.org/schema/kernel-3/"; - protected static final String OPENAIRE_COAR_RESOURCE_TYPES_3_1 = "openaire::coar_resource_types_3_1"; - - public static final String OPENAIRE_USER_RESOURCE_TYPES = "openaire::user_resource_types"; - protected static final Qualifier ORCID_PID_TYPE = qualifier( ModelConstants.ORCID_PENDING, ModelConstants.ORCID_CLASSNAME, @@ -129,7 +125,9 @@ public abstract class AbstractMdRecordToOafMapper { final String type = getResultType(doc, instances); - return createOafs(doc, type, instances, collectedFrom, entityInfo, lastUpdateTimestamp); + final Qualifier metaResourceType = getMetaResourceType(instances); + + return createOafs(doc, type, metaResourceType, instances, collectedFrom, entityInfo, lastUpdateTimestamp); } catch (DocumentException e) { log.error("Error with record:\n" + xml); return Lists.newArrayList(); @@ -155,6 +153,30 @@ public abstract class AbstractMdRecordToOafMapper { return type; } + protected Qualifier getMetaResourceType(final List instances) { + + if (vocs.vocabularyExists(OPENAIRE_META_RESOURCE_TYPE)) { + Optional instanceTypeMapping = instances + .stream() + .flatMap(i -> i.getInstanceTypeMapping().stream()) + .filter(t -> OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(t.getVocabularyName())) + .findFirst(); + + if (!instanceTypeMapping.isPresent()) { + throw new IllegalStateException("unable to find an instance from " + OPENAIRE_COAR_RESOURCE_TYPES_3_1); + } else { + final String typeCode = instanceTypeMapping.get().getTypeCode(); + return Optional + .ofNullable(vocs.lookupTermBySynonym(OPENAIRE_META_RESOURCE_TYPE, typeCode)) + .orElseThrow(() -> + new IllegalStateException("unable to find a synonym for '" + typeCode + "' in " + + OPENAIRE_META_RESOURCE_TYPE)); + } + } else { + throw new IllegalStateException("vocabulary '" + OPENAIRE_META_RESOURCE_TYPE + "' not available"); + } + } + private KeyValue getProvenanceDatasource(final Document doc, final String xpathId, final String xpathName) { final String dsId = doc.valueOf(xpathId); final String dsName = doc.valueOf(xpathName); @@ -169,12 +191,13 @@ public abstract class AbstractMdRecordToOafMapper { protected List createOafs( final Document doc, final String type, + final Qualifier metaResourceType, final List instances, final KeyValue collectedFrom, final DataInfo info, final long lastUpdateTimestamp) { - final OafEntity entity = createEntity(doc, type, instances, collectedFrom, info, lastUpdateTimestamp); + final OafEntity entity = createEntity(doc, type, metaResourceType, instances, collectedFrom, info, lastUpdateTimestamp); final Set originalId = Sets.newHashSet(entity.getOriginalId()); originalId.add(entity.getId()); @@ -207,6 +230,7 @@ public abstract class AbstractMdRecordToOafMapper { private OafEntity createEntity(final Document doc, final String type, + final Qualifier metaResourceType, final List instances, final KeyValue collectedFrom, final DataInfo info, @@ -214,12 +238,12 @@ public abstract class AbstractMdRecordToOafMapper { switch (type.toLowerCase()) { case "publication": final Publication p = new Publication(); - populateResultFields(p, doc, instances, collectedFrom, info, lastUpdateTimestamp); + populateResultFields(p, metaResourceType, doc, instances, collectedFrom, info, lastUpdateTimestamp); p.setJournal(prepareJournal(doc, info)); return p; case "dataset": final Dataset d = new Dataset(); - populateResultFields(d, doc, instances, collectedFrom, info, lastUpdateTimestamp); + populateResultFields(d, metaResourceType, doc, instances, collectedFrom, info, lastUpdateTimestamp); d.setStoragedate(prepareDatasetStorageDate(doc, info)); d.setDevice(prepareDatasetDevice(doc, info)); d.setSize(prepareDatasetSize(doc, info)); @@ -230,7 +254,7 @@ public abstract class AbstractMdRecordToOafMapper { return d; case "software": final Software s = new Software(); - populateResultFields(s, doc, instances, collectedFrom, info, lastUpdateTimestamp); + populateResultFields(s, metaResourceType, doc, instances, collectedFrom, info, lastUpdateTimestamp); s.setDocumentationUrl(prepareSoftwareDocumentationUrls(doc, info)); s.setLicense(prepareSoftwareLicenses(doc, info)); s.setCodeRepositoryUrl(prepareSoftwareCodeRepositoryUrl(doc, info)); @@ -240,7 +264,7 @@ public abstract class AbstractMdRecordToOafMapper { case "otherresearchproducts": default: final OtherResearchProduct o = new OtherResearchProduct(); - populateResultFields(o, doc, instances, collectedFrom, info, lastUpdateTimestamp); + populateResultFields(o, metaResourceType, doc, instances, collectedFrom, info, lastUpdateTimestamp); o.setContactperson(prepareOtherResearchProductContactPersons(doc, info)); o.setContactgroup(prepareOtherResearchProductContactGroups(doc, info)); o.setTool(prepareOtherResearchProductTools(doc, info)); @@ -377,11 +401,13 @@ public abstract class AbstractMdRecordToOafMapper { private void populateResultFields( final Result r, + final Qualifier metaResourceType, final Document doc, final List instances, final KeyValue collectedFrom, final DataInfo info, final long lastUpdateTimestamp) { + r.setMetaResourceType(metaResourceType); r.setDataInfo(info); r.setLastupdatetimestamp(lastUpdateTimestamp); r.setId(createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false)); @@ -531,15 +557,17 @@ public abstract class AbstractMdRecordToOafMapper { if (vocs.vocabularyExists(OPENAIRE_COAR_RESOURCE_TYPES_3_1)) { // TODO verify what the vocabs return when a synonym is not defined - Qualifier coarTerm = vocs.lookupTermBySynonym(OPENAIRE_COAR_RESOURCE_TYPES_3_1, originalType); - mappings.add(OafMapperUtils.instanceTypeMapping(originalType, coarTerm)); + Optional.ofNullable(vocs.lookupTermBySynonym(OPENAIRE_COAR_RESOURCE_TYPES_3_1, originalType)) + .ifPresent(coarTerm -> { + mappings.add(OafMapperUtils.instanceTypeMapping(originalType, coarTerm)); + if (vocs.vocabularyExists(OPENAIRE_USER_RESOURCE_TYPES)) { - if (vocs.vocabularyExists(OPENAIRE_USER_RESOURCE_TYPES)) { - - // TODO verify what the vocabs return when a synonym is not defined - Qualifier userTerm = vocs.lookupTermBySynonym(OPENAIRE_USER_RESOURCE_TYPES, coarTerm.getClassid()); - mappings.add(OafMapperUtils.instanceTypeMapping(originalType, userTerm)); - } + // TODO verify what the vocabs return when a synonym is not defined + Optional + .ofNullable(vocs.lookupTermBySynonym(OPENAIRE_USER_RESOURCE_TYPES, coarTerm.getClassid())) + .ifPresent(type -> mappings.add(OafMapperUtils.instanceTypeMapping(originalType, type))); + } + }); } return mappings; diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java index 5cdb434f1..6bcc8ec44 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java @@ -1,29 +1,27 @@ package eu.dnetlib.dhp.oa.graph.raw; -import static eu.dnetlib.dhp.schema.common.ModelConstants.*; -import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.*; - -import java.net.URLDecoder; -import java.util.*; -import java.util.stream.Collectors; - -import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; -import org.apache.commons.lang3.StringUtils; -import org.dom4j.Document; -import org.dom4j.Element; -import org.dom4j.Node; - import com.google.common.collect.Lists; - import eu.dnetlib.dhp.common.PacePerson; import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions; import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; import eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits; +import org.apache.commons.lang3.StringUtils; +import org.dom4j.Document; +import org.dom4j.Element; +import org.dom4j.Node; -import static org.apache.commons.lang3.StringUtils.contains; +import java.net.URLDecoder; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import static eu.dnetlib.dhp.schema.common.ModelConstants.*; +import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.*; public class OafToOafMapper extends AbstractMdRecordToOafMapper { diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java index c01775327..c383e9300 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java @@ -220,11 +220,6 @@ public class OdfToOafMapper extends AbstractMdRecordToOafMapper { } } - @Override - protected List prepareInstanceTypeMapping(Document doc) { - return null; - } - /** * The Datacite element * @@ -235,13 +230,20 @@ public class OdfToOafMapper extends AbstractMdRecordToOafMapper { */ @Override protected String findOriginalType(Document doc) { - final Element resourceType = (Element) doc.selectSingleNode( - "//metadata/*[local-name() = 'resource']/*[local-name() = 'resourceType']"); + String resourceType = Optional.ofNullable((Element) doc.selectSingleNode( + "//*[local-name()='metadata']/*[local-name() = 'resource']/*[local-name() = 'resourceType']")) + .map(element -> { + final String resourceTypeURI = element.attributeValue("anyURI"); + final String resourceTypeTxt = element.getText(); - final String resourceTypeURI = resourceType.attributeValue("anyURI"); - final String resourceTypeTxt = resourceType.getText(); + return ObjectUtils.firstNonNull(resourceTypeURI, resourceTypeTxt); + }) + .orElse(doc.valueOf( + "//*[local-name()='metadata']/*[local-name() = 'resource']/*[local-name() = 'CobjCategory']/text()") + ); + + return resourceType; - return ObjectUtils.firstNonNull(resourceTypeURI, resourceTypeTxt); } @Override diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java index 5df2b7a3b..4d30a3fa8 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java @@ -74,7 +74,7 @@ class MappersTest { assertTrue(StringUtils.isNotBlank(p.getDateofcollection())); assertTrue(StringUtils.isNotBlank(p.getDateoftransformation())); - assertTrue(p.getAuthor().size() > 0); + assertFalse(p.getAuthor().isEmpty()); final Optional author = p .getAuthor() .stream() @@ -97,14 +97,14 @@ class MappersTest { assertEquals("Votsi", author.get().getSurname()); assertEquals("Nefta", author.get().getName()); - assertTrue(p.getSubject().size() > 0); + assertFalse(p.getSubject().isEmpty()); assertTrue(StringUtils.isNotBlank(p.getJournal().getIssnOnline())); assertTrue(StringUtils.isNotBlank(p.getJournal().getName())); assertTrue(p.getPid().isEmpty()); assertNotNull(p.getInstance()); - assertTrue(p.getInstance().size() > 0); + assertFalse(p.getInstance().isEmpty()); p .getInstance() .forEach(i -> { @@ -121,7 +121,7 @@ class MappersTest { Optional coarType = instance.getInstanceTypeMapping() .stream() - .filter(itm -> AbstractMdRecordToOafMapper.OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(itm.getVocabularyName())) + .filter(itm -> ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(itm.getVocabularyName())) .findFirst(); assertTrue(coarType.isPresent()); @@ -130,7 +130,7 @@ class MappersTest { Optional userType = instance.getInstanceTypeMapping() .stream() - .filter(itm -> AbstractMdRecordToOafMapper.OPENAIRE_USER_RESOURCE_TYPES.equals(itm.getVocabularyName())) + .filter(itm -> ModelConstants.OPENAIRE_USER_RESOURCE_TYPES.equals(itm.getVocabularyName())) .findFirst(); assertTrue(userType.isPresent()); @@ -228,7 +228,7 @@ class MappersTest { assertTrue(StringUtils.isNotBlank(p.getDateofcollection())); assertTrue(StringUtils.isNotBlank(p.getDateoftransformation())); - assertTrue(p.getAuthor().size() > 0); + assertFalse(p.getAuthor().isEmpty()); final Optional author = p .getAuthor() .stream() @@ -251,13 +251,13 @@ class MappersTest { assertEquals("Votsi", author.get().getSurname()); assertEquals("Nefta", author.get().getName()); - assertTrue(p.getSubject().size() > 0); - assertTrue(p.getPid().size() > 0); + assertFalse(p.getSubject().isEmpty()); + assertFalse(p.getPid().isEmpty()); assertEquals("PMC1517292", p.getPid().get(0).getValue()); assertEquals("pmc", p.getPid().get(0).getQualifier().getClassid()); assertNotNull(p.getInstance()); - assertTrue(p.getInstance().size() > 0); + assertFalse(p.getInstance().isEmpty()); p .getInstance() .forEach(i -> { @@ -287,7 +287,7 @@ class MappersTest { final List list = new OafToOafMapper(vocs, true, true).processMdRecord(xml); - assertTrue(list.size() > 0); + assertFalse(list.isEmpty()); assertTrue(list.get(0) instanceof Publication); final Publication p = (Publication) list.get(0); @@ -343,7 +343,7 @@ class MappersTest { assertTrue(d.getOriginalId().stream().anyMatch(oid -> oid.equals("oai:zenodo.org:3234526"))); assertValidId(d.getCollectedfrom().get(0).getKey()); assertTrue(StringUtils.isNotBlank(d.getTitle().get(0).getValue())); - assertTrue(d.getAuthor().size() > 0); + assertFalse(d.getAuthor().isEmpty()); final Optional author = d .getAuthor() @@ -377,13 +377,13 @@ class MappersTest { final Field affiliation = opAff.get(); assertEquals("ISTI-CNR", affiliation.getValue()); - assertTrue(d.getSubject().size() > 0); - assertTrue(d.getInstance().size() > 0); - assertTrue(d.getContext().size() > 0); - assertTrue(d.getContext().get(0).getId().length() > 0); + assertFalse(d.getSubject().isEmpty()); + assertFalse(d.getInstance().isEmpty()); + assertFalse(d.getContext().isEmpty()); + assertFalse(d.getContext().get(0).getId().isEmpty()); assertNotNull(d.getInstance()); - assertTrue(d.getInstance().size() > 0); + assertFalse(d.getInstance().isEmpty()); d .getInstance() .forEach(i -> { @@ -457,7 +457,7 @@ class MappersTest { // assertEquals("oai:pub.uni-bielefeld.de:2949739", p.getOriginalId().get(0)); assertValidId(p.getCollectedfrom().get(0).getKey()); - assertTrue(p.getAuthor().size() > 0); + assertFalse(p.getAuthor().isEmpty()); final Optional author = p .getAuthor() @@ -469,14 +469,14 @@ class MappersTest { assertEquals("Potwarka", author.get().getSurname()); assertEquals("Luke R.", author.get().getName()); - assertTrue(p.getSubject().size() > 0); - assertTrue(p.getInstance().size() > 0); + assertFalse(p.getSubject().isEmpty()); + assertFalse(p.getInstance().isEmpty()); assertNotNull(p.getTitle()); assertFalse(p.getTitle().isEmpty()); assertNotNull(p.getInstance()); - assertTrue(p.getInstance().size() > 0); + assertFalse(p.getInstance().isEmpty()); p .getInstance() .forEach(i -> { @@ -624,9 +624,9 @@ class MappersTest { assertValidId(s.getId()); assertValidId(s.getCollectedfrom().get(0).getKey()); assertTrue(StringUtils.isNotBlank(s.getTitle().get(0).getValue())); - assertTrue(s.getAuthor().size() > 0); - assertTrue(s.getSubject().size() > 0); - assertTrue(s.getInstance().size() > 0); + assertFalse(s.getAuthor().isEmpty()); + assertFalse(s.getSubject().isEmpty()); + assertFalse(s.getInstance().isEmpty()); final Relation r1 = (Relation) list.get(1); final Relation r2 = (Relation) list.get(2); @@ -882,7 +882,7 @@ class MappersTest { assertEquals(2, p.getOriginalId().size()); assertTrue(p.getOriginalId().stream().anyMatch(oid -> oid.equals("df76e73f-0483-49a4-a9bb-63f2f985574a"))); assertValidId(p.getCollectedfrom().get(0).getKey()); - assertTrue(p.getAuthor().size() > 0); + assertFalse(p.getAuthor().isEmpty()); final Optional author = p .getAuthor() @@ -892,14 +892,14 @@ class MappersTest { assertEquals("Museum Sønderjylland", author.get().getFullname()); - assertTrue(p.getSubject().size() > 0); - assertTrue(p.getInstance().size() > 0); + assertFalse(p.getSubject().isEmpty()); + assertFalse(p.getInstance().isEmpty()); assertNotNull(p.getTitle()); assertFalse(p.getTitle().isEmpty()); assertNotNull(p.getInstance()); - assertTrue(p.getInstance().size() > 0); + assertFalse(p.getInstance().isEmpty()); p .getInstance() .forEach(i -> { @@ -939,10 +939,10 @@ class MappersTest { System.out.println("***************"); final Dataset p = (Dataset) list.get(0); - assertTrue(p.getInstance().size() > 0); + assertFalse(p.getInstance().isEmpty()); for (String url : p.getInstance().get(0).getUrl()) { System.out.println(url); - assertTrue(!url.contains("&")); + assertFalse(url.contains("&")); } } @@ -959,7 +959,7 @@ class MappersTest { assertTrue(o.isPresent()); Publication p = (Publication) o.get(); - assertTrue(p.getInstance().size() > 0); + assertFalse(p.getInstance().isEmpty()); assertEquals("https://doi.org/10.1155/2015/439379", p.getInstance().get(0).getUrl().get(0)); diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/synonyms.txt b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/synonyms.txt index b3fa94b20..a17a3949a 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/synonyms.txt +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/synonyms.txt @@ -1246,8 +1246,857 @@ dnet:relation_subRelType @=@ relationship @=@ publicationDataset dnet:provenanceActions @=@ iis @=@ erroneous label to be cleaned FOS @=@ 0101 mathematics @=@ FOS: Mathematics FOS @=@ 0102 computer and information sciences @=@ FOS: Computer and information sciences +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ddb1 @=@ Chemical Structures +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ddb1 @=@ Data Cube +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ddb1 @=@ Dataset/Dataset +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ddb1 @=@ Ensemble de données / Dataset +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ddb1 @=@ Research Data +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ddb1 @=@ dataset +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ddb1 @=@ http://purl.org/coar/resource_type/c_ddb1 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/ACF7-8YT9 @=@ collection +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/ACF7-8YT9 @=@ Collection of Datasets +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/ACF7-8YT9 @=@ RO-crate +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/ACF7-8YT9 @=@ Supplementary Collection of Datasets +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/ACF7-8YT9 @=@ Supplementary Dataset +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_12cc @=@ http://purl.org/coar/resource_type/c_12cc +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_12cc @=@ Planimetría +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_cb28 @=@ clinicalTrial +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_cb28 @=@ http://purl.org/coar/resource_type/c_cb28 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_12cd @=@ Carte géographique / Map +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_12cd @=@ http://purl.org/coar/resource_type/c_12cd +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_12cd @=@ map +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_12cd @=@ Mapa +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18cc @=@ Audio +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18cc @=@ http://purl.org/coar/resource_type/c_18cc +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18cc @=@ sound +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_86bc @=@ Book Prospectus +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_86bc @=@ Dictionary Entry +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_86bc @=@ Disclosure +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_86bc @=@ Literature review +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_86bc @=@ Reseña bibliográfica +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_86bc @=@ bibliography +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_86bc @=@ http://purl.org/coar/resource_type/c_86bc +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8a7e @=@ AUDIOVISUAL_DOCUMENT +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8a7e @=@ Audiovisual/Audiovisual +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8a7e @=@ http://purl.org/coar/resource_type/c_8a7e +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ Diagram +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ Drawing +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ Figure +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ Image/Image +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ Imagen - Image +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ Imagen 3-D +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ Imagen +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ Photo +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ Plot +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ StillImage|PRESERVED_SPECIMEN +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ fotó +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ grafika +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ http://purl.org/coar/resource_type/c_ecc8 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ image-diagram +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ image-drawing +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ image-figure +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ image-other +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ image-photo +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ image-plot +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ image +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_12ce @=@ film +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_12ce @=@ Film, vidéo / Motion picture, video +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_12ce @=@ http://purl.org/coar/resource_type/c_12ce +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_12ce @=@ Video +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_12ce @=@ vídeo +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/C53B-JCY5 @=@ H1 Myönnetty patentti +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/C53B-JCY5 @=@ http://purl.org/coar/resource_type/C53B-JCY5 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_15cd @=@ brevet +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_15cd @=@ http://purl.org/coar/resource_type/c_15cd +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_15cd @=@ Patent +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_15cd @=@ Patente +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_15cd @=@ Registered Copyright +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_15cd @=@ Traducción de patente +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/SB3Y-W4EH @=@ Solicitud de patente +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_e9a0 @=@ http://purl.org/coar/resource_type/c_e9a0 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_e9a0 @=@ interactiveResource +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_e059 @=@ http://purl.org/coar/resource_type/c_e059 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_e059 @=@ Learning Object +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_e059 @=@ learningObject +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_e059 @=@ PEDAGOGICAL_DOCUMENT +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ Autre / Other +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ EGI Virtual Appliance +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ Event/Event +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ Exhibition +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ Kita / Other +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ Model/Model +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ Otro - Other +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ Physical Object +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ Research Tool +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ Service +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ Trabajo de divulgación +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ UNKNOWN +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ University Academic Unit +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ application +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ artefact +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ carte +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ composition +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ corpus +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ event +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ http://purl.org/coar/resource_type/c_1843 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ http://purl.org/coar/resource_type/c_26e4 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ info:eu-repo/semantics/other +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ misc +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ model +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ other research product +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ other +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ otro +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ physicalObject +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ revue +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ sonstige Veröffentlichung +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ text/tg.edition+tg.aggregation+xml +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/9DKX-KSAF @=@ Modelo de utilidad +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7ad9 @=@ http://purl.org/coar/resource_type/c_7ad9 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7ad9 @=@ Online Resource +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7ad9 @=@ Sitio web +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7ad9 @=@ Web publication/site +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7877 @=@ Clinical Study +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7877 @=@ http://purl.org/coar/resource_type/c_7877 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Livre / Book +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Book (monograph) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Book (non peer-reviewed) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Book (peer-reviewed) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Book - monograph - editorial book +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Book as author +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Books +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Buch +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Edited Book +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ International Book/Monograph +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Knyga / Book +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Książka +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Lehr- oder Fachbuch +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Libro - Book +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Libro +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Monografia +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Monograph +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ National Book/Monograph +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ atlas +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ book-series +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ book-set +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ book-track +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ book +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ book_series +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ book_title +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ doc-type:book +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ eBook +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ edited-book +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ http://purl.org/coar/resource_type/c_2f33 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ http://purl.org/eprint/type/Book +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ info:eu-repo/semantics/book +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ könyv +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ ouvrage +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ publication-book +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ reference-book +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ scientific book +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Книга +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Учебник +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ Монография +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ A3 Kirjan tai muun kokoomateoksen osa +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Beitrag in einem Lehr- oder Fachbuch +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Book Part (author) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Book Section / Chapter +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Book Section +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Book chapter or Essay in book +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Book editorial +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Book section +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ BookChapter +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Book_Chapter +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Buchbeitrag +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Capítulo de Libro - Book Section +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Capítulo de libro +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Chapitre de livre / Book chapter +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Contribution to International Book/Monograph ISI/JCR +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Contribution to International Book/Monograph +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Contribution to National Book/Monograph +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Contribution to book (non peer-reviewed) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Contribution to book (peer-reviewed) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Knygos dalis / Book chapter +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Part of book - chapter +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Rozdział z książki +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Scientific publication - Book Chapter +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ book chapter +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ book part +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ book-chapter +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ book-part +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ book-section +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ bookPart +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ book_content +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ chapitre_ouvrage +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ chapter +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ doc-type:bookPart +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ http://purl.org/coar/resource_type/c_3248 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ http://purl.org/eprint/type/BookItem +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ info:eu-repo/semantics/Chapter +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ info:eu-repo/semantics/bookPart +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ könyvfejezet +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ publication-section +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ reference-entry +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ reference_entry +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ scientific book chapter +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ Глава монографии +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ba08 @=@ Book Review +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ba08 @=@ book-review +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ba08 @=@ http://purl.org/coar/resource_type/c_ba08 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ba08 @=@ info:eu-repo/semantics/bookreview +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ba08 @=@ reseña de libro +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ A4 Artikkeli konferenssijulkaisussa +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ Article in monograph or in proceedings +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ Article in proceedings +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ Conference Paper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ Conference article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ Conference papers +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ ConferencePaper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ PROCEEDING_PAPER +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ Papers in Conference Proceedings openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ Proceedings paper -openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ Conference article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ Scientific publication - Conference Paper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ Scientific publication - Conference Short Paper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ conference_paper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ http://purl.org/coar/resource_type/c_5794 openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ http://purl.org/eprint/type/ConferencePaper -openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ Conference article -openaire::user_resource_types @=@ Article @=@ http://purl.org/coar/resource_type/c_5794 \ No newline at end of file +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ http://purl.org/escidoc/metadata/ves/publication-types/conference-report +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ info:eu-repo/semantics/Conference Paper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ info:eu-repo/semantics/conferencePaper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ proceeding with peer review +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ proceedings-article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ publication-conferencepaper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ Επιστημονική δημοσίευση - Ανακοίνωση Συνεδρίου (Short Paper) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18cp @=@ Conference preprint +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18cp @=@ http://purl.org/coar/resource_type/c_18cp +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18co @=@ http://purl.org/coar/resource_type/c_18co +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7acd @=@ correction +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7acd @=@ corrigenda +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7acd @=@ http://purl.org/coar/resource_type/c_7acd +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7acd @=@ partial-retraction +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7acd @=@ reply +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7acd @=@ retraction +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ab20 @=@ Data Management Plan (NSF Generic) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ab20 @=@ Data Management Plan +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ab20 @=@ OutputManagementPlan +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ab20 @=@ http://purl.org/coar/resource_type/c_ab20 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ab20 @=@ http://purl.org/spar/fabio/DataMangementPlan +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ab20 @=@ plan de gestión de datos +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ab20 @=@ publication-datamanagementplan +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_beb9 @=@ Data Descriptor +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_beb9 @=@ data-article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_beb9 @=@ DataPaper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_beb9 @=@ http://purl.org/coar/resource_type/c_beb9 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/H9BQ-739P @=@ Peer review +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/H9BQ-739P @=@ peer-review +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_816b @=@ http://purl.org/coar/resource_type/c_816b +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_816b @=@ info:eu-repo/semantics/preprint +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_816b @=@ Pre Print +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_816b @=@ Pre-print +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_816b @=@ publication-preprint +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_816b @=@ Препринт +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ ACTIVITY_REPORT +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ Case Report +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ Commissioned report +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ Internal note +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ Rapport / Report +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ brief-report +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ case-report +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ chapitre_rapport +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ compte rendu +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ doc-type:report +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ document_institutionnel +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ http://purl.org/coar/resource_type/c_18hj +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ http://purl.org/coar/resource_type/c_18wq +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ http://purl.org/coar/resource_type/c_18ww +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ http://purl.org/coar/resource_type/c_93fc +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ http://purl.org/coar/resource_type/c_ba1f +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ http://purl.org/eprint/type/Report +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ info:eu-repo/semantics/report +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ publication-report +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ rapport_expertise +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ rapport_mission +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ report-paper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ report-paper_title +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ report-series +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ report +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ support_cours +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ A1 Alkuperäisartikkeli tieteellisessä aikakauslehdessä +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Article (author) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Article from Conference in a Journal +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Article in journal +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ ArticleArtikel +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Articles in Journals +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Articolo +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Artículo - Article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Artículo +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Articulo +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Aufsatz +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Journal article (on-line or printed) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Journal article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Journal articles +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Journal paper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ JournalArticle +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Makale +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Original article (non peer-reviewed) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Original article (peer-reviewed) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Peer-reviewed Article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Publication - Article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Published Journal Article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Research Article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Scientific publication - Journal Article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Straipsnis / Article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Taxonomic Paper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Zeitschriftenbeitrag +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ art +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ article in non peer-reviewed journal +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ article in peer-reviewed journal +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ article_site_web +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ artykuł +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ doc-type:Journal Article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ doc-type:article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ foly +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ folyóiratcikk +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ http://purl.org/coar/resource_type/c_6501 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ http://purl.org/escidoc/metadata/ves/publication-types/article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ http://purl.org/ontology/bibo/AcademicArticle +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ info:eu-repo/semantics/Journal Article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ info:eu-repo/semantics/article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ journal-article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ journal_article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ non peer-reviewed article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ publication-article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ rapid-communication +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ research-article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ text (article) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Статья +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ Рецензована стаття +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_dcae04bc @=@ A2 Katsausartikkeli tieteellisessä aikakauslehdessä +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_dcae04bc @=@ Journal Article/Review +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_dcae04bc @=@ Review article (non peer-reviewed) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_dcae04bc @=@ Review article (peer-reviewed) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_dcae04bc @=@ article-commentary +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_dcae04bc @=@ artículo de revisión +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_dcae04bc @=@ Review Article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_dcae04bc @=@ review-article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_dcae04bc @=@ Revisión +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_dcae04bc @=@ RezensionReview +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_dcae04bc @=@ http://purl.org/coar/resource_type/c_dcae04bc +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7bab @=@ http://purl.org/coar/resource_type/c_7bab +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_71bd @=@ Documento tecnico +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_71bd @=@ http://purl.org/coar/resource_type/c_71bd +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_71bd @=@ publication-softwaredocumentation +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_71bd @=@ Software documentation +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/H6QP-SC1X @=@ Trademark +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_393c @=@ http://purl.org/coar/resource_type/c_393c +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_393c @=@ Workflow +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_393c @=@ Workflow/Workflow +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7a1f @=@ Bachelor's Degree +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7a1f @=@ Bachelor's +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7a1f @=@ Bachelors Thesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7a1f @=@ Graduate Thesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7a1f @=@ Proyecto fin de carrera +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7a1f @=@ Undergraduate Thesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7a1f @=@ Undergraduate diploma +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7a1f @=@ bachelor thesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7a1f @=@ http://purl.org/coar/resource_type/c_7a1f +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7a1f @=@ info:eu-repo/semantics/bachelorThesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7a1f @=@ выпускная бакалаврская работа +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ Daktaro disertacija / Doctoral dissertation +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ Diss +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ Dissertation +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ Doctoral Dissertation +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ Doctoral +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ DoctoralThesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ HabilitationThesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ PhD Theses +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ PhD thesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ Tenure-Promotion +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ Tesi di dottorato +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ Tesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ Text.Thesis.Doctoral +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ Theses +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ Thesis or Dissertation +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ Thesis.Doctoral +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ Thesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ doc-type:doctoralThesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ doctoral thesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ dok +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ doktori dolgozat +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ http://purl.org/coar/resource_type/c_db06 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ http://purl.org/eprint/type/Thesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ info:eu-repo/semantics/doctoralThesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ publication-thesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ tesis doctoral +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ these exercice +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ these +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ Diploma Project +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ MSc Thesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ Magistro darbas / Master thesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ Master Degree +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ Master's Degree +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ Master's +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ Masterarbeit u.a. +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ Masters (Taught) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ Masters thesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ Masters-Thesis.Magister +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ Tesina +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ Thesis.Master +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ Trabajo fin de Máster +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ doc-type:masterThesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ hdr +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ http://purl.org/coar/resource_type/c_bdcc +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ info:eu-repo/semantics/masterThesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ master thesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ masterThesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ memoire +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ tesis de maestría +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18wz @=@ http://purl.org/coar/resource_type/c_18wz +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_186u @=@ http://purl.org/coar/resource_type/c_186u +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_186u @=@ http://purl.org/spar/fabio/DataManagementPolicy +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_186u @=@ http://purl.org/spar/fabio/DataManagementPolicyDocument +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_186u @=@ JRC Reference Reports +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18op @=@ Deliverable +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18op @=@ http://purl.org/coar/resource_type/c_18op +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18op @=@ Project deliverable +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18op @=@ Project Report +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18op @=@ publication-deliverable +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18ws @=@ D4 Julkaistu kehittämis- tai tutkimusraportti tai -selvitys +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18ws @=@ EUR - Scientific and Technical Research Reports +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18ws @=@ Project milestone +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18ws @=@ RESEARCH_REPORT +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18ws @=@ http://purl.org/coar/resource_type/c_18ws +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18ws @=@ research report +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18ws @=@ научный доклад +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18gh @=@ Departmental Technical Report +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18gh @=@ Informe Técnico +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18gh @=@ Tech-Report +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18gh @=@ Technical Report +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18gh @=@ document_technique +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18gh @=@ http://purl.org/coar/resource_type/c_18gh +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18gh @=@ informe a organismo financiador +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18gh @=@ publication-technicalnote +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_46ec @=@ Graduate diploma +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_46ec @=@ Hochschulschrift +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_46ec @=@ Tesis/trabajos de grado – Thesis +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_46ec @=@ Thèse ou mémoire / Thesis or Dissertation +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8042 @=@ Arbeitspapier +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8042 @=@ Departmental Bulletin Paper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8042 @=@ Documento de trabajo +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8042 @=@ Paper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8042 @=@ Project description +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8042 @=@ Research-Paper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8042 @=@ ResearchPaper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8042 @=@ Working / discussion paper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8042 @=@ Working Paper / Technical Report +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8042 @=@ Working Paper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8042 @=@ doc-type:workingPaper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8042 @=@ http://purl.org/coar/resource_type/c_8042 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8042 @=@ http://purl.org/escidoc/metadata/ves/publication-types/paper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8042 @=@ info:eu-repo/semantics/paper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8042 @=@ info:eu-repo/semantics/workingPaper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8042 @=@ publication-workingpaper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8042 @=@ workingPaper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_6947 @=@ Blog +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_6947 @=@ Entrada de blog +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_6947 @=@ http://purl.org/coar/resource_type/c_6947 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ Conference Abstract +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ Conference Paper/Proceeding/Abstract +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ Conference Program +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ Conference contribution +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ Conference or Workshop Item +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ Conference paper, poster, etc. +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ Conference paper/abstract +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ Conference report +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ Contribution à un congrès / Conference object +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ Contributions to Conferences +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ International Conference Abstract/Poster +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ International Conference ISI/JCR +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ International Conference communication/abstract/poster +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ Konferenzbeitrag +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ Resumen comunicación Congreso +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ Resúmen comunicación Congreso +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ communication_invitee +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ communication_sans_actes +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ conference item +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ conference object +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ conference +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ conferenceObject +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ doc-type:conferenceObject +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ http://purl.org/coar/resource_type/c_c94f +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ info:eu-repo/semantics/conferenceItem +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ info:eu-repo/semantics/conferenceObject +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ info:eu-repo/semantics/conferenceitemnotinproceedings +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_6670 @=@ Comunicación Congreso +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_6670 @=@ Comunicación de congreso +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_6670 @=@ Conference Poster +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_6670 @=@ National Conference Abstract/Poster +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_6670 @=@ National Conference communication/abstract/poster +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_6670 @=@ Póster de congreso +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_6670 @=@ Póster +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_6670 @=@ Scientific publication - Conference Poster +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_6670 @=@ http://purl.org/coar/resource_type/c_6670 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_6670 @=@ http://purl.org/eprint/type/ConferencePoster +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_6670 @=@ plakat +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_6670 @=@ poster +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/R60J-J5BD @=@ Conference lecture +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/R60J-J5BD @=@ Ponencia - Conference or Workshop Item +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/R60J-J5BD @=@ Presentación +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/R60J-J5BD @=@ Presentation +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/R60J-J5BD @=@ invited conference talk +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/R60J-J5BD @=@ ponencia +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/R60J-J5BD @=@ prezentacja +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ Actes de congrès / Conference proceedings +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ Conference Proceedings +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ GL4 Conference Proceedings +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ International Conference +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ National Conference +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ PREFACE_PROCEEDINGS +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ Proceedings (peer-reviewed) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ Proceedings of a Conference +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ Tagungsband +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ actas de congreso +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ actes_congres +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ communication_avec_actes +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ communication_par_affiche +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ conference proceeding +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ http://purl.org/coar/resource_type/c_f744 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ http://purl.org/escidoc/metadata/ves/publication-types/proceedings +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ info:eu-repo/semantics/conferenceproceedings +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ proceeding, seminar, workshop without peer review +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ proceedings +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_b239 @=@ Editorial +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_b239 @=@ Editorial ISI/JCR +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_b239 @=@ Editors +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_b239 @=@ Editors (non peer-reviewed) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_b239 @=@ Editors (peer-reviewed) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_b239 @=@ http://purl.org/coar/resource_type/c_b239 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0640 @=@ http://purl.org/coar/resource_type/c_0640 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0640 @=@ International Journal +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0640 @=@ International Journal ISI/JCR +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0640 @=@ Journal (full / special issue) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0640 @=@ National Journal +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0857 @=@ habilitation à diriger des recherches +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0857 @=@ http://purl.org/coar/resource_type/c_0857 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_545b @=@ Article - letter to the editor +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_545b @=@ Article / Letter to editor +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_545b @=@ Article / Letter to the editor +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_545b @=@ Article-letter to the editor +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_545b @=@ Article/Letter to editor +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_545b @=@ http://purl.org/coar/resource_type/c_545b +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_545b @=@ letter +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2cd9 @=@ Magazine Article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2cd9 @=@ Revista +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2cd9 @=@ Sammelband +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2cd9 @=@ revista divulgativa +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0040 @=@ foreword +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0040 @=@ Abstract +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0040 @=@ Beitrag im Sammelband +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0040 @=@ Manuscript +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0040 @=@ Manuscrito +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0040 @=@ afterword +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0040 @=@ avantpropos +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0040 @=@ http://purl.org/coar/resource_type/c_0040 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0040 @=@ http://purl.org/coar/resource_type/c_3e5a +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0040 @=@ postface +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2fe3 @=@ revuedepresse +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_998f @=@ Article / Newspaper +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_998f @=@ Newspaper Article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_998f @=@ Newspaper or magazine article +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_998f @=@ http://purl.org/coar/resource_type/c_998f +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_998f @=@ in-brief +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ Berichtsreihe +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ Encyclopedia Entry +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ Funding Submission +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ Índice +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ License +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ Manual +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ Other publication (non peer-review) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ Other publication (peer-review) +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ Supervised Student Publication +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ Text/Text +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ Text +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ Translation +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ Travail étudiant / Student work +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ chronique +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ contributionToPeriodical +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ historicalDocument +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ info:eu-repo/semantics/contributionToPeriodical +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ literature +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ other publication +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ publication-other +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ sa_component +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ standard-series +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ standard +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_baaf @=@ Proposal +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_baaf @=@ http://purl.org/coar/resource_type/c_baaf +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_baaf @=@ research-proposal +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_efa0 @=@ Book/Film/Article review +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_efa0 @=@ http://purl.org/coar/resource_type/c_efa0 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_efa0 @=@ info:eu-repo/semantics/review +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18cf @=@ http://purl.org/coar/resource_type/c_18cf +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1162 @=@ Comentario +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1162 @=@ Comment/debate +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1162 @=@ annotation +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1162 @=@ http://purl.org/coar/resource_type/c_1162 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1162 @=@ info:eu-repo/semantics/annotation +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8544 @=@ Inaugural lecture +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8544 @=@ Material didáctico +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8544 @=@ Matériel didactique / Educational material +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8544 @=@ Public-Lecture +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8544 @=@ Teaching Resource +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8544 @=@ http://purl.org/coar/resource_type/c_8544 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8544 @=@ info:eu-repo/semantics/lecture +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8544 @=@ lesson +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8544 @=@ note de lecture +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8544 @=@ notedelecture +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8544 @=@ Учебный материал +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18cd @=@ Partitura +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18cd @=@ document_audiovisuel +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18cd @=@ http://purl.org/coar/resource_type/c_18cd +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18cw @=@ Estudio y edición crítica de música +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18cw @=@ http://purl.org/coar/resource_type/c_18cw +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c950 @=@ Software/Software +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c950 @=@ Software +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c950 @=@ http://purl.org/coar/resource_type/c_c950 +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5ce6 @=@ Jupyter Notebook +openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5ce6 @=@ http://purl.org/coar/resource_type/c_5ce6 +openaire::user_resource_types @=@ Article @=@ text:conference_output:conference_proceedings:conference_paper +openaire::user_resource_types @=@ Article @=@ http://purl.org/coar/resource_type/c_5794 +openaire::user_resource_types @=@ Article @=@ text:conference_output:conference_paper_not_in_proceedings +openaire::user_resource_types @=@ Article @=@ http://purl.org/coar/resource_type/c_18cp +openaire::user_resource_types @=@ Article @=@ text:conference_output:conference_poster_not_in_proceedings +openaire::user_resource_types @=@ Article @=@ http://purl.org/coar/resource_type/c_18co +openaire::user_resource_types @=@ Article @=@ text:journal:journal_article:corrigendum +openaire::user_resource_types @=@ Article @=@ http://purl.org/coar/resource_type/c_7acd +openaire::user_resource_types @=@ Article @=@ text:journal:journal_article:data_paper +openaire::user_resource_types @=@ Article @=@ http://purl.org/coar/resource_type/c_beb9 +openaire::user_resource_types @=@ Article @=@ text:journal:journal_article +openaire::user_resource_types @=@ Article @=@ http://purl.org/coar/resource_type/c_6501 +openaire::user_resource_types @=@ Article @=@ text:preprint +openaire::user_resource_types @=@ Article @=@ http://purl.org/coar/resource_type/c_816b +openaire::user_resource_types @=@ Article @=@ text:journal:journal_article:research_article +openaire::user_resource_types @=@ Article @=@ http://purl.org/coar/resource_type/c_2df8fbb1 +openaire::user_resource_types @=@ Article @=@ text:journal:journal_article:review_article +openaire::user_resource_types @=@ Article @=@ http://purl.org/coar/resource_type/c_dcae04bc +openaire::user_resource_types @=@ Article @=@ text:journal:journal_article:software_paper +openaire::user_resource_types @=@ Article @=@ http://purl.org/coar/resource_type/c_7bab +openaire::user_resource_types @=@ Article @=@ text:working_paper +openaire::user_resource_types @=@ Article @=@ http://purl.org/coar/resource_type/c_8042 +openaire::user_resource_types @=@ Thesis @=@ text:thesis:bachelor_thesis +openaire::user_resource_types @=@ Thesis @=@ http://purl.org/coar/resource_type/c_7a1f +openaire::user_resource_types @=@ Thesis @=@ text:thesis:doctoral_thesis +openaire::user_resource_types @=@ Thesis @=@ http://purl.org/coar/resource_type/c_db06 +openaire::user_resource_types @=@ Thesis @=@ text:thesis:master_thesis +openaire::user_resource_types @=@ Thesis @=@ http://purl.org/coar/resource_type/c_bdcc +openaire::user_resource_types @=@ Thesis @=@ text:thesis +openaire::user_resource_types @=@ Thesis @=@ http://purl.org/coar/resource_type/c_46ec +openaire::user_resource_types @=@ Report @=@ text:report +openaire::user_resource_types @=@ Report @=@ http://purl.org/coar/resource_type/c_93fc +openaire::user_resource_types @=@ Report @=@ text:report:memorandum +openaire::user_resource_types @=@ Report @=@ http://purl.org/coar/resource_type/c_18wz +openaire::user_resource_types @=@ Report @=@ text:report:policy_report +openaire::user_resource_types @=@ Report @=@ http://purl.org/coar/resource_type/c_186u +openaire::user_resource_types @=@ Report @=@ text:report:research_protocol +openaire::user_resource_types @=@ Report @=@ http://purl.org/coar/resource_type/YZ1N-ZFT9 +openaire::user_resource_types @=@ Report @=@ text:report:research_report +openaire::user_resource_types @=@ Report @=@ http://purl.org/coar/resource_type/c_18ws +openaire::user_resource_types @=@ Report @=@ text:report:technical_report +openaire::user_resource_types @=@ Report @=@ http://purl.org/coar/resource_type/c_18gh +openaire::user_resource_types @=@ Project result @=@ text:report:data_management_plan +openaire::user_resource_types @=@ Project result @=@ http://purl.org/coar/resource_type/c_ab20 +openaire::user_resource_types @=@ Project result @=@ text:report:project_deliverable +openaire::user_resource_types @=@ Project result @=@ http://purl.org/coar/resource_type/c_18op +openaire::user_resource_types @=@ Book/Chapter @=@ text:book +openaire::user_resource_types @=@ Book/Chapter @=@ http://purl.org/coar/resource_type/c_2f33 +openaire::user_resource_types @=@ Book/Chapter @=@ text:book:book_part +openaire::user_resource_types @=@ Book/Chapter @=@ http://purl.org/coar/resource_type/c_3248 +openaire::user_resource_types @=@ Clinical Study @=@ text:report:clinical_study +openaire::user_resource_types @=@ Clinical Study @=@ http://purl.org/coar/resource_type/c_7877 +openaire::meta_resource_types @=@ Research Literature @=@ text:bibliography +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_86bc +openaire::meta_resource_types @=@ Research Literature @=@ patent:design_patent +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/C53B-JCY5 +openaire::meta_resource_types @=@ Research Literature @=@ patent +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_15cd +openaire::meta_resource_types @=@ Research Literature @=@ patent:PCT_application +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/SB3Y-W4EH +openaire::meta_resource_types @=@ Research Literature @=@ patent:plant_patent +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/Z907-YMBB +openaire::meta_resource_types @=@ Research Literature @=@ patent:plant_variety_protection +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/GPQ7-G5VE +openaire::meta_resource_types @=@ Research Literature @=@ patent:software_patent +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/MW8G-3CR8 +openaire::meta_resource_types @=@ Research Literature @=@ patent:utility_model +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/9DKX-KSAF +openaire::meta_resource_types @=@ Research Literature @=@ text:report:clinical_study +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_7877 +openaire::meta_resource_types @=@ Research Literature @=@ text:book +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_2f33 +openaire::meta_resource_types @=@ Research Literature @=@ text:book:book_part +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_3248 +openaire::meta_resource_types @=@ Research Literature @=@ text:review:book_review +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_ba08 +openaire::meta_resource_types @=@ Research Literature @=@ text:review:commentary +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/D97F-VB57 +openaire::meta_resource_types @=@ Research Literature @=@ text:conference_output:conference_proceedings:conference_paper +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_5794 +openaire::meta_resource_types @=@ Research Literature @=@ text:conference_output:conference_paper_not_in_proceedings +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_18cp +openaire::meta_resource_types @=@ Research Literature @=@ text:conference_output:conference_poster_not_in_proceedings +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_18co +openaire::meta_resource_types @=@ Research Literature @=@ text:journal:journal_article:corrigendum +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_7acd +openaire::meta_resource_types @=@ Research Literature @=@ text:report:data_management_plan +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_ab20 +openaire::meta_resource_types @=@ Research Literature @=@ text:journal:journal_article:data_paper +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_beb9 +openaire::meta_resource_types @=@ Research Literature @=@ text:journal:journal_article +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_6501 +openaire::meta_resource_types @=@ Research Literature @=@ text:review:peer_review +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/H9BQ-739P +openaire::meta_resource_types @=@ Research Literature @=@ text:preprint +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_816b +openaire::meta_resource_types @=@ Research Literature @=@ text:report +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_93fc +openaire::meta_resource_types @=@ Research Literature @=@ text:journal:journal_article:research_article +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_2df8fbb1 +openaire::meta_resource_types @=@ Research Literature @=@ text:journal:journal_article:review_article +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_dcae04bc +openaire::meta_resource_types @=@ Research Literature @=@ text:journal:journal_article:software_paper +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_7bab +openaire::meta_resource_types @=@ Research Literature @=@ text:technical_documentation +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_71bd +openaire::meta_resource_types @=@ Research Literature @=@ text:transcription +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/6NC7-GK9S +openaire::meta_resource_types @=@ Research Literature @=@ text:thesis:bachelor_thesis +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_7a1f +openaire::meta_resource_types @=@ Research Literature @=@ text:thesis:doctoral_thesis +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_db06 +openaire::meta_resource_types @=@ Research Literature @=@ text:thesis:master_thesis +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_bdcc +openaire::meta_resource_types @=@ Research Literature @=@ text:report:memorandum +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_18wz +openaire::meta_resource_types @=@ Research Literature @=@ text:report:policy_report +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_186u +openaire::meta_resource_types @=@ Research Literature @=@ text:report:project_deliverable +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_18op +openaire::meta_resource_types @=@ Research Literature @=@ text:report:research_protocol +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/YZ1N-ZFT9 +openaire::meta_resource_types @=@ Research Literature @=@ text:report:research_report +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_18ws +openaire::meta_resource_types @=@ Research Literature @=@ text:report:technical_report +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_18gh +openaire::meta_resource_types @=@ Research Literature @=@ text:thesis +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_46ec +openaire::meta_resource_types @=@ Research Literature @=@ text:working_paper +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_8042 +openaire::meta_resource_types @=@ Research Literature @=@ text:blog_post +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_6947 +openaire::meta_resource_types @=@ Research Literature @=@ text:conference_output +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_c94f +openaire::meta_resource_types @=@ Research Literature @=@ text:conference_output:conference_proceedings:conference_poster +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_6670 +openaire::meta_resource_types @=@ Research Literature @=@ text:conference_output:conference_presentation +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/R60J-J5BD +openaire::meta_resource_types @=@ Research Literature @=@ text:conference_output:conference_proceedings +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_f744 +openaire::meta_resource_types @=@ Research Literature @=@ text:journal:editorial +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_b239 +openaire::meta_resource_types @=@ Research Literature @=@ text:journal +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_0640 +openaire::meta_resource_types @=@ Research Literature @=@ text:letter +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_0857 +openaire::meta_resource_types @=@ Research Literature @=@ text:journal:letter_to_the_editor +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_545b +openaire::meta_resource_types @=@ Research Literature @=@ text:magazine +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_2cd9 +openaire::meta_resource_types @=@ Research Literature @=@ text:manuscript +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_0040 +openaire::meta_resource_types @=@ Research Literature @=@ text:newspaper +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_2fe3 +openaire::meta_resource_types @=@ Research Literature @=@ text:newspaper:newspaper_article +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_998f +openaire::meta_resource_types @=@ Research Literature @=@ text:other_periodical +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/QX5C-AR31 +openaire::meta_resource_types @=@ Research Literature @=@ text:research_proposal +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_baaf +openaire::meta_resource_types @=@ Research Literature @=@ text:review +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_efa0 +openaire::meta_resource_types @=@ Research Literature @=@ text +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_18cf +openaire::meta_resource_types @=@ Research Literature @=@ text:annotation +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_1162 +openaire::meta_resource_types @=@ Research Literature @=@ text:lecture +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_8544 +openaire::meta_resource_types @=@ Research Literature @=@ text:musical_notation +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_18cw +openaire::meta_resource_types @=@ Research Data @=@ dataset +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_ddb1 +openaire::meta_resource_types @=@ Research Data @=@ dataset:aggregated_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/ACF7-8YT9 +openaire::meta_resource_types @=@ Research Data @=@ cartographic_material +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_12cc +openaire::meta_resource_types @=@ Research Data @=@ dataset:clinical_trial_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_cb28 +openaire::meta_resource_types @=@ Research Data @=@ dataset:compiled_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/FXF3-D3G7 +openaire::meta_resource_types @=@ Research Data @=@ dataset:encoded_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/AM6W-6QAW +openaire::meta_resource_types @=@ Research Data @=@ dataset:experimental_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/63NG-B465 +openaire::meta_resource_types @=@ Research Data @=@ dataset:genomic_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/A8F1-NPV9 +openaire::meta_resource_types @=@ Research Data @=@ dataset:geospatial_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/2H0M-X761 +openaire::meta_resource_types @=@ Research Data @=@ dataset:laboratory_notebook +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/H41Y-FW7B +openaire::meta_resource_types @=@ Research Data @=@ cartographic_material:map +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_12cd +openaire::meta_resource_types @=@ Research Data @=@ dataset:measurement_and_test_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/DD58-GFSX +openaire::meta_resource_types @=@ Research Data @=@ dataset:observational_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/FF4C-28RK +openaire::meta_resource_types @=@ Research Data @=@ dataset:recorded_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/CQMR-7K63 +openaire::meta_resource_types @=@ Research Data @=@ dataset:simulation_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/W2XT-7017 +openaire::meta_resource_types @=@ Research Data @=@ sound +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_18cc +openaire::meta_resource_types @=@ Research Data @=@ dataset:survey_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/NHD0-W6SY +openaire::meta_resource_types @=@ Research Data @=@ image +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_c513 +openaire::meta_resource_types @=@ Research Data @=@ image:moving_image +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_8a7e +openaire::meta_resource_types @=@ Research Data @=@ image:still_image +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_ecc8 +openaire::meta_resource_types @=@ Research Data @=@ image:moving_image:video +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_12ce +openaire::meta_resource_types @=@ Research Data @=@ design +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/542X-3S04 +openaire::meta_resource_types @=@ Research Data @=@ design:industrial_design +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/JBNF-DYAD +openaire::meta_resource_types @=@ Research Data @=@ interactive_resource +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_e9a0 +openaire::meta_resource_types @=@ Research Data @=@ design:layout_design +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/BW7T-YM2G +openaire::meta_resource_types @=@ Research Data @=@ interactive_resource:website +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_7ad9 +openaire::meta_resource_types @=@ Research Data @=@ sound:musical_composition +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_18cd +openaire::meta_resource_types @=@ Research Software @=@ software:research_software +openaire::meta_resource_types @=@ Research Software @=@ http://purl.org/coar/resource_type/c_c950 +openaire::meta_resource_types @=@ Research Software @=@ software +openaire::meta_resource_types @=@ Research Software @=@ http://purl.org/coar/resource_type/c_5ce6 +openaire::meta_resource_types @=@ Research Software @=@ software:source_code +openaire::meta_resource_types @=@ Research Software @=@ http://purl.org/coar/resource_type/QH80-2R4E +openaire::meta_resource_types @=@ Other Research Products @=@ learning_object +openaire::meta_resource_types @=@ Other Research Products @=@ http://purl.org/coar/resource_type/c_e059 +openaire::meta_resource_types @=@ Other Research Products @=@ other +openaire::meta_resource_types @=@ Other Research Products @=@ http://purl.org/coar/resource_type/c_1843 +openaire::meta_resource_types @=@ Other Research Products @=@ trademark +openaire::meta_resource_types @=@ Other Research Products @=@ http://purl.org/coar/resource_type/H6QP-SC1X +openaire::meta_resource_types @=@ Other Research Products @=@ workflow +openaire::meta_resource_types @=@ Other Research Products @=@ http://purl.org/coar/resource_type/c_393c \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/terms.txt b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/terms.txt index a38c0e987..68828b3d8 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/terms.txt +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/terms.txt @@ -1122,5 +1122,295 @@ dnet:relation_subRelType @=@ dnet:relation_subRelType @=@ version @=@ version FOS @=@ Fields of Science and Technology classification @=@ 0101 mathematics @=@ 0101 mathematics FOS @=@ Fields of Science and Technology classification @=@ 0102 computer and information sciences @=@ 0102 computer and information sciences FOS @=@ Fields of Science and Technology classification @=@ 0103 physical sciences @=@ 0103 physical sciences +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ddb1 @=@ dataset +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/ACF7-8YT9 @=@ aggregated data +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_12cc @=@ cartographic material +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_cb28 @=@ clinical trial data +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/FXF3-D3G7 @=@ compiled data +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/AM6W-6QAW @=@ encoded data +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/63NG-B465 @=@ experimental data +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/A8F1-NPV9 @=@ genomic data +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/2H0M-X761 @=@ geospatial data +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/H41Y-FW7B @=@ laboratory notebook +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_12cd @=@ map +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/DD58-GFSX @=@ measurement and test data +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/FF4C-28RK @=@ observational data +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/CQMR-7K63 @=@ recorded data +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/W2XT-7017 @=@ simulation data +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18cc @=@ sound +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/NHD0-W6SY @=@ survey data +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_86bc @=@ bibliography +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c513 @=@ image +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8a7e @=@ moving image +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ecc8 @=@ still image +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_12ce @=@ video +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/C53B-JCY5 @=@ design patent +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_15cd @=@ patent +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/SB3Y-W4EH @=@ PCT application +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/Z907-YMBB @=@ plant patent +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/GPQ7-G5VE @=@ plant variety protection +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/MW8G-3CR8 @=@ software patent +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/542X-3S04 @=@ design +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/JBNF-DYAD @=@ industrial design +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_e9a0 @=@ interactive resource +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/BW7T-YM2G @=@ layout design +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_e059 @=@ learning object +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1843 @=@ other +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/9DKX-KSAF @=@ utility model +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7ad9 @=@ website +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7877 @=@ clinical study +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2f33 @=@ book +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_3248 @=@ book part +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ba08 @=@ book review +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/D97F-VB57 @=@ commentary openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5794 @=@ conference paper -openaire::user_resource_types @=@ openaire::user_resource_types @=@ Article @=@ Article \ No newline at end of file +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18cp @=@ conference paper not in proceedings +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18co @=@ conference poster not in proceedings +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7acd @=@ corrigendum +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_ab20 @=@ data management plan +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_beb9 @=@ data paper +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_6501 @=@ journal article +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/H9BQ-739P @=@ peer review +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_816b @=@ preprint +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_93fc @=@ report +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2df8fbb1 @=@ research article +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_dcae04bc @=@ review article +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7bab @=@ software paper +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_71bd @=@ technical documentation +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/6NC7-GK9S @=@ transcription +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/H6QP-SC1X @=@ trademark +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_393c @=@ workflow +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_7a1f @=@ bachelor thesis +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_db06 @=@ doctoral thesis +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_bdcc @=@ master thesis +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18wz @=@ memorandum +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_186u @=@ policy report +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18op @=@ project deliverable +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/YZ1N-ZFT9 @=@ research protocol +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18ws @=@ research report +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18gh @=@ technical report +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_46ec @=@ thesis +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8042 @=@ working paper +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_6947 @=@ blog post +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c94f @=@ conference output +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_6670 @=@ conference poster +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/R60J-J5BD @=@ conference presentation +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_f744 @=@ conference proceedings +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_b239 @=@ editorial +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0640 @=@ journal +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0857 @=@ letter +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_545b @=@ letter to the editor +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2cd9 @=@ magazine +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_0040 @=@ manuscript +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_2fe3 @=@ newspaper +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_998f @=@ newspaper article +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QX5C-AR31 @=@ other periodical +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_baaf @=@ research proposal +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_efa0 @=@ review +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18cf @=@ text +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_1162 @=@ annotation +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_8544 @=@ lecture +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18cd @=@ musical composition +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_18cw @=@ musical notation +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_c950 @=@ research software +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/c_5ce6 @=@ software +openaire::coar_resource_types_3_1 @=@ openaire::coar_resource_types_3_1 @=@ http://purl.org/coar/resource_type/QH80-2R4E @=@ source code +openaire::user_resource_types @=@ openaire::user_resource_types @=@ Article @=@ Article +openaire::user_resource_types @=@ openaire::user_resource_types @=@ Thesis @=@ Thesis +openaire::user_resource_types @=@ openaire::user_resource_types @=@ Report @=@ Report +openaire::user_resource_types @=@ openaire::user_resource_types @=@ Project result @=@ Project result +openaire::user_resource_types @=@ openaire::user_resource_types @=@ Research Data @=@ Research Data +openaire::user_resource_types @=@ openaire::user_resource_types @=@ Presentation @=@ Presentation +openaire::user_resource_types @=@ openaire::user_resource_types @=@ Book/Chapter @=@ Book/Chapter +openaire::user_resource_types @=@ openaire::user_resource_types @=@ Multimedia @=@ Multimedia +openaire::user_resource_types @=@ openaire::user_resource_types @=@ Clinical Study @=@ Clinical Study +openaire::meta_resource_types @=@ openaire::meta_resource_types @=@ Research Literature @=@ Research Literature +openaire::meta_resource_types @=@ openaire::meta_resource_types @=@ Research Data @=@ Research Data +openaire::meta_resource_types @=@ openaire::meta_resource_types @=@ Research Software @=@ Research Software +openaire::meta_resource_types @=@ openaire::meta_resource_types @=@ Other Research Products @=@ Other Research Products +openaire::meta_resource_types @=@ Research Literature @=@ text:bibliography +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_86bc +openaire::meta_resource_types @=@ Research Literature @=@ patent:design_patent +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/C53B-JCY5 +openaire::meta_resource_types @=@ Research Literature @=@ patent +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_15cd +openaire::meta_resource_types @=@ Research Literature @=@ patent:PCT_application +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/SB3Y-W4EH +openaire::meta_resource_types @=@ Research Literature @=@ patent:plant_patent +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/Z907-YMBB +openaire::meta_resource_types @=@ Research Literature @=@ patent:plant_variety_protection +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/GPQ7-G5VE +openaire::meta_resource_types @=@ Research Literature @=@ patent:software_patent +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/MW8G-3CR8 +openaire::meta_resource_types @=@ Research Literature @=@ patent:utility_model +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/9DKX-KSAF +openaire::meta_resource_types @=@ Research Literature @=@ text:report:clinical_study +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_7877 +openaire::meta_resource_types @=@ Research Literature @=@ text:book +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_2f33 +openaire::meta_resource_types @=@ Research Literature @=@ text:book:book_part +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_3248 +openaire::meta_resource_types @=@ Research Literature @=@ text:review:book_review +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_ba08 +openaire::meta_resource_types @=@ Research Literature @=@ text:review:commentary +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/D97F-VB57 +openaire::meta_resource_types @=@ Research Literature @=@ text:conference_output:conference_proceedings:conference_paper +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_5794 +openaire::meta_resource_types @=@ Research Literature @=@ text:conference_output:conference_paper_not_in_proceedings +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_18cp +openaire::meta_resource_types @=@ Research Literature @=@ text:conference_output:conference_poster_not_in_proceedings +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_18co +openaire::meta_resource_types @=@ Research Literature @=@ text:journal:journal_article:corrigendum +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_7acd +openaire::meta_resource_types @=@ Research Literature @=@ text:report:data_management_plan +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_ab20 +openaire::meta_resource_types @=@ Research Literature @=@ text:journal:journal_article:data_paper +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_beb9 +openaire::meta_resource_types @=@ Research Literature @=@ text:journal:journal_article +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_6501 +openaire::meta_resource_types @=@ Research Literature @=@ text:review:peer_review +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/H9BQ-739P +openaire::meta_resource_types @=@ Research Literature @=@ text:preprint +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_816b +openaire::meta_resource_types @=@ Research Literature @=@ text:report +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_93fc +openaire::meta_resource_types @=@ Research Literature @=@ text:journal:journal_article:research_article +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_2df8fbb1 +openaire::meta_resource_types @=@ Research Literature @=@ text:journal:journal_article:review_article +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_dcae04bc +openaire::meta_resource_types @=@ Research Literature @=@ text:journal:journal_article:software_paper +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_7bab +openaire::meta_resource_types @=@ Research Literature @=@ text:technical_documentation +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_71bd +openaire::meta_resource_types @=@ Research Literature @=@ text:transcription +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/6NC7-GK9S +openaire::meta_resource_types @=@ Research Literature @=@ text:thesis:bachelor_thesis +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_7a1f +openaire::meta_resource_types @=@ Research Literature @=@ text:thesis:doctoral_thesis +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_db06 +openaire::meta_resource_types @=@ Research Literature @=@ text:thesis:master_thesis +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_bdcc +openaire::meta_resource_types @=@ Research Literature @=@ text:report:memorandum +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_18wz +openaire::meta_resource_types @=@ Research Literature @=@ text:report:policy_report +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_186u +openaire::meta_resource_types @=@ Research Literature @=@ text:report:project_deliverable +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_18op +openaire::meta_resource_types @=@ Research Literature @=@ text:report:research_protocol +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/YZ1N-ZFT9 +openaire::meta_resource_types @=@ Research Literature @=@ text:report:research_report +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_18ws +openaire::meta_resource_types @=@ Research Literature @=@ text:report:technical_report +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_18gh +openaire::meta_resource_types @=@ Research Literature @=@ text:thesis +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_46ec +openaire::meta_resource_types @=@ Research Literature @=@ text:working_paper +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_8042 +openaire::meta_resource_types @=@ Research Literature @=@ text:blog_post +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_6947 +openaire::meta_resource_types @=@ Research Literature @=@ text:conference_output +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_c94f +openaire::meta_resource_types @=@ Research Literature @=@ text:conference_output:conference_proceedings:conference_poster +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_6670 +openaire::meta_resource_types @=@ Research Literature @=@ text:conference_output:conference_presentation +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/R60J-J5BD +openaire::meta_resource_types @=@ Research Literature @=@ text:conference_output:conference_proceedings +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_f744 +openaire::meta_resource_types @=@ Research Literature @=@ text:journal:editorial +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_b239 +openaire::meta_resource_types @=@ Research Literature @=@ text:journal +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_0640 +openaire::meta_resource_types @=@ Research Literature @=@ text:letter +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_0857 +openaire::meta_resource_types @=@ Research Literature @=@ text:journal:letter_to_the_editor +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_545b +openaire::meta_resource_types @=@ Research Literature @=@ text:magazine +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_2cd9 +openaire::meta_resource_types @=@ Research Literature @=@ text:manuscript +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_0040 +openaire::meta_resource_types @=@ Research Literature @=@ text:newspaper +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_2fe3 +openaire::meta_resource_types @=@ Research Literature @=@ text:newspaper:newspaper_article +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_998f +openaire::meta_resource_types @=@ Research Literature @=@ text:other_periodical +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/QX5C-AR31 +openaire::meta_resource_types @=@ Research Literature @=@ text:research_proposal +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_baaf +openaire::meta_resource_types @=@ Research Literature @=@ text:review +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_efa0 +openaire::meta_resource_types @=@ Research Literature @=@ text +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_18cf +openaire::meta_resource_types @=@ Research Literature @=@ text:annotation +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_1162 +openaire::meta_resource_types @=@ Research Literature @=@ text:lecture +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_8544 +openaire::meta_resource_types @=@ Research Literature @=@ text:musical_notation +openaire::meta_resource_types @=@ Research Literature @=@ http://purl.org/coar/resource_type/c_18cw +openaire::meta_resource_types @=@ Research Data @=@ dataset +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_ddb1 +openaire::meta_resource_types @=@ Research Data @=@ dataset:aggregated_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/ACF7-8YT9 +openaire::meta_resource_types @=@ Research Data @=@ cartographic_material +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_12cc +openaire::meta_resource_types @=@ Research Data @=@ dataset:clinical_trial_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_cb28 +openaire::meta_resource_types @=@ Research Data @=@ dataset:compiled_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/FXF3-D3G7 +openaire::meta_resource_types @=@ Research Data @=@ dataset:encoded_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/AM6W-6QAW +openaire::meta_resource_types @=@ Research Data @=@ dataset:experimental_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/63NG-B465 +openaire::meta_resource_types @=@ Research Data @=@ dataset:genomic_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/A8F1-NPV9 +openaire::meta_resource_types @=@ Research Data @=@ dataset:geospatial_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/2H0M-X761 +openaire::meta_resource_types @=@ Research Data @=@ dataset:laboratory_notebook +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/H41Y-FW7B +openaire::meta_resource_types @=@ Research Data @=@ cartographic_material:map +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_12cd +openaire::meta_resource_types @=@ Research Data @=@ dataset:measurement_and_test_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/DD58-GFSX +openaire::meta_resource_types @=@ Research Data @=@ dataset:observational_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/FF4C-28RK +openaire::meta_resource_types @=@ Research Data @=@ dataset:recorded_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/CQMR-7K63 +openaire::meta_resource_types @=@ Research Data @=@ dataset:simulation_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/W2XT-7017 +openaire::meta_resource_types @=@ Research Data @=@ sound +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_18cc +openaire::meta_resource_types @=@ Research Data @=@ dataset:survey_data +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/NHD0-W6SY +openaire::meta_resource_types @=@ Research Data @=@ image +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_c513 +openaire::meta_resource_types @=@ Research Data @=@ image:moving_image +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_8a7e +openaire::meta_resource_types @=@ Research Data @=@ image:still_image +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_ecc8 +openaire::meta_resource_types @=@ Research Data @=@ image:moving_image:video +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_12ce +openaire::meta_resource_types @=@ Research Data @=@ design +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/542X-3S04 +openaire::meta_resource_types @=@ Research Data @=@ design:industrial_design +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/JBNF-DYAD +openaire::meta_resource_types @=@ Research Data @=@ interactive_resource +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_e9a0 +openaire::meta_resource_types @=@ Research Data @=@ design:layout_design +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/BW7T-YM2G +openaire::meta_resource_types @=@ Research Data @=@ interactive_resource:website +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_7ad9 +openaire::meta_resource_types @=@ Research Data @=@ sound:musical_composition +openaire::meta_resource_types @=@ Research Data @=@ http://purl.org/coar/resource_type/c_18cd +openaire::meta_resource_types @=@ Research Software @=@ software:research_software +openaire::meta_resource_types @=@ Research Software @=@ http://purl.org/coar/resource_type/c_c950 +openaire::meta_resource_types @=@ Research Software @=@ software +openaire::meta_resource_types @=@ Research Software @=@ http://purl.org/coar/resource_type/c_5ce6 +openaire::meta_resource_types @=@ Research Software @=@ software:source_code +openaire::meta_resource_types @=@ Research Software @=@ http://purl.org/coar/resource_type/QH80-2R4E +openaire::meta_resource_types @=@ Other Research Products @=@ learning_object +openaire::meta_resource_types @=@ Other Research Products @=@ http://purl.org/coar/resource_type/c_e059 +openaire::meta_resource_types @=@ Other Research Products @=@ other +openaire::meta_resource_types @=@ Other Research Products @=@ http://purl.org/coar/resource_type/c_1843 +openaire::meta_resource_types @=@ Other Research Products @=@ trademark +openaire::meta_resource_types @=@ Other Research Products @=@ http://purl.org/coar/resource_type/H6QP-SC1X +openaire::meta_resource_types @=@ Other Research Products @=@ workflow +openaire::meta_resource_types @=@ Other Research Products @=@ http://purl.org/coar/resource_type/c_393c \ No newline at end of file From 159388f9c220e33810d50c7a4552ac640c78eb92 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 16 Oct 2023 11:26:07 +0200 Subject: [PATCH 06/60] testing and fix some issues --- .../main/java/eu/dnetlib/dhp/api/Utils.java | 7 + .../dnetlib/dhp/bulktag/SparkBulkTagJob.java | 11 +- .../community/QueryInformationSystem.java | 34 -- ...kResultToCommunityFromOrganizationJob.java | 50 +-- .../PrepareResultCommunitySet.java | 15 +- .../SparkResultToCommunityFromProject.java | 13 +- .../input_communitytoresult_parameters.json | 13 +- ...t_preparecommunitytoresult_parameters.json | 12 +- .../oozie_app/workflow.xml | 115 +------ .../input_communitytoresult_parameters.json | 28 ++ ...t_preparecommunitytoresult_parameters.json | 33 ++ .../QueryCommunityAPITest.java | 16 +- .../PrepareAssocTest.java | 95 ++++++ .../PrepareAssocTest.java | 88 +++++ .../ResultToCommunityJobTest.java | 323 ++++++++++++++++++ .../preparedInfo/resultCommunityList.json | 36 ++ .../preparedInfo/resultCommunityList.json.gz | Bin 939 -> 0 bytes .../relation/relation | 20 ++ .../preparedInfo/resultcommunitylist | 4 + .../relation/relation | 20 ++ .../sample/dataset | 10 + .../sample/otherresearchproduct | 0 .../sample/publication | 0 .../sample/software | 0 24 files changed, 736 insertions(+), 207 deletions(-) delete mode 100644 dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/QueryInformationSystem.java create mode 100644 dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/input_communitytoresult_parameters.json create mode 100644 dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/input_preparecommunitytoresult_parameters.json rename dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/{bulktag => api}/QueryCommunityAPITest.java (90%) create mode 100644 dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareAssocTest.java create mode 100644 dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareAssocTest.java create mode 100644 dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/ResultToCommunityJobTest.java create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo/resultCommunityList.json delete mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo/resultCommunityList.json.gz create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/relation/relation create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromproject/preparedInfo/resultcommunitylist create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromproject/relation/relation create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromproject/sample/dataset create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromproject/sample/otherresearchproduct create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromproject/sample/publication create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromproject/sample/software diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java index a0eacb774..43d5e7e98 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java @@ -11,6 +11,9 @@ import java.util.stream.Collectors; import javax.management.Query; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import com.amazonaws.util.StringUtils; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Maps; @@ -21,6 +24,7 @@ import eu.dnetlib.dhp.bulktag.community.CommunityConfiguration; import eu.dnetlib.dhp.bulktag.community.Provider; import eu.dnetlib.dhp.bulktag.criteria.VerbResolver; import eu.dnetlib.dhp.bulktag.criteria.VerbResolverFactory; +import eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob; /** * @author miriam.baglioni @@ -30,6 +34,8 @@ public class Utils implements Serializable { private static final ObjectMapper MAPPER = new ObjectMapper(); private static final VerbResolver resolver = VerbResolverFactory.newInstance(); + private static final Logger log = LoggerFactory.getLogger(Utils.class); + public static CommunityConfiguration getCommunityConfiguration(boolean production) throws IOException { final Map communities = Maps.newHashMap(); List validCommunities = new ArrayList<>(); @@ -126,6 +132,7 @@ public class Utils implements Serializable { throw new RuntimeException(e); } }); + return organizationMap; } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java index b24ee129a..68c740dd5 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java @@ -95,10 +95,7 @@ public class SparkBulkTagJob { Dataset datasources = readPath( spark, inputPath - .substring( - 0, - inputPath.lastIndexOf("/")) - + "/datasource", + + "datasource", Datasource.class) .filter((FilterFunction) ds -> isOKDatasource(ds)) .map((MapFunction) ds -> ds.getId(), Encoders.STRING()); @@ -106,10 +103,10 @@ public class SparkBulkTagJob { Map>> dsm = cc.getEoscDatasourceMap(); for (String ds : datasources.collectAsList()) { - final String dsId = ds.substring(3); - if (!dsm.containsKey(dsId)) { + // final String dsId = ds.substring(3); + if (!dsm.containsKey(ds)) { ArrayList> eoscList = new ArrayList<>(); - dsm.put(dsId, eoscList); + dsm.put(ds, eoscList); } } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/QueryInformationSystem.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/QueryInformationSystem.java deleted file mode 100644 index 5fe3cf81f..000000000 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/community/QueryInformationSystem.java +++ /dev/null @@ -1,34 +0,0 @@ - -package eu.dnetlib.dhp.bulktag.community; - -import java.io.IOException; -import java.util.List; - -import org.apache.commons.io.IOUtils; -import org.dom4j.DocumentException; -import org.xml.sax.SAXException; - -import com.google.common.base.Joiner; - -import eu.dnetlib.dhp.utils.ISLookupClientFactory; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; - -public class QueryInformationSystem { - - public static CommunityConfiguration getCommunityConfiguration(final String isLookupUrl) - throws ISLookUpException, DocumentException, SAXException, IOException { - ISLookUpService isLookUp = ISLookupClientFactory.getLookUpService(isLookupUrl); - final List res = isLookUp - .quickSearchProfile( - IOUtils - .toString( - QueryInformationSystem.class - .getResourceAsStream( - "/eu/dnetlib/dhp/bulktag/query.xq"))); - - final String xmlConf = "" + Joiner.on(" ").join(res) + ""; - - return CommunityConfigurationFactory.newInstance(xmlConf); - } -} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java index 43f425b68..fe79f1be1 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java @@ -2,7 +2,7 @@ package eu.dnetlib.dhp.resulttocommunityfromorganization; 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; @@ -22,6 +22,7 @@ import org.slf4j.LoggerFactory; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.Context; import eu.dnetlib.dhp.schema.oaf.Result; import scala.Tuple2; @@ -53,22 +54,15 @@ public class SparkResultToCommunityFromOrganizationJob { final String possibleupdatespath = parser.get("preparedInfoPath"); log.info("preparedInfoPath: {}", possibleupdatespath); - final String resultClassName = parser.get("resultTableName"); - log.info("resultTableName: {}", resultClassName); - - @SuppressWarnings("unchecked") - Class resultClazz = (Class) Class.forName(resultClassName); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession( + runWithSparkSession( conf, isSparkSessionManaged, spark -> { - removeOutputDir(spark, outputPath); + // removeOutputDir(spark, outputPath); - execPropagation(spark, inputPath, outputPath, resultClazz, possibleupdatespath); + execPropagation(spark, inputPath, outputPath, possibleupdatespath); }); } @@ -77,22 +71,32 @@ public class SparkResultToCommunityFromOrganizationJob { SparkSession spark, String inputPath, String outputPath, - Class resultClazz, String possibleUpdatesPath) { Dataset possibleUpdates = readPath(spark, possibleUpdatesPath, ResultCommunityList.class); - Dataset result = readPath(spark, inputPath, resultClazz); - result - .joinWith( - possibleUpdates, - result.col("id").equalTo(possibleUpdates.col("resultId")), - "left_outer") - .map(resultCommunityFn(), Encoders.bean(resultClazz)) - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .json(outputPath); + ModelSupport.entityTypes + .keySet() + .parallelStream() + .forEach(e -> { + if (ModelSupport.isResult(e)) { + Class resultClazz = ModelSupport.entityTypes.get(e); + removeOutputDir(spark, outputPath + e.name()); + Dataset result = readPath(spark, inputPath + e.name(), resultClazz); + + result + .joinWith( + possibleUpdates, + result.col("id").equalTo(possibleUpdates.col("resultId")), + "left_outer") + .map(resultCommunityFn(), Encoders.bean(resultClazz)) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath + e.name()); + } + }); + } private static MapFunction, R> resultCommunityFn() { diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java index b0fbf8056..883f5ca86 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java @@ -3,6 +3,7 @@ package eu.dnetlib.dhp.resulttocommunityfromproject; 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.*; @@ -51,16 +52,15 @@ public class PrepareResultCommunitySet { final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); - final boolean production = Boolean.valueOf(parser.get("outputPath")); + final boolean production = Boolean.valueOf(parser.get("production")); log.info("production: {}", production); final CommunityEntityMap projectsMap = Utils.getCommunityProjects(production); log.info("projectsMap: {}", new Gson().toJson(projectsMap)); SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession( + runWithSparkSession( conf, isSparkSessionManaged, spark -> { @@ -94,24 +94,27 @@ public class PrepareResultCommunitySet { .select( new Column("source").as("resultId"), new Column("target").as("projectId")) - .groupByKey((MapFunction) r -> (String) r.getAs("source"), Encoders.STRING()) + .groupByKey((MapFunction) r -> (String) r.getAs("resultId"), Encoders.STRING()) .mapGroups((MapGroupsFunction) (k, v) -> { ResultProjectList rpl = new ResultProjectList(); rpl.setResultId(k); ArrayList cl = new ArrayList<>(); - cl.addAll(projectMap.get(v.next().getAs("target"))); + cl.addAll(projectMap.get(v.next().getAs("projectId"))); v.forEachRemaining(r -> { projectMap - .get(r.getAs("target")) + .get(r.getAs("projectId")) .forEach(c -> { if (!cl.contains(c)) cl.add(c); }); }); + if(cl.size() == 0) + return null; rpl.setCommunityList(cl); return rpl; }, Encoders.bean(ResultProjectList.class)) + .filter(Objects::nonNull) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/SparkResultToCommunityFromProject.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/SparkResultToCommunityFromProject.java index 6d4779ea1..daef6a317 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/SparkResultToCommunityFromProject.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/SparkResultToCommunityFromProject.java @@ -4,6 +4,7 @@ package eu.dnetlib.dhp.resulttocommunityfromproject; import static eu.dnetlib.dhp.PropagationConstant.*; import static eu.dnetlib.dhp.PropagationConstant.PROPAGATION_RESULT_COMMUNITY_ORGANIZATION_CLASS_NAME; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; import java.io.Serializable; import java.util.ArrayList; @@ -61,20 +62,14 @@ public class SparkResultToCommunityFromProject implements Serializable { final String possibleupdatespath = parser.get("preparedInfoPath"); log.info("preparedInfoPath: {}", possibleupdatespath); - final String resultClassName = parser.get("resultTableName"); - log.info("resultTableName: {}", resultClassName); - - @SuppressWarnings("unchecked") - Class resultClazz = (Class) Class.forName(resultClassName); SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - runWithSparkHiveSession( + + runWithSparkSession( conf, isSparkSessionManaged, spark -> { -// removeOutputDir(spark, outputPath); execPropagation(spark, inputPath, outputPath, possibleupdatespath); @@ -108,7 +103,7 @@ public class SparkResultToCommunityFromProject implements Serializable { .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") - .json(outputPath); + .json(outputPath + e.name()); } }); diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json index b6eb309a5..0db8085d1 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_communitytoresult_parameters.json @@ -5,12 +5,7 @@ "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", @@ -23,12 +18,6 @@ "paramDescription": "true if the spark session is managed, false otherwise", "paramRequired": false }, - { - "paramName":"tn", - "paramLongName":"resultTableName", - "paramDescription": "the name of the result table we are currently working on", - "paramRequired": true - }, { "paramName": "p", "paramLongName": "preparedInfoPath", diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json index 8df509abf..8b6291e5d 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json @@ -5,12 +5,6 @@ "paramDescription": "the path of the sequencial file to read", "paramRequired": true }, - { - "paramName":"ocm", - "paramLongName":"organizationtoresultcommunitymap", - "paramDescription": "the map for the association organization communities", - "paramRequired": true - }, { "paramName":"h", "paramLongName":"hive_metastore_uris", @@ -28,6 +22,12 @@ "paramLongName": "outputPath", "paramDescription": "the path used to store temporary output files", "paramRequired": true + }, + { + "paramName": "p", + "paramLongName": "production", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml index b25822ad0..55490e25c 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml @@ -22,7 +22,7 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] @@ -90,8 +90,8 @@ eu.dnetlib.dhp.resulttocommunityfromorganization.PrepareResultCommunitySet dhp-enrichment-${projectVersion}.jar - --executor-cores=${sparkExecutorCores} - --executor-memory=${sparkExecutorMemory} + --executor-cores=4 + --executor-memory=10G --driver-memory=${sparkDriverMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} @@ -103,20 +103,13 @@ --sourcePath${sourcePath}/relation --outputPath${workingDir}/preparedInfo/resultCommunityList --hive_metastore_uris${hive_metastore_uris} - + --production${production} - + - - - - - - - - + yarn cluster @@ -135,104 +128,14 @@ --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} --preparedInfoPath${workingDir}/preparedInfo/resultCommunityList - --sourcePath${sourcePath}/publication - --outputPath${outputPath}/publication - --hive_metastore_uris${hive_metastore_uris} - --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication - --saveGraph${saveGraph} + --sourcePath${sourcePath}/ + --outputPath${outputPath}/ - + - - - yarn - cluster - community2resultfromorganization-Dataset - eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob - dhp-enrichment-${projectVersion}.jar - - --executor-cores=${sparkExecutorCores} - --executor-memory=${sparkExecutorMemory} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - - --preparedInfoPath${workingDir}/preparedInfo/resultCommunityList - --sourcePath${sourcePath}/dataset - --outputPath${outputPath}/dataset - --hive_metastore_uris${hive_metastore_uris} - --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset - --saveGraph${saveGraph} - - - - - - - yarn - cluster - community2resultfromorganization-ORP - eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob - dhp-enrichment-${projectVersion}.jar - - --executor-cores=${sparkExecutorCores} - --executor-memory=${sparkExecutorMemory} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - - --preparedInfoPath${workingDir}/preparedInfo/resultCommunityList - --sourcePath${sourcePath}/otherresearchproduct - --outputPath${outputPath}/otherresearchproduct - --hive_metastore_uris${hive_metastore_uris} - --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct - --saveGraph${saveGraph} - - - - - - - - yarn - cluster - community2resultfromorganization-Software - eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob - dhp-enrichment-${projectVersion}.jar - - --executor-cores=${sparkExecutorCores} - --executor-memory=${sparkExecutorMemory} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} - - --preparedInfoPath${workingDir}/preparedInfo/resultCommunityList - --sourcePath${sourcePath}/software - --outputPath${outputPath}/software - --hive_metastore_uris${hive_metastore_uris} - --resultTableNameeu.dnetlib.dhp.schema.oaf.Software - --saveGraph${saveGraph} - - - - - - diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/input_communitytoresult_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/input_communitytoresult_parameters.json new file mode 100644 index 000000000..0db8085d1 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/input_communitytoresult_parameters.json @@ -0,0 +1,28 @@ +[ + { + "paramName":"s", + "paramLongName":"sourcePath", + "paramDescription": "the path of the sequencial file to read", + "paramRequired": true + }, + + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true + }, + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "true if the spark session is managed, false otherwise", + "paramRequired": false + }, + { + "paramName": "p", + "paramLongName": "preparedInfoPath", + "paramDescription": "the path where prepared info have been stored", + "paramRequired": true + } + +] \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/input_preparecommunitytoresult_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/input_preparecommunitytoresult_parameters.json new file mode 100644 index 000000000..8b6291e5d --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/input_preparecommunitytoresult_parameters.json @@ -0,0 +1,33 @@ +[ + { + "paramName":"s", + "paramLongName":"sourcePath", + "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": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "true if the spark session is managed, false otherwise", + "paramRequired": false + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true + }, + { + "paramName": "p", + "paramLongName": "production", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true + } + +] \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/QueryCommunityAPITest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/api/QueryCommunityAPITest.java similarity index 90% rename from dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/QueryCommunityAPITest.java rename to dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/api/QueryCommunityAPITest.java index a0083dab8..6ee01a6f0 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/QueryCommunityAPITest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/api/QueryCommunityAPITest.java @@ -1,5 +1,5 @@ -package eu.dnetlib.dhp.bulktag; +package eu.dnetlib.dhp.api; import java.util.List; @@ -98,14 +98,22 @@ public class QueryCommunityAPITest { @Test void getCommunityProjects() throws Exception { CommunityEntityMap projectMap = Utils.getCommunityProjects(true); - Assertions.assertFalse(projectMap.containsKey("mes")); - Assertions.assertEquals(33, projectMap.size()); + Assertions .assertTrue( projectMap .keySet() .stream() - .allMatch(k -> projectMap.get(k).stream().allMatch(p -> p.startsWith("40|")))); + .allMatch(k -> k.startsWith("40|"))); + + System.out.println(projectMap); + } + + @Test + void getCommunityOrganizations() throws Exception { + CommunityEntityMap organizationMap = Utils.getCommunityOrganization(true); + Assertions.assertTrue(organizationMap.keySet().stream().allMatch(k -> k.startsWith("20|"))); + } } diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareAssocTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareAssocTest.java new file mode 100644 index 000000000..6536ecc85 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareAssocTest.java @@ -0,0 +1,95 @@ + +package eu.dnetlib.dhp.resulttocommunityfromorganization; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Encoders; +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 org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.gson.Gson; + +import eu.dnetlib.dhp.api.Utils; +import eu.dnetlib.dhp.api.model.CommunityEntityMap; +import eu.dnetlib.dhp.bulktag.BulkTagJobTest; +import eu.dnetlib.dhp.bulktag.SparkBulkTagJob; +import eu.dnetlib.dhp.schema.oaf.Dataset; + +/** + * @author miriam.baglioni + * @Date 13/10/23 + */ +public class PrepareAssocTest { + + private static SparkSession spark; + + private static Path workingDir; + + private static final Logger log = LoggerFactory.getLogger(PrepareAssocTest.class); + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(BulkTagJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(BulkTagJobTest.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(PrepareAssocTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + void test1() throws Exception { + + PrepareResultCommunitySet + .main( + new String[] { + + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", + getClass().getResource("/eu/dnetlib/dhp/resulttocommunityfromorganization/relation/").getPath(), + "-outputPath", workingDir.toString() + "/prepared", + "-production", Boolean.TRUE.toString(), + "-hive_metastore_uris", "" + }); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/prepared") + .map(item -> new ObjectMapper().readValue(item, ResultCommunityList.class)); + + tmp.foreach(r -> System.out.println(new ObjectMapper().writeValueAsString(r))); + } + +} diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareAssocTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareAssocTest.java new file mode 100644 index 000000000..0e10b3edf --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareAssocTest.java @@ -0,0 +1,88 @@ + +package eu.dnetlib.dhp.resulttocommunityfromproject; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + +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.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.bulktag.BulkTagJobTest; +import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; + +/** + * @author miriam.baglioni + * @Date 13/10/23 + */ +public class PrepareAssocTest { + + private static SparkSession spark; + + private static Path workingDir; + + private static final Logger log = LoggerFactory.getLogger(PrepareAssocTest.class); + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(BulkTagJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(BulkTagJobTest.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(PrepareAssocTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + void test1() throws Exception { + + PrepareResultCommunitySet + .main( + new String[] { + + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", + getClass().getResource("/eu/dnetlib/dhp/resulttocommunityfromproject/relation/").getPath(), + "-outputPath", workingDir.toString() + "/prepared", + "-production", Boolean.TRUE.toString(), + "-hive_metastore_uris", "" + }); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/prepared") + .map(item -> new ObjectMapper().readValue(item, ResultProjectList.class)); + + tmp.foreach(r -> System.out.println(new ObjectMapper().writeValueAsString(r))); + } + +} diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/ResultToCommunityJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/ResultToCommunityJobTest.java new file mode 100644 index 000000000..6a5726cbe --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/ResultToCommunityJobTest.java @@ -0,0 +1,323 @@ + +package eu.dnetlib.dhp.resulttocommunityfromproject; + +import static org.apache.spark.sql.functions.desc; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + +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.Encoders; +import org.apache.spark.sql.Row; +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 org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.orcidtoresultfromsemrel.OrcidPropagationJobTest; +import eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob; +import eu.dnetlib.dhp.schema.oaf.Dataset; + +public class ResultToCommunityJobTest { + + private static final Logger log = LoggerFactory.getLogger(ResultToCommunityJobTest.class); + + 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(ResultToCommunityJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(ResultToCommunityJobTest.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(OrcidPropagationJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + void testSparkResultToCommunityFromProjectJob() throws Exception { + final String preparedInfoPath = getClass() + .getResource("/eu/dnetlib/dhp/resulttocommunityfromproject/preparedInfo") + .getPath(); + SparkResultToCommunityFromProject + .main( + new String[] { + + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", getClass() + .getResource("/eu/dnetlib/dhp/resulttocommunityfromproject/sample/") + .getPath(), + + "-outputPath", workingDir.toString() + "/", + "-preparedInfoPath", preparedInfoPath + }); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/dataset") + .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); + + tmp.foreach(d -> System.out.println(new ObjectMapper().writeValueAsString(d))); +// Assertions.assertEquals(10, tmp.count()); +// org.apache.spark.sql.Dataset verificationDataset = spark +// .createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); +// +// verificationDataset.createOrReplaceTempView("dataset"); +// +// String query = "select id, MyT.id community " +// + "from dataset " +// + "lateral view explode(context) c as MyT " +// + "lateral view explode(MyT.datainfo) d as MyD " +// + "where MyD.inferenceprovenance = 'propagation'"; +// +// org.apache.spark.sql.Dataset resultExplodedProvenance = spark.sql(query); +// Assertions.assertEquals(5, resultExplodedProvenance.count()); +// Assertions +// .assertEquals( +// 0, +// resultExplodedProvenance +// .filter("id = '50|dedup_wf_001::afaf128022d29872c4dad402b2db04fe'") +// .count()); +// Assertions +// .assertEquals( +// 1, +// resultExplodedProvenance +// .filter("id = '50|dedup_wf_001::3f62cfc27024d564ea86760c494ba93b'") +// .count()); +// Assertions +// .assertEquals( +// "beopen", +// resultExplodedProvenance +// .select("community") +// .where( +// resultExplodedProvenance +// .col("id") +// .equalTo( +// "50|dedup_wf_001::3f62cfc27024d564ea86760c494ba93b")) +// .collectAsList() +// .get(0) +// .getString(0)); +// +// Assertions +// .assertEquals( +// 2, +// resultExplodedProvenance +// .filter("id = '50|od________18::8887b1df8b563c4ea851eb9c882c9d7b'") +// .count()); +// Assertions +// .assertEquals( +// "mes", +// resultExplodedProvenance +// .select("community") +// .where( +// resultExplodedProvenance +// .col("id") +// .equalTo( +// "50|od________18::8887b1df8b563c4ea851eb9c882c9d7b")) +// .sort(desc("community")) +// .collectAsList() +// .get(0) +// .getString(0)); +// Assertions +// .assertEquals( +// "euromarine", +// resultExplodedProvenance +// .select("community") +// .where( +// resultExplodedProvenance +// .col("id") +// .equalTo( +// "50|od________18::8887b1df8b563c4ea851eb9c882c9d7b")) +// .sort(desc("community")) +// .collectAsList() +// .get(1) +// .getString(0)); +// +// Assertions +// .assertEquals( +// 1, +// resultExplodedProvenance +// .filter("id = '50|doajarticles::8d817039a63710fcf97e30f14662c6c8'") +// .count()); +// Assertions +// .assertEquals( +// "mes", +// resultExplodedProvenance +// .select("community") +// .where( +// resultExplodedProvenance +// .col("id") +// .equalTo( +// "50|doajarticles::8d817039a63710fcf97e30f14662c6c8")) +// .sort(desc("community")) +// .collectAsList() +// .get(0) +// .getString(0)); +// +// Assertions +// .assertEquals( +// 1, +// resultExplodedProvenance +// .filter("id = '50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6'") +// .count()); +// Assertions +// .assertEquals( +// "mes", +// resultExplodedProvenance +// .select("community") +// .where( +// resultExplodedProvenance +// .col("id") +// .equalTo( +// "50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6")) +// .sort(desc("community")) +// .collectAsList() +// .get(0) +// .getString(0)); +// +// query = "select id, MyT.id community " +// + "from dataset " +// + "lateral view explode(context) c as MyT " +// + "lateral view explode(MyT.datainfo) d as MyD "; +// +// org.apache.spark.sql.Dataset resultCommunityId = spark.sql(query); +// +// Assertions.assertEquals(10, resultCommunityId.count()); +// +// Assertions +// .assertEquals( +// 1, +// resultCommunityId +// .filter("id = '50|dedup_wf_001::afaf128022d29872c4dad402b2db04fe'") +// .count()); +// Assertions +// .assertEquals( +// "beopen", +// resultCommunityId +// .select("community") +// .where( +// resultCommunityId +// .col("id") +// .equalTo( +// "50|dedup_wf_001::afaf128022d29872c4dad402b2db04fe")) +// .collectAsList() +// .get(0) +// .getString(0)); +// +// Assertions +// .assertEquals( +// 1, +// resultCommunityId +// .filter("id = '50|dedup_wf_001::3f62cfc27024d564ea86760c494ba93b'") +// .count()); +// +// Assertions +// .assertEquals( +// 3, +// resultCommunityId +// .filter("id = '50|od________18::8887b1df8b563c4ea851eb9c882c9d7b'") +// .count()); +// Assertions +// .assertEquals( +// "beopen", +// resultCommunityId +// .select("community") +// .where( +// resultCommunityId +// .col("id") +// .equalTo( +// "50|od________18::8887b1df8b563c4ea851eb9c882c9d7b")) +// .sort(desc("community")) +// .collectAsList() +// .get(2) +// .getString(0)); +// +// Assertions +// .assertEquals( +// 2, +// resultCommunityId +// .filter("id = '50|doajarticles::8d817039a63710fcf97e30f14662c6c8'") +// .count()); +// Assertions +// .assertEquals( +// "euromarine", +// resultCommunityId +// .select("community") +// .where( +// resultCommunityId +// .col("id") +// .equalTo( +// "50|doajarticles::8d817039a63710fcf97e30f14662c6c8")) +// .sort(desc("community")) +// .collectAsList() +// .get(1) +// .getString(0)); +// +// Assertions +// .assertEquals( +// 3, +// resultCommunityId +// .filter("id = '50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6'") +// .count()); +// Assertions +// .assertEquals( +// "euromarine", +// resultCommunityId +// .select("community") +// .where( +// resultCommunityId +// .col("id") +// .equalTo( +// "50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6")) +// .sort(desc("community")) +// .collectAsList() +// .get(2) +// .getString(0)); +// Assertions +// .assertEquals( +// "ni", +// resultCommunityId +// .select("community") +// .where( +// resultCommunityId +// .col("id") +// .equalTo( +// "50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6")) +// .sort(desc("community")) +// .collectAsList() +// .get(0) +// .getString(0)); + } +} diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo/resultCommunityList.json b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo/resultCommunityList.json new file mode 100644 index 000000000..bd4674c4f --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo/resultCommunityList.json @@ -0,0 +1,36 @@ +{"communityList":["beopen"],"resultId":"50|dedup_wf_001::afaf128022d29872c4dad402b2db04fe"} +{"communityList":["beopen"],"resultId":"50|dedup_wf_001::3f62cfc27024d564ea86760c494ba93b"} +{"communityList":["euromarine","mes"],"resultId":"50|od________18::8887b1df8b563c4ea851eb9c882c9d7b"} +{"communityList":["euromarine","mes"],"resultId":"50|doajarticles::8d817039a63710fcf97e30f14662c6c8"} +{"communityList":["euromarine","mes"],"resultId":"50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6"} +{"communityList":["euromarine","mes"],"resultId":"50|doajarticles::53b70ea6e0769d02ddf93307ec8e3e92"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::ef1ac6efc10f420fa9e190e49644f1f2"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::b738aa93950dddfb0294df2e8fdf0579"} +{"communityList":["euromarine","mes"],"resultId":"50|doajarticles::befccb1f9e6b833fd82e587737ae9e7d"} +{"communityList":["euromarine","mes"],"resultId":"50|doajarticles::bf1cba621615e27db1692865a5f35a0b"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::e105de571b336daae05f0e75cf740c5c"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::1fe4f347c9df657b7ba520987d79436e"} +{"communityList":["euromarine","mes"],"resultId":"50|doajarticles::6d7c00a8c8e59f0215459e2e4ee3fd6c"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::39ec88ef4127db0ea1b88938f1c52889"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::3496709db804d98f76c45d7ed023dd95"} +{"communityList":["euromarine","mes"],"resultId":"50|doajarticles::dc97fffbdb6d35f792fc0ab428ff065c"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::b61d082d96619d9b7a876e6dce44cf65"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::32a96881c3036cf2d2165bb2d276ea82"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::76e9e6a959ba588483c74ec580369864"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::1487a0a92572376d95d6cc3f066504b7"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::413a0a0656f888cce9c15f6be6df60e3"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::25c3e91960cbd7a8f95a2e511cbffddd"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::4d523b52094a689856e479bb99063c7a"} +{"communityList":["euromarine","mes"],"resultId":"50|od______2663::393c7262bb71642b7bb4c67cfeab02c5"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::0b3333d875b91ffa4db0735efec94e7a"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::0699c30043edfae40786d80acd20d300"} +{"communityList":["euromarine","mes"],"resultId":"50|doajarticles::7f29ade677e66ffbf1312fa837bc73ca"} +{"communityList":["euromarine","mes"],"resultId":"50|doajarticles::7ba6627ac7590d367cc01bbac4d518e8"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::8ad9bc047433401947dc0cdb4a989cee"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::cce70f27d85df658479d0ec0046a4eb3"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::74304be834b7013dbaeb73c3a19a654b"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::46fc13a87befb6a83ac9c63580528ab0"} +{"communityList":["euromarine","mes"],"resultId":"50|doajarticles::66f1867488b62d9c9fb734273775e203"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::79236995d5c30e5234a47cee4a728cae"} +{"communityList":["euromarine","mes"],"resultId":"50|od______2386::cb7f6cb01d1a835612731d645842f699"} +{"communityList":["euromarine","mes"],"resultId":"50|dedup_wf_001::d424daa43f97a434eb0a12289410cade"} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo/resultCommunityList.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/preparedInfo/resultCommunityList.json.gz deleted file mode 100644 index 8b452d0e17e06baf8f3ac6d62277c2a08d88a13c..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 939 zcmV;c162GUiwFqE4x3&819D|^b!>D)Z*6ULZfSIROlfm;E^2dcZUC)UNs44g4Bh(} zx$cD`LZP8@l!t}WKNm@oi~mx3dP9D_`Sa7$m-mmK zfBoU{)92g6kGGT0AM*X}r#H8c`RU8M&)>)G;kMzo$oTT%?JxBfU_3lnS&b<$XG|5I zj$wf?&x`|DdHeR=*L$D6il&!OhKaEW(}bA7QSq3r^Uu%DFCU+u?BnD6yuG~3&%s|TnnNK`39&|9gmQR^^)cy&=s6Bd#i*H5QLtIB+1TkZ{EX71BWqR2%q`4|Ej2kV zLaE`$Xbu`XC&wA2Q)t_+wGHmSqYzhtV#169)z=f?4T2SrFGW$MVAci z#cRb&=If|3Y@6Y=v(gpw{#pKlLaMD5z?D&!2rLu=<}o4q`)eGAYpO*Oa}u@F)L3Jd z$>QU@{Qg;9qjXTxWOLryKfW1p80T2`*F7Zf?_;kvCf4TVfQ)M)3`GtHkAtS7p7*EN zE0oqDLv^;uehEwIGY-fOHYCG57pbymXd_8k?Iti*9Lp|GzHD@<-FQ*EK Date: Mon, 16 Oct 2023 11:46:12 +0200 Subject: [PATCH 07/60] new spark parrameter updated --- .../oozie_app/workflow.xml | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml index 55490e25c..7a5fd7434 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml @@ -90,14 +90,16 @@ eu.dnetlib.dhp.resulttocommunityfromorganization.PrepareResultCommunitySet dhp-enrichment-${projectVersion}.jar - --executor-cores=4 - --executor-memory=10G + --executor-cores=6 + --executor-memory=5G + --spark.executor.memoryOverhead=3g + --conf spark.sql.shuffle.partitions=3284 --driver-memory=${sparkDriverMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} --sourcePath${sourcePath}/relation @@ -117,14 +119,15 @@ eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob dhp-enrichment-${projectVersion}.jar - --executor-cores=${sparkExecutorCores} - --executor-memory=${sparkExecutorMemory} + --executor-cores=6 + --executor-memory=5G + --spark.executor.memoryOverhead=3g + --conf spark.sql.shuffle.partitions=3284 --driver-memory=${sparkDriverMemory} --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.enabled=true --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} --preparedInfoPath${workingDir}/preparedInfo/resultCommunityList From 6dfcd0c9a2072c589709a97769185a966def58b4 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Mon, 16 Oct 2023 12:57:18 +0200 Subject: [PATCH 08/60] [raw graph] mapping original resource types --- .../raw/AbstractMdRecordToOafMapper.java | 63 ++++--- .../dhp/oa/graph/raw/OafToOafMapper.java | 41 +++-- .../dhp/oa/graph/raw/OdfToOafMapper.java | 22 ++- .../oa/graph/raw/OriginalTypeComparator.java | 43 ++--- .../dnetlib/dhp/oa/graph/raw/MappersTest.java | 168 ++++++++++++++++-- .../dnetlib/dhp/oa/graph/raw/oaf_crossref.xml | 68 +++++++ 6 files changed, 308 insertions(+), 97 deletions(-) create mode 100644 dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_crossref.xml diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java index 5f0acbab7..49133cedb 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java @@ -157,20 +157,20 @@ public abstract class AbstractMdRecordToOafMapper { if (vocs.vocabularyExists(OPENAIRE_META_RESOURCE_TYPE)) { Optional instanceTypeMapping = instances - .stream() - .flatMap(i -> i.getInstanceTypeMapping().stream()) - .filter(t -> OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(t.getVocabularyName())) - .findFirst(); + .stream() + .flatMap(i -> i.getInstanceTypeMapping().stream()) + .filter(t -> OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(t.getVocabularyName())) + .findFirst(); if (!instanceTypeMapping.isPresent()) { - throw new IllegalStateException("unable to find an instance from " + OPENAIRE_COAR_RESOURCE_TYPES_3_1); + return null; } else { final String typeCode = instanceTypeMapping.get().getTypeCode(); return Optional - .ofNullable(vocs.lookupTermBySynonym(OPENAIRE_META_RESOURCE_TYPE, typeCode)) - .orElseThrow(() -> - new IllegalStateException("unable to find a synonym for '" + typeCode + "' in " + - OPENAIRE_META_RESOURCE_TYPE)); + .ofNullable(vocs.lookupTermBySynonym(OPENAIRE_META_RESOURCE_TYPE, typeCode)) + .orElseThrow( + () -> new IllegalStateException("unable to find a synonym for '" + typeCode + "' in " + + OPENAIRE_META_RESOURCE_TYPE)); } } else { throw new IllegalStateException("vocabulary '" + OPENAIRE_META_RESOURCE_TYPE + "' not available"); @@ -197,7 +197,8 @@ public abstract class AbstractMdRecordToOafMapper { final DataInfo info, final long lastUpdateTimestamp) { - final OafEntity entity = createEntity(doc, type, metaResourceType, instances, collectedFrom, info, lastUpdateTimestamp); + final OafEntity entity = createEntity( + doc, type, metaResourceType, instances, collectedFrom, info, lastUpdateTimestamp); final Set originalId = Sets.newHashSet(entity.getOriginalId()); originalId.add(entity.getId()); @@ -550,29 +551,33 @@ public abstract class AbstractMdRecordToOafMapper { protected abstract String findOriginalType(Document doc); protected List prepareInstanceTypeMapping(Document doc) { - return Optional.ofNullable(findOriginalType(doc)) - .map(originalType -> { - final List mappings = Lists.newArrayList(); + return Optional + .ofNullable(findOriginalType(doc)) + .map(originalType -> { + final List mappings = Lists.newArrayList(); - if (vocs.vocabularyExists(OPENAIRE_COAR_RESOURCE_TYPES_3_1)) { + if (vocs.vocabularyExists(OPENAIRE_COAR_RESOURCE_TYPES_3_1)) { - // TODO verify what the vocabs return when a synonym is not defined - Optional.ofNullable(vocs.lookupTermBySynonym(OPENAIRE_COAR_RESOURCE_TYPES_3_1, originalType)) - .ifPresent(coarTerm -> { - mappings.add(OafMapperUtils.instanceTypeMapping(originalType, coarTerm)); - if (vocs.vocabularyExists(OPENAIRE_USER_RESOURCE_TYPES)) { + // TODO verify what the vocabs return when a synonym is not defined + Optional + .ofNullable(vocs.lookupTermBySynonym(OPENAIRE_COAR_RESOURCE_TYPES_3_1, originalType)) + .ifPresent(coarTerm -> { + mappings.add(OafMapperUtils.instanceTypeMapping(originalType, coarTerm)); + if (vocs.vocabularyExists(OPENAIRE_USER_RESOURCE_TYPES)) { - // TODO verify what the vocabs return when a synonym is not defined - Optional - .ofNullable(vocs.lookupTermBySynonym(OPENAIRE_USER_RESOURCE_TYPES, coarTerm.getClassid())) - .ifPresent(type -> mappings.add(OafMapperUtils.instanceTypeMapping(originalType, type))); - } - }); - } + // TODO verify what the vocabs return when a synonym is not defined + Optional + .ofNullable( + vocs.lookupTermBySynonym(OPENAIRE_USER_RESOURCE_TYPES, coarTerm.getClassid())) + .ifPresent( + type -> mappings.add(OafMapperUtils.instanceTypeMapping(originalType, type))); + } + }); + } - return mappings; - }) - .orElse(new ArrayList<>()); + return mappings; + }) + .orElse(new ArrayList<>()); } private Journal prepareJournal(final Document doc, final DataInfo info) { diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java index 6bcc8ec44..a63296d18 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OafToOafMapper.java @@ -1,17 +1,8 @@ package eu.dnetlib.dhp.oa.graph.raw; -import com.google.common.collect.Lists; -import eu.dnetlib.dhp.common.PacePerson; -import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; -import eu.dnetlib.dhp.schema.oaf.*; -import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions; -import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; -import eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits; -import org.apache.commons.lang3.StringUtils; -import org.dom4j.Document; -import org.dom4j.Element; -import org.dom4j.Node; +import static eu.dnetlib.dhp.schema.common.ModelConstants.*; +import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.*; import java.net.URLDecoder; import java.util.ArrayList; @@ -20,8 +11,19 @@ import java.util.List; import java.util.Set; import java.util.stream.Collectors; -import static eu.dnetlib.dhp.schema.common.ModelConstants.*; -import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.*; +import org.apache.commons.lang3.StringUtils; +import org.dom4j.Document; +import org.dom4j.Element; +import org.dom4j.Node; + +import com.google.common.collect.Lists; + +import eu.dnetlib.dhp.common.PacePerson; +import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions; +import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; +import eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits; public class OafToOafMapper extends AbstractMdRecordToOafMapper { @@ -201,12 +203,13 @@ public class OafToOafMapper extends AbstractMdRecordToOafMapper { */ @Override protected String findOriginalType(Document doc) { - return (String) doc.selectNodes("//dc:type") - .stream() - .map(o -> "" + ((Node) o).getText().trim()) - .sorted(new OriginalTypeComparator()) - .findFirst() - .orElse(null); + return (String) doc + .selectNodes("//dc:type") + .stream() + .map(o -> "" + ((Node) o).getText().trim()) + .sorted(new OriginalTypeComparator()) + .findFirst() + .orElse(null); } @Override diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java index c383e9300..e63b01a00 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OdfToOafMapper.java @@ -230,20 +230,18 @@ public class OdfToOafMapper extends AbstractMdRecordToOafMapper { */ @Override protected String findOriginalType(Document doc) { - String resourceType = Optional.ofNullable((Element) doc.selectSingleNode( + return Optional + .ofNullable( + (Element) doc + .selectSingleNode( "//*[local-name()='metadata']/*[local-name() = 'resource']/*[local-name() = 'resourceType']")) - .map(element -> { - final String resourceTypeURI = element.attributeValue("anyURI"); - final String resourceTypeTxt = element.getText(); - - return ObjectUtils.firstNonNull(resourceTypeURI, resourceTypeTxt); - }) - .orElse(doc.valueOf( - "//*[local-name()='metadata']/*[local-name() = 'resource']/*[local-name() = 'CobjCategory']/text()") - ); - - return resourceType; + .map(element -> { + final String resourceTypeURI = element.attributeValue("anyURI"); + final String resourceTypeTxt = element.getText(); + return ObjectUtils.firstNonNull(resourceTypeURI, resourceTypeTxt); + }) + .orElse(null); } @Override diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OriginalTypeComparator.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OriginalTypeComparator.java index 2eeead32e..c3d8b4789 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OriginalTypeComparator.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/OriginalTypeComparator.java @@ -1,32 +1,33 @@ -package eu.dnetlib.dhp.oa.graph.raw; -import java.util.Comparator; +package eu.dnetlib.dhp.oa.graph.raw; import static org.apache.commons.lang3.StringUtils.contains; import static org.apache.commons.lang3.StringUtils.startsWith; +import java.util.Comparator; + public class OriginalTypeComparator implements Comparator { - @Override - public int compare(String t1, String t2) { + @Override + public int compare(String t1, String t2) { - if (t1.equals(t2)) { - return 0; - } - if (startsWith(t1, "http") && contains(t1, "coar") && contains(t1, "resource_type")) { - return -1; - } - if (startsWith(t2, "http") && contains(t2, "coar") && contains(t2, "resource_type")) { - return 1; - } - if (startsWith(t1, "info:eu-repo/semantics")) { - return -1; - } - if (startsWith(t2, "info:eu-repo/semantics")) { - return 1; - } + if (t1.equals(t2)) { + return 0; + } + if (startsWith(t1, "http") && contains(t1, "coar") && contains(t1, "resource_type")) { + return -1; + } + if (startsWith(t2, "http") && contains(t2, "coar") && contains(t2, "resource_type")) { + return 1; + } + if (startsWith(t1, "info:eu-repo/semantics")) { + return -1; + } + if (startsWith(t2, "info:eu-repo/semantics")) { + return 1; + } - return t1.compareTo(t2); - } + return t1.compareTo(t2); + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java index 4d30a3fa8..e6997ac1d 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java @@ -14,6 +14,8 @@ import java.util.stream.Collectors; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Encoders; import org.dom4j.DocumentException; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -25,8 +27,11 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.common.Constants; import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; +import eu.dnetlib.dhp.oa.graph.clean.CleaningRuleMap; +import eu.dnetlib.dhp.oa.graph.clean.OafCleaner; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.utils.GraphCleaningFunctions; import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; import eu.dnetlib.dhp.schema.oaf.utils.PidType; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; @@ -119,19 +124,21 @@ class MappersTest { assertNotNull(instance.getInstanceTypeMapping()); assertEquals(2, instance.getInstanceTypeMapping().size()); - Optional coarType = instance.getInstanceTypeMapping() - .stream() - .filter(itm -> ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(itm.getVocabularyName())) - .findFirst(); + Optional coarType = instance + .getInstanceTypeMapping() + .stream() + .filter(itm -> ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(itm.getVocabularyName())) + .findFirst(); assertTrue(coarType.isPresent()); assertEquals("http://purl.org/coar/resource_type/c_5794", coarType.get().getTypeCode()); assertEquals("conference paper", coarType.get().getTypeLabel()); - Optional userType = instance.getInstanceTypeMapping() - .stream() - .filter(itm -> ModelConstants.OPENAIRE_USER_RESOURCE_TYPES.equals(itm.getVocabularyName())) - .findFirst(); + Optional userType = instance + .getInstanceTypeMapping() + .stream() + .filter(itm -> ModelConstants.OPENAIRE_USER_RESOURCE_TYPES.equals(itm.getVocabularyName())) + .findFirst(); assertTrue(userType.isPresent()); assertEquals("Article", userType.get().getTypeCode()); @@ -266,8 +273,8 @@ class MappersTest { }); Publication p_cleaned = cleanup(p, vocs); - assertEquals("0000", p_cleaned.getInstance().get(0).getRefereed().getClassid()); - assertEquals("Unknown", p_cleaned.getInstance().get(0).getRefereed().getClassname()); + assertEquals("0002", p_cleaned.getInstance().get(0).getRefereed().getClassid()); + assertEquals("nonPeerReviewed", p_cleaned.getInstance().get(0).getRefereed().getClassname()); assertNotNull(p.getInstance().get(0).getPid()); assertEquals(2, p.getInstance().get(0).getPid().size()); @@ -485,8 +492,8 @@ class MappersTest { }); Publication p_cleaned = cleanup(p, vocs); - assertEquals("0000", p_cleaned.getInstance().get(0).getRefereed().getClassid()); - assertEquals("Unknown", p_cleaned.getInstance().get(0).getRefereed().getClassname()); + assertEquals("0002", p_cleaned.getInstance().get(0).getRefereed().getClassid()); + assertEquals("nonPeerReviewed", p_cleaned.getInstance().get(0).getRefereed().getClassname()); } @Test @@ -604,8 +611,137 @@ class MappersTest { assertTrue(i.getUrl().contains("https://clinicaltrials.gov/ct2/show/NCT02321059")); Dataset d_cleaned = cleanup(d, vocs); - assertEquals("0000", d_cleaned.getInstance().get(0).getRefereed().getClassid()); - assertEquals("Unknown", d_cleaned.getInstance().get(0).getRefereed().getClassname()); + assertEquals("0002", d_cleaned.getInstance().get(0).getRefereed().getClassid()); + assertEquals("nonPeerReviewed", d_cleaned.getInstance().get(0).getRefereed().getClassname()); + } + + @Test + void test_record_from_Crossref() throws IOException { + + final CleaningRuleMap mapping = CleaningRuleMap.create(vocs); + + final String xml = IOUtils + .toString(Objects.requireNonNull(getClass().getResourceAsStream("oaf_crossref.xml"))); + final List list = new OafToOafMapper(vocs, false, true).processMdRecord(xml); + + assertEquals(1, list.size()); + assertTrue(list.get(0) instanceof Publication); + + final Publication p = OafCleaner.apply(fixVocabularyNames((Publication) list.get(0)), mapping); + + assertNotNull(p.getDateofcollection()); + assertEquals("2020-08-06T07:04:09.62Z", p.getDateofcollection()); + + assertNotNull(p.getDateoftransformation()); + assertEquals("2020-08-06T07:20:57.911Z", p.getDateoftransformation()); + + assertNotNull(p.getDataInfo()); + assertFalse(p.getDataInfo().getInvisible()); + assertFalse(p.getDataInfo().getDeletedbyinference()); + assertEquals("0.9", p.getDataInfo().getTrust()); + + assertValidId(p.getId()); + assertEquals(2, p.getOriginalId().size()); + + assertEquals("50|doi_________::7f0f7807f17db50e5c2b5c452ccaf06d", p.getOriginalId().get(0)); + assertValidId(p.getCollectedfrom().get(0).getKey()); + + assertNotNull(p.getTitle()); + assertEquals(1, p.getTitle().size()); + assertEquals( + "A case report of serious haemolysis in a glucose-6-phosphate dehydrogenase-deficient COVID-19 patient receiving hydroxychloroquine", + p + .getTitle() + .get(0) + .getValue()); + + assertNotNull(p.getDescription()); + assertEquals(0, p.getDescription().size()); + + assertEquals(8, p.getAuthor().size()); + + assertNotNull(p.getInstance()); + assertEquals(1, p.getInstance().size()); + + final Instance i = p.getInstance().get(0); + + assertNotNull(i.getAccessright()); + assertEquals(ModelConstants.DNET_ACCESS_MODES, i.getAccessright().getSchemeid()); + assertEquals(ModelConstants.DNET_ACCESS_MODES, i.getAccessright().getSchemename()); + assertEquals("OPEN", i.getAccessright().getClassid()); + assertEquals("Open Access", i.getAccessright().getClassname()); + + assertNotNull(i.getCollectedfrom()); + assertEquals("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2", i.getCollectedfrom().getKey()); + assertEquals("Crossref", i.getCollectedfrom().getValue()); + + assertNotNull(i.getHostedby()); + assertEquals("10|openaire____::55045bd2a65019fd8e6741a755395c8c", i.getHostedby().getKey()); + assertEquals("Unknown Repository", i.getHostedby().getValue()); + + assertNotNull(i.getInstancetype()); + assertEquals("0001", i.getInstancetype().getClassid()); + assertEquals("Article", i.getInstancetype().getClassname()); + assertEquals(ModelConstants.DNET_PUBLICATION_RESOURCE, i.getInstancetype().getSchemeid()); + assertEquals(ModelConstants.DNET_PUBLICATION_RESOURCE, i.getInstancetype().getSchemename()); + + assertNull(i.getLicense()); + assertNotNull(i.getDateofacceptance()); + assertEquals("2020-06-04", i.getDateofacceptance().getValue()); + + assertNull(i.getProcessingchargeamount()); + assertNull(i.getProcessingchargecurrency()); + + assertNotNull(i.getPid()); + assertEquals(1, i.getPid().size()); + + assertNotNull(i.getAlternateIdentifier()); + assertEquals(0, i.getAlternateIdentifier().size()); + + assertNotNull(i.getUrl()); + assertEquals(1, i.getUrl().size()); + assertTrue(i.getUrl().contains("http://dx.doi.org/10.1080/23744235.2020.1774644")); + + assertEquals("", p.getInstance().get(0).getRefereed().getClassid()); + assertEquals("", p.getInstance().get(0).getRefereed().getClassname()); + + Publication p_cleaned = cleanup(p, vocs); + + assertEquals("0001", p_cleaned.getInstance().get(0).getRefereed().getClassid()); + assertEquals("peerReviewed", p_cleaned.getInstance().get(0).getRefereed().getClassname()); + + assertNotNull(p_cleaned.getMetaResourceType()); + assertEquals("Research Literature", p_cleaned.getMetaResourceType().getClassid()); + assertEquals("Research Literature", p_cleaned.getMetaResourceType().getClassname()); + assertEquals(ModelConstants.OPENAIRE_META_RESOURCE_TYPE, p_cleaned.getMetaResourceType().getSchemeid()); + assertEquals(ModelConstants.OPENAIRE_META_RESOURCE_TYPE, p_cleaned.getMetaResourceType().getSchemename()); + + assertNotNull(p_cleaned.getInstance().get(0).getInstanceTypeMapping()); + assertEquals(2, p_cleaned.getInstance().get(0).getInstanceTypeMapping().size()); + + assertTrue( + p_cleaned + .getInstance() + .get(0) + .getInstanceTypeMapping() + .stream() + .anyMatch( + t -> "journal-article".equals(t.getOriginalType()) && + ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(t.getVocabularyName()) && + "http://purl.org/coar/resource_type/c_2df8fbb1".equals(t.getTypeCode()) && + "research article".equals(t.getTypeLabel()))); + + assertTrue( + p_cleaned + .getInstance() + .get(0) + .getInstanceTypeMapping() + .stream() + .anyMatch( + t -> "journal-article".equals(t.getOriginalType()) && + ModelConstants.OPENAIRE_USER_RESOURCE_TYPES.equals(t.getVocabularyName()) && + "Article".equals(t.getTypeCode()) && + "Article".equals(t.getTypeLabel()))); } @Test @@ -908,8 +1044,8 @@ class MappersTest { }); Dataset p_cleaned = cleanup(p, vocs); - assertEquals("0000", p_cleaned.getInstance().get(0).getRefereed().getClassid()); - assertEquals("Unknown", p_cleaned.getInstance().get(0).getRefereed().getClassname()); + assertEquals("0002", p_cleaned.getInstance().get(0).getRefereed().getClassid()); + assertEquals("nonPeerReviewed", p_cleaned.getInstance().get(0).getRefereed().getClassname()); } @Test diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_crossref.xml b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_crossref.xml new file mode 100644 index 000000000..182820a08 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/oaf_crossref.xml @@ -0,0 +1,68 @@ + + +
+ doi_________::7f0f7807f17db50e5c2b5c452ccaf06d + doi_________::7f0f7807f17db50e5c2b5c452ccaf06d + 2020-08-06T07:04:09.62Z + + + + + + 2020-08-06T07:20:57.911Z + openaire____ +
+ + A case report of serious haemolysis in a glucose-6-phosphate dehydrogenase-deficient COVID-19 patient receiving hydroxychloroquine + Maillart, E. + Leemans, S. + Van Noten, H. + Vandergraesen, T. + Mahadeb, B. + Salaouatchi, M. T. + De Bels, D. + Clevenbergh, P. + + http://dx.doi.org/10.1080/23744235.2020.1774644 + + Informa UK Limited + Crossref + Infectious Diseases + Microbiology (medical) + General Immunology and Microbiology + Infectious Diseases + General Medicine + journal-article + 0001 + 2020-06-04 + + OPEN + + + 10.1080/23744235.2020.1774644 + Infectious Diseases + + + + + file%3A%2F%2F%2Fsrv%2Fclaims%2Frecords%2Fpublication%2Fcrossref + + + + + + + false + false + 0.9 + + + + +
\ No newline at end of file From d28b7085f6f13a644926302e47da314dcb22de5d Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 17 Oct 2023 11:09:31 +0200 Subject: [PATCH 09/60] more NPE checks --- .../common/vocabulary/VocabularyGroup.java | 25 ++++++++----------- 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/common/vocabulary/VocabularyGroup.java b/dhp-common/src/main/java/eu/dnetlib/dhp/common/vocabulary/VocabularyGroup.java index 4c1feac45..64b6f91af 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/common/vocabulary/VocabularyGroup.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/common/vocabulary/VocabularyGroup.java @@ -136,24 +136,21 @@ public class VocabularyGroup implements Serializable { } public Qualifier lookupTermBySynonym(final String vocId, final String syn) { - if (StringUtils.isBlank(vocId)) { - return OafMapperUtils.unknown("", ""); - } - - final Vocabulary vocabulary = vocs.get(vocId.toLowerCase()); - - return Optional - .ofNullable(vocabulary.getTerm(syn)) + return find(vocId) .map( - term -> OafMapperUtils - .qualifier(term.getId(), term.getName(), vocabulary.getId(), vocabulary.getName())) - .orElse( - Optional - .ofNullable(vocabulary.getTermBySynonym(syn)) + vocabulary -> Optional + .ofNullable(vocabulary.getTerm(syn)) .map( term -> OafMapperUtils .qualifier(term.getId(), term.getName(), vocabulary.getId(), vocabulary.getName())) - .orElse(null)); + .orElse( + Optional + .ofNullable(vocabulary.getTermBySynonym(syn)) + .map( + term -> OafMapperUtils + .qualifier(term.getId(), term.getName(), vocabulary.getId(), vocabulary.getName())) + .orElse(null))) + .orElse(null); } /** From a4214ced1e0b71b4af87cf4cdddf05b30f92afb1 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 20 Oct 2023 10:14:20 +0200 Subject: [PATCH 10/60] fixing issue on propagation organization. added --config to workflow definition. added oozie_app to communtiy project --- .../PrepareResultCommunitySet.java | 4 +- .../SparkResultToCommunityFromProject.java | 2 - .../oozie_app/workflow.xml | 4 +- ...t_preparecommunitytoresult_parameters.json | 7 +- .../oozie_app/config-default.xml | 58 +++++++ .../oozie_app/workflow.xml | 144 ++++++++++++++++++ 6 files changed, 207 insertions(+), 12 deletions(-) create mode 100644 dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java index 883f5ca86..c47075272 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java @@ -109,12 +109,12 @@ public class PrepareResultCommunitySet { }); }); - if(cl.size() == 0) + if (cl.size() == 0) return null; rpl.setCommunityList(cl); return rpl; }, Encoders.bean(ResultProjectList.class)) - .filter(Objects::nonNull) + .filter(Objects::nonNull) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/SparkResultToCommunityFromProject.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/SparkResultToCommunityFromProject.java index daef6a317..90c30f812 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/SparkResultToCommunityFromProject.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/SparkResultToCommunityFromProject.java @@ -62,10 +62,8 @@ public class SparkResultToCommunityFromProject implements Serializable { final String possibleupdatespath = parser.get("preparedInfoPath"); log.info("preparedInfoPath: {}", possibleupdatespath); - SparkConf conf = new SparkConf(); - runWithSparkSession( conf, isSparkSessionManaged, diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml index 7a5fd7434..d5d75fbc1 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml @@ -92,7 +92,7 @@ --executor-cores=6 --executor-memory=5G - --spark.executor.memoryOverhead=3g + --conf spark.executor.memoryOverhead=3g --conf spark.sql.shuffle.partitions=3284 --driver-memory=${sparkDriverMemory} --conf spark.extraListeners=${spark2ExtraListeners} @@ -121,7 +121,7 @@ --executor-cores=6 --executor-memory=5G - --spark.executor.memoryOverhead=3g + --conf spark.executor.memoryOverhead=3g --conf spark.sql.shuffle.partitions=3284 --driver-memory=${sparkDriverMemory} --conf spark.extraListeners=${spark2ExtraListeners} diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/input_preparecommunitytoresult_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/input_preparecommunitytoresult_parameters.json index 8b6291e5d..9a50c79fa 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/input_preparecommunitytoresult_parameters.json +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/input_preparecommunitytoresult_parameters.json @@ -5,12 +5,7 @@ "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": "ssm", "paramLongName": "isSparkSessionManaged", diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/oozie_app/config-default.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/oozie_app/config-default.xml new file mode 100644 index 000000000..2744ea92b --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/oozie_app/config-default.xml @@ -0,0 +1,58 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + + hive_metastore_uris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + + spark2YarnHistoryServerAddress + http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089 + + + spark2EventLogDir + /user/spark/spark2ApplicationHistory + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + + + sparkExecutorNumber + 4 + + + sparkDriverMemory + 15G + + + sparkExecutorMemory + 6G + + + sparkExecutorCores + 1 + + + spark2MaxExecutors + 50 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/oozie_app/workflow.xml new file mode 100644 index 000000000..21cc2d887 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/oozie_app/workflow.xml @@ -0,0 +1,144 @@ + + + + sourcePath + the source path + + + + outputPath + the output path + + + + + ${jobTracker} + ${nameNode} + + + oozie.action.sharelib.for.spark + ${oozieActionShareLibForSpark2} + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + + + + + + + + + + + + ${nameNode}/${sourcePath}/relation + ${nameNode}/${outputPath}/relation + + + + + + + + ${nameNode}/${sourcePath}/organization + ${nameNode}/${outputPath}/organization + + + + + + + + ${nameNode}/${sourcePath}/project + ${nameNode}/${outputPath}/project + + + + + + + + ${nameNode}/${sourcePath}/datasource + ${nameNode}/${outputPath}/datasource + + + + + + + + + + yarn + cluster + Prepare-Community-Result-Organization + eu.dnetlib.dhp.resulttocommunityfromproject.PrepareResultCommunitySet + dhp-enrichment-${projectVersion}.jar + + --executor-cores=6 + --executor-memory=5G + --conf spark.executor.memoryOverhead=3g + --conf spark.sql.shuffle.partitions=3284 + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --sourcePath${sourcePath}/relation + --outputPath${workingDir}/preparedInfo/resultCommunityList + --production${production} + + + + + + + + yarn + cluster + community2resultfromproject + eu.dnetlib.dhp.resulttocommunityfromproject.SparkResultToCommunityFromProject + dhp-enrichment-${projectVersion}.jar + + --executor-cores=6 + --executor-memory=5G + --conf spark.executor.memoryOverhead=3g + --conf spark.sql.shuffle.partitions=3284 + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + + --preparedInfoPath${workingDir}/preparedInfo/resultCommunityList + --sourcePath${sourcePath}/ + --outputPath${outputPath}/ + + + + + + + + + + \ No newline at end of file From 89c4dfbaf455d9621e76d0eebf2ca6ea7c718686 Mon Sep 17 00:00:00 2001 From: dimitrispie Date: Tue, 24 Oct 2023 09:48:23 +0300 Subject: [PATCH 11/60] StatsDB workflow to export actionsets about OA routes, diamond, and publicly-funded A new oozie workflow capable to read from the stats db to produce a new actionSet for updating results with: - green_oa ={true, false} - openAccesColor = {gold, hybrid, bronze} - in_diamond_journal={true, false} - publicly_funded={true, false} Inputs: - outputPath - statsDB --- dhp-workflows/dhp-stats-actionsets/pom.xml | 114 +++++++ .../dnetlib/dhp/actionmanager/Constants.java | 102 +++++++ .../StatsAtomicActionsJob.java | 286 ++++++++++++++++++ .../stats_actionsets/StatsDiamondOAModel.java | 29 ++ .../stats_actionsets/StatsGreenOAModel.java | 29 ++ .../stats_actionsets/StatsOAColourModel.java | 47 +++ .../StatsPubliclyFundedModel.java | 29 ++ .../input_actionset_parameter.json | 32 ++ .../oozie_app/config-default.xml | 30 ++ .../stats_actionsets/oozie_app/workflow.xml | 125 ++++++++ .../src/main/resources/log4j.properties | 12 + 11 files changed, 835 insertions(+) create mode 100644 dhp-workflows/dhp-stats-actionsets/pom.xml create mode 100644 dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java create mode 100644 dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsAtomicActionsJob.java create mode 100644 dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsDiamondOAModel.java create mode 100644 dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsGreenOAModel.java create mode 100644 dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsOAColourModel.java create mode 100644 dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsPubliclyFundedModel.java create mode 100644 dhp-workflows/dhp-stats-actionsets/src/main/resources/eu/dnetlib/dhp/actionmanager/stats_actionsets/input_actionset_parameter.json create mode 100644 dhp-workflows/dhp-stats-actionsets/src/main/resources/eu/dnetlib/dhp/actionmanager/stats_actionsets/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-stats-actionsets/src/main/resources/eu/dnetlib/dhp/actionmanager/stats_actionsets/oozie_app/workflow.xml create mode 100644 dhp-workflows/dhp-stats-actionsets/src/main/resources/log4j.properties diff --git a/dhp-workflows/dhp-stats-actionsets/pom.xml b/dhp-workflows/dhp-stats-actionsets/pom.xml new file mode 100644 index 000000000..670fd4a5e --- /dev/null +++ b/dhp-workflows/dhp-stats-actionsets/pom.xml @@ -0,0 +1,114 @@ + + + 4.0.0 + + eu.dnetlib.dhp + dhp-workflows + 1.2.5-SNAPSHOT + + dhp-stats-actionsets + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + org.apache.httpcomponents + httpclient + + + + org.apache.spark + spark-core_2.11 + + + org.apache.spark + spark-sql_2.11 + + + + eu.dnetlib.dhp + dhp-common + ${project.version} + + + + net.sf.saxon + Saxon-HE + + + dom4j + dom4j + + + + xml-apis + xml-apis + + + + jaxen + jaxen + + + + org.json + json + + + + + org.apache.poi + poi-ooxml + + + + + org.apache.commons + commons-compress + + + + org.mongodb + mongo-java-driver + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java new file mode 100644 index 000000000..0df34364d --- /dev/null +++ b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java @@ -0,0 +1,102 @@ + +package eu.dnetlib.dhp.actionmanager; + +import java.util.Optional; + +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.SparkSession; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.StructuredProperty; +import eu.dnetlib.dhp.schema.oaf.Subject; +import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; + +public class Constants { + + public static final String DOI = "doi"; + public static final String DOI_CLASSNAME = "Digital Object Identifier"; + + public static final String DEFAULT_DELIMITER = ","; + public static final String DEFAULT_FOS_DELIMITER = "\t"; + + public static final String UPDATE_DATA_INFO_TYPE = "update"; +// public static final String UPDATE_SUBJECT_FOS_CLASS_ID = "subject:fos"; + public static final String UPDATE_CLASS_NAME = "Inferred by OpenAIRE"; +// public static final String UPDATE_MEASURE_BIP_CLASS_ID = "measure:bip"; +// public static final String UPDATE_SUBJECT_SDG_CLASS_ID = "subject:sdg"; +// public static final String UPDATE_MEASURE_USAGE_COUNTS_CLASS_ID = "measure:usage_counts"; +// public static final String UPDATE_KEY_USAGE_COUNTS = "count"; + + public static final String UPDATE_MEASURE_STATS_MODEL_CLASS_ID = "measure:stats_model"; + public static final String UPDATE_KEY_STATS_MODEL = "stats_model"; + +// public static final String UPDATE_MEASURE_PUBLICLY_FUNDED_CLASS_ID = "measure:publicly_funded"; +// public static final String UPDATE_KEY_PUBLICLY_FUNDED = "publicly_funded"; + +// public static final String FOS_CLASS_ID = "FOS"; +// public static final String FOS_CLASS_NAME = "Fields of Science and Technology classification"; +// +// public static final String SDG_CLASS_ID = "SDG"; +// public static final String SDG_CLASS_NAME = "Sustainable Development Goals"; + + public static final String NULL = "NULL"; + + public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private Constants() { + } + + public static Boolean isSparkSessionManaged(ArgumentApplicationParser parser) { + return Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + } + + public static Dataset readPath( + SparkSession spark, String inputPath, Class clazz) { + return spark + .read() + .textFile(inputPath) + .map((MapFunction) value -> OBJECT_MAPPER.readValue(value, clazz), Encoders.bean(clazz)); + } + + public static Subject getSubject(String sbj, String classid, String classname, + String diqualifierclassid) { + if (sbj == null || sbj.equals(NULL)) + return null; + Subject s = new Subject(); + s.setValue(sbj); + s + .setQualifier( + OafMapperUtils + .qualifier( + classid, + classname, + ModelConstants.DNET_SUBJECT_TYPOLOGIES, + ModelConstants.DNET_SUBJECT_TYPOLOGIES)); + s + .setDataInfo( + OafMapperUtils + .dataInfo( + false, + UPDATE_DATA_INFO_TYPE, + true, + false, + OafMapperUtils + .qualifier( + diqualifierclassid, + UPDATE_CLASS_NAME, + ModelConstants.DNET_PROVENANCE_ACTIONS, + ModelConstants.DNET_PROVENANCE_ACTIONS), + "")); + + return s; + + } +} diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsAtomicActionsJob.java b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsAtomicActionsJob.java new file mode 100644 index 000000000..f6e2cfe01 --- /dev/null +++ b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsAtomicActionsJob.java @@ -0,0 +1,286 @@ + +package eu.dnetlib.dhp.actionmanager.stats_actionsets; + +import static eu.dnetlib.dhp.actionmanager.Constants.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.SequenceFileOutputFormat; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.MapGroupsFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; +import scala.Tuple2; + +/** + * created the Atomic Action for each type of results + */ +public class StatsAtomicActionsJob implements Serializable { + + private static final Logger log = LoggerFactory.getLogger(StatsAtomicActionsJob.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(String[] args) throws Exception { + + String jsonConfiguration = IOUtils + .toString( + StatsAtomicActionsJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/actionmanager/stats_actionsets/input_actionset_parameter.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath {}: ", outputPath); + + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + + final String dbname = parser.get("statsDB"); + + final String workingPath = parser.get("workingPath"); + + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + prepareGreenData(dbname, spark, workingPath + "/greenOADB", "indi_pub_green_oa", "id"); + prepareDiamondData(dbname, spark, workingPath + "/diamondOADΒ", "indi_pub_diamond", "id"); + preparePubliclyFundedData( + dbname, spark, workingPath + "/publiclyFundedDΒ", "indi_funded_result_with_fundref", "id"); + prepareOAColourData(dbname, spark, workingPath + "/oacolourDB", "", "id"); + writeActionSet(spark, workingPath, outputPath); + }); + } + + private static void prepareGreenData(String dbname, SparkSession spark, String workingPath, String tableName, + String resultAttributeName) { + spark + .sql( + String + .format( + "select %s as id, green_oa as green_oa " + + "from %s.%s", + resultAttributeName, dbname, tableName)) + .as(Encoders.bean(StatsGreenOAModel.class)) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(workingPath); + } + + private static void prepareDiamondData(String dbname, SparkSession spark, String workingPath, String tableName, + String resultAttributeName) { + spark + .sql( + String + .format( + "select %s as id, in_diamond_journal as in_diamond_journal " + + "from %s.%s", + resultAttributeName, dbname, tableName)) + .as(Encoders.bean(StatsDiamondOAModel.class)) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(workingPath); + } + + private static void preparePubliclyFundedData(String dbname, SparkSession spark, String workingPath, + String tableName, + String resultAttributeName) { + spark + .sql( + String + .format( + "select %s as id, fundref as publicly_funded " + + "from %s.%s", + resultAttributeName, dbname, tableName)) + .as(Encoders.bean(StatsPubliclyFundedModel.class)) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(workingPath); + } + + private static void prepareOAColourData(String dbname, SparkSession spark, String workingPath, String tableName, + String resultAttributeName) { + spark + .sql( + String + .format( + "select b.%s as id, is_gold, is_bronze_oa, is_hybrid from %s.indi_pub_bronze_oa b " + + "left outer join %s.indi_pub_gold_oa g on g.id=b.id " + + "left outer join %s.indi_pub_hybrid h on b.id=h.id", + resultAttributeName, dbname, dbname, dbname)) + .as(Encoders.bean(StatsOAColourModel.class)) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(workingPath); + } + + public static void writeActionSet(SparkSession spark, String inputPath, String outputPath) { + + getFinalIndicatorsGreenResult(spark, inputPath + "/greenOADB") + .toJavaRDD() + .map(p -> new AtomicAction(p.getClass(), p)) + .union( + getFinalIndicatorsDiamondResult(spark, inputPath + "/diamondOADΒ") + .toJavaRDD() + .map(p -> new AtomicAction(p.getClass(), p))) + .union( + getFinalIndicatorsPubliclyFundedResult(spark, inputPath + "/publiclyFundedDΒ") + .toJavaRDD() + .map(p -> new AtomicAction(p.getClass(), p))) + .union( + getFinalIndicatorsOAColourResult(spark, inputPath + "/oacolourDB") + .toJavaRDD() + .map(p -> new AtomicAction(p.getClass(), p))) + .mapToPair( + aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), + new Text(OBJECT_MAPPER.writeValueAsString(aa)))) + .saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class); + } + + public static Measure newMeasureInstance(String id) { + Measure m = new Measure(); + m.setId(id); + m.setUnit(new ArrayList<>()); + return m; + } + + private static Dataset getFinalIndicatorsGreenResult(SparkSession spark, String inputPath) { + + return readPath(spark, inputPath, StatsGreenOAModel.class) + .map((MapFunction) usm -> { + Result r = new Result(); + r.setId("50|" + usm.getId()); + r.setMeasures(getMeasure(usm.isGreen_oa(), "green_oa")); + return r; + }, Encoders.bean(Result.class)); + } + + private static Dataset getFinalIndicatorsDiamondResult(SparkSession spark, String inputPath) { + + return readPath(spark, inputPath, StatsDiamondOAModel.class) + .map((MapFunction) usm -> { + Result r = new Result(); + r.setId("50|" + usm.getId()); + r.setMeasures(getMeasure(usm.isIn_diamond_journal(), "in_diamond_journal")); + return r; + }, Encoders.bean(Result.class)); + } + + private static Dataset getFinalIndicatorsPubliclyFundedResult(SparkSession spark, String inputPath) { + + return readPath(spark, inputPath, StatsPubliclyFundedModel.class) + .map((MapFunction) usm -> { + Result r = new Result(); + r.setId("50|" + usm.getId()); + r.setMeasures(getMeasure(usm.isPublicly_funded(), "publicly_funded")); + return r; + }, Encoders.bean(Result.class)); + } + + private static Dataset getFinalIndicatorsOAColourResult(SparkSession spark, String inputPath) { + + return readPath(spark, inputPath, StatsOAColourModel.class) + .map((MapFunction) usm -> { + Result r = new Result(); + r.setId("50|" + usm.getId()); + r.setMeasures(getMeasureOAColour(usm.isIs_gold(), usm.isIs_bronze_oa(), usm.isIs_hybrid())); + return r; + }, Encoders.bean(Result.class)); + } + + private static List getMeasure(Boolean is_model_oa, String model_type) { + DataInfo dataInfo = OafMapperUtils + .dataInfo( + false, + UPDATE_DATA_INFO_TYPE, + true, + false, + OafMapperUtils + .qualifier( + UPDATE_MEASURE_STATS_MODEL_CLASS_ID, + UPDATE_CLASS_NAME, + ModelConstants.DNET_PROVENANCE_ACTIONS, + ModelConstants.DNET_PROVENANCE_ACTIONS), + ""); + + return Arrays + .asList( + OafMapperUtils + .newMeasureInstance(model_type, String.valueOf(is_model_oa), UPDATE_KEY_STATS_MODEL, dataInfo)); + } + + private static List getMeasureOAColour(Boolean is_gold, Boolean is_bronze_oa, Boolean is_hybrid) { + DataInfo dataInfo = OafMapperUtils + .dataInfo( + false, + UPDATE_DATA_INFO_TYPE, + true, + false, + OafMapperUtils + .qualifier( + UPDATE_MEASURE_STATS_MODEL_CLASS_ID, + UPDATE_CLASS_NAME, + ModelConstants.DNET_PROVENANCE_ACTIONS, + ModelConstants.DNET_PROVENANCE_ACTIONS), + ""); + + return Arrays + .asList( + OafMapperUtils + .newMeasureInstance("is_gold", String.valueOf(is_gold), UPDATE_KEY_STATS_MODEL, dataInfo), + OafMapperUtils + .newMeasureInstance("is_bronze_oa", String.valueOf(is_bronze_oa), UPDATE_KEY_STATS_MODEL, dataInfo), + OafMapperUtils + .newMeasureInstance("is_hybrid", String.valueOf(is_hybrid), UPDATE_KEY_STATS_MODEL, dataInfo)); + + } + + private static void removeOutputDir(SparkSession spark, String path) { + HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); + } + + public static Dataset readPath( + SparkSession spark, String inputPath, Class clazz) { + return spark + .read() + .textFile(inputPath) + .map((MapFunction) value -> OBJECT_MAPPER.readValue(value, clazz), Encoders.bean(clazz)); + } + +} diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsDiamondOAModel.java b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsDiamondOAModel.java new file mode 100644 index 000000000..340790fa8 --- /dev/null +++ b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsDiamondOAModel.java @@ -0,0 +1,29 @@ + +package eu.dnetlib.dhp.actionmanager.stats_actionsets; + +import java.io.Serializable; + +/** + * @author dimitris.pierrakos + * @Date 30/10/23 + */ +public class StatsDiamondOAModel implements Serializable { + private String id; + private boolean in_diamond_journal; + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public boolean isIn_diamond_journal() { + return in_diamond_journal; + } + + public void setIn_diamond_journal(boolean in_diamond_journal) { + this.in_diamond_journal = in_diamond_journal; + } +} diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsGreenOAModel.java b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsGreenOAModel.java new file mode 100644 index 000000000..3d59bdf95 --- /dev/null +++ b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsGreenOAModel.java @@ -0,0 +1,29 @@ + +package eu.dnetlib.dhp.actionmanager.stats_actionsets; + +import java.io.Serializable; + +/** + * @author dimitris.pierrakos + * @Date 30/10/23 + */ +public class StatsGreenOAModel implements Serializable { + private String id; + private boolean green_oa; + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public boolean isGreen_oa() { + return green_oa; + } + + public void setGreen_oa(boolean green_oa) { + this.green_oa = green_oa; + } +} diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsOAColourModel.java b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsOAColourModel.java new file mode 100644 index 000000000..a83eb4f60 --- /dev/null +++ b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsOAColourModel.java @@ -0,0 +1,47 @@ + +package eu.dnetlib.dhp.actionmanager.stats_actionsets; + +import java.io.Serializable; + +/** + * @author dimitris.pierrakos + * @Date 30/10/23 + */ +public class StatsOAColourModel implements Serializable { + private String id; + private boolean is_gold; + private boolean is_bronze_oa; + private boolean is_hybrid; + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public boolean isIs_gold() { + return is_gold; + } + + public void setIs_gold(boolean is_gold) { + this.is_gold = is_gold; + } + + public boolean isIs_bronze_oa() { + return is_bronze_oa; + } + + public void setIs_bronze_oa(boolean is_bronze_oa) { + this.is_bronze_oa = is_bronze_oa; + } + + public boolean isIs_hybrid() { + return is_hybrid; + } + + public void setIs_hybrid(boolean is_hybrid) { + this.is_hybrid = is_hybrid; + } +} diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsPubliclyFundedModel.java b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsPubliclyFundedModel.java new file mode 100644 index 000000000..6bdc78220 --- /dev/null +++ b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsPubliclyFundedModel.java @@ -0,0 +1,29 @@ + +package eu.dnetlib.dhp.actionmanager.stats_actionsets; + +import java.io.Serializable; + +/** + * @author dimitris.pierrakos + * @Date 30/10/23 + */ +public class StatsPubliclyFundedModel implements Serializable { + private String id; + private boolean publicly_funded; + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public boolean isPublicly_funded() { + return publicly_funded; + } + + public void setPublicly_funded(boolean publicly_funded) { + this.publicly_funded = publicly_funded; + } +} diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/resources/eu/dnetlib/dhp/actionmanager/stats_actionsets/input_actionset_parameter.json b/dhp-workflows/dhp-stats-actionsets/src/main/resources/eu/dnetlib/dhp/actionmanager/stats_actionsets/input_actionset_parameter.json new file mode 100644 index 000000000..bb24d4c99 --- /dev/null +++ b/dhp-workflows/dhp-stats-actionsets/src/main/resources/eu/dnetlib/dhp/actionmanager/stats_actionsets/input_actionset_parameter.json @@ -0,0 +1,32 @@ +[ + { + "paramName": "issm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "when true will stop SparkSession after job execution", + "paramRequired": false + }, + { + "paramName": "hmu", + "paramLongName": "hive_metastore_uris", + "paramDescription": "the URI for the hive metastore", + "paramRequired": true + }, + { + "paramName": "o", + "paramLongName": "outputPath", + "paramDescription": "the path of the new ActionSet", + "paramRequired": true + }, + { + "paramName": "sdb", + "paramLongName": "statsDB", + "paramDescription": "the name of the stats db to be used", + "paramRequired": true + }, + { + "paramName": "wp", + "paramLongName": "workingPath", + "paramDescription": "the workingPath where to save the content of the usage_stats table", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/resources/eu/dnetlib/dhp/actionmanager/stats_actionsets/oozie_app/config-default.xml b/dhp-workflows/dhp-stats-actionsets/src/main/resources/eu/dnetlib/dhp/actionmanager/stats_actionsets/oozie_app/config-default.xml new file mode 100644 index 000000000..d262cb6e0 --- /dev/null +++ b/dhp-workflows/dhp-stats-actionsets/src/main/resources/eu/dnetlib/dhp/actionmanager/stats_actionsets/oozie_app/config-default.xml @@ -0,0 +1,30 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + hiveMetastoreUris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + + hiveJdbcUrl + jdbc:hive2://iis-cdh5-test-m3.ocean.icm.edu.pl:10000 + + + hiveDbName + openaire + + + oozie.launcher.mapreduce.user.classpath.first + true + + diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/resources/eu/dnetlib/dhp/actionmanager/stats_actionsets/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-actionsets/src/main/resources/eu/dnetlib/dhp/actionmanager/stats_actionsets/oozie_app/workflow.xml new file mode 100644 index 000000000..1aa4e1050 --- /dev/null +++ b/dhp-workflows/dhp-stats-actionsets/src/main/resources/eu/dnetlib/dhp/actionmanager/stats_actionsets/oozie_app/workflow.xml @@ -0,0 +1,125 @@ + + + + outputPath + the path where to store the actionset + + + statsDB + the name of the stats db to be used + + + sparkDriverMemory + memory for driver process + + + sparkExecutorMemory + memory for individual executor + + + sparkExecutorCores + number of cores used by single executor + + + oozieActionShareLibForSpark2 + oozie action sharelib for spark 2.* + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + spark 2.* extra listeners classname + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + spark 2.* sql query execution listeners classname + + + spark2YarnHistoryServerAddress + spark 2.* yarn history server address + + + spark2EventLogDir + spark 2.* event log dir location + + + + + ${jobTracker} + ${nameNode} + + + mapreduce.job.queuename + ${queueName} + + + oozie.launcher.mapred.job.queue.name + ${oozieLauncherQueueName} + + + oozie.action.sharelib.for.spark + ${oozieActionShareLibForSpark2} + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + yarn + cluster + Produces the atomic action with the usage stats count for results + + eu.dnetlib.dhp.actionmanager.stats_actionsets.SparkAtomicActionGreenOAJob + dhp-stats-actionsets-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + + --hive_metastore_uris${hiveMetastoreUris} + --outputPath${outputPath} + --statsDB${statsDB} + --workingPath${workingDir} + + + + + + + + yarn + cluster + Produces the atomic action with the stats green_oa for results + + eu.dnetlib.dhp.actionmanager.stats_actionsets.StatsAtomicActionsJob + dhp-stats-actionsets-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + + --hive_metastore_uris${hiveMetastoreUris} + --outputPath${outputPath} + --statsDB${statsDB} + --workingPath${workingDir} + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/resources/log4j.properties b/dhp-workflows/dhp-stats-actionsets/src/main/resources/log4j.properties new file mode 100644 index 000000000..81458d1f7 --- /dev/null +++ b/dhp-workflows/dhp-stats-actionsets/src/main/resources/log4j.properties @@ -0,0 +1,12 @@ +# Set root logger level to DEBUG and its only appender to A1. +log4j.rootLogger=INFO, A1 + +# A1 is set to be a ConsoleAppender. +log4j.appender.A1=org.apache.log4j.ConsoleAppender + +# A1 uses PatternLayout. +log4j.appender.A1.layout=org.apache.log4j.PatternLayout +log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n + +log4j.logger.org.apache.spark=FATAL +log4j.logger.org.spark_project=FATAL From c80b768af062166e4ae94abcf529af0aac994b7b Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 31 Oct 2023 15:49:42 +0100 Subject: [PATCH 12/60] test for project propagation --- .../PrepareResultCommunitySet.java | 2 +- .../PrepareAssocTest.java | 14 +- .../ResultToCommunityJobTest.java | 257 ++---------------- 3 files changed, 36 insertions(+), 237 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java index c47075272..9a2f25fa6 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java @@ -56,7 +56,7 @@ public class PrepareResultCommunitySet { log.info("production: {}", production); final CommunityEntityMap projectsMap = Utils.getCommunityProjects(production); - log.info("projectsMap: {}", new Gson().toJson(projectsMap)); + //log.info("projectsMap: {}", new Gson().toJson(projectsMap)); SparkConf conf = new SparkConf(); diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareAssocTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareAssocTest.java index 0e10b3edf..172873af3 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareAssocTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareAssocTest.java @@ -10,9 +10,7 @@ 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.BeforeAll; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,6 +59,7 @@ public class PrepareAssocTest { spark.stop(); } + @Disabled @Test void test1() throws Exception { @@ -72,8 +71,8 @@ public class PrepareAssocTest { "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/resulttocommunityfromproject/relation/").getPath(), "-outputPath", workingDir.toString() + "/prepared", - "-production", Boolean.TRUE.toString(), - "-hive_metastore_uris", "" + "-production", Boolean.TRUE.toString() + }); final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); @@ -82,7 +81,10 @@ public class PrepareAssocTest { .textFile(workingDir.toString() + "/prepared") .map(item -> new ObjectMapper().readValue(item, ResultProjectList.class)); - tmp.foreach(r -> System.out.println(new ObjectMapper().writeValueAsString(r))); + Assertions.assertEquals(4, tmp.count()); + Assertions.assertEquals(2, tmp.filter(rpl -> rpl.getCommunityList().contains("aurora")).count()); + Assertions.assertEquals(1, tmp.filter(rpl -> rpl.getCommunityList().contains("sdsn-gr")).count()); + Assertions.assertEquals(1, tmp.filter(rpl -> rpl.getCommunityList().contains("netherlands")).count()); } } diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/ResultToCommunityJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/ResultToCommunityJobTest.java index 6a5726cbe..b705af8ac 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/ResultToCommunityJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/ResultToCommunityJobTest.java @@ -6,7 +6,9 @@ import static org.apache.spark.sql.functions.desc; import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.util.List; +import eu.dnetlib.dhp.schema.oaf.Context; import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; @@ -89,235 +91,30 @@ public class ResultToCommunityJobTest { .textFile(workingDir.toString() + "/dataset") .map(item -> OBJECT_MAPPER.readValue(item, Dataset.class)); - tmp.foreach(d -> System.out.println(new ObjectMapper().writeValueAsString(d))); -// Assertions.assertEquals(10, tmp.count()); -// org.apache.spark.sql.Dataset verificationDataset = spark -// .createDataset(tmp.rdd(), Encoders.bean(Dataset.class)); -// -// verificationDataset.createOrReplaceTempView("dataset"); -// -// String query = "select id, MyT.id community " -// + "from dataset " -// + "lateral view explode(context) c as MyT " -// + "lateral view explode(MyT.datainfo) d as MyD " -// + "where MyD.inferenceprovenance = 'propagation'"; -// -// org.apache.spark.sql.Dataset resultExplodedProvenance = spark.sql(query); -// Assertions.assertEquals(5, resultExplodedProvenance.count()); -// Assertions -// .assertEquals( -// 0, -// resultExplodedProvenance -// .filter("id = '50|dedup_wf_001::afaf128022d29872c4dad402b2db04fe'") -// .count()); -// Assertions -// .assertEquals( -// 1, -// resultExplodedProvenance -// .filter("id = '50|dedup_wf_001::3f62cfc27024d564ea86760c494ba93b'") -// .count()); -// Assertions -// .assertEquals( -// "beopen", -// resultExplodedProvenance -// .select("community") -// .where( -// resultExplodedProvenance -// .col("id") -// .equalTo( -// "50|dedup_wf_001::3f62cfc27024d564ea86760c494ba93b")) -// .collectAsList() -// .get(0) -// .getString(0)); -// -// Assertions -// .assertEquals( -// 2, -// resultExplodedProvenance -// .filter("id = '50|od________18::8887b1df8b563c4ea851eb9c882c9d7b'") -// .count()); -// Assertions -// .assertEquals( -// "mes", -// resultExplodedProvenance -// .select("community") -// .where( -// resultExplodedProvenance -// .col("id") -// .equalTo( -// "50|od________18::8887b1df8b563c4ea851eb9c882c9d7b")) -// .sort(desc("community")) -// .collectAsList() -// .get(0) -// .getString(0)); -// Assertions -// .assertEquals( -// "euromarine", -// resultExplodedProvenance -// .select("community") -// .where( -// resultExplodedProvenance -// .col("id") -// .equalTo( -// "50|od________18::8887b1df8b563c4ea851eb9c882c9d7b")) -// .sort(desc("community")) -// .collectAsList() -// .get(1) -// .getString(0)); -// -// Assertions -// .assertEquals( -// 1, -// resultExplodedProvenance -// .filter("id = '50|doajarticles::8d817039a63710fcf97e30f14662c6c8'") -// .count()); -// Assertions -// .assertEquals( -// "mes", -// resultExplodedProvenance -// .select("community") -// .where( -// resultExplodedProvenance -// .col("id") -// .equalTo( -// "50|doajarticles::8d817039a63710fcf97e30f14662c6c8")) -// .sort(desc("community")) -// .collectAsList() -// .get(0) -// .getString(0)); -// -// Assertions -// .assertEquals( -// 1, -// resultExplodedProvenance -// .filter("id = '50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6'") -// .count()); -// Assertions -// .assertEquals( -// "mes", -// resultExplodedProvenance -// .select("community") -// .where( -// resultExplodedProvenance -// .col("id") -// .equalTo( -// "50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6")) -// .sort(desc("community")) -// .collectAsList() -// .get(0) -// .getString(0)); -// -// query = "select id, MyT.id community " -// + "from dataset " -// + "lateral view explode(context) c as MyT " -// + "lateral view explode(MyT.datainfo) d as MyD "; -// -// org.apache.spark.sql.Dataset resultCommunityId = spark.sql(query); -// -// Assertions.assertEquals(10, resultCommunityId.count()); -// -// Assertions -// .assertEquals( -// 1, -// resultCommunityId -// .filter("id = '50|dedup_wf_001::afaf128022d29872c4dad402b2db04fe'") -// .count()); -// Assertions -// .assertEquals( -// "beopen", -// resultCommunityId -// .select("community") -// .where( -// resultCommunityId -// .col("id") -// .equalTo( -// "50|dedup_wf_001::afaf128022d29872c4dad402b2db04fe")) -// .collectAsList() -// .get(0) -// .getString(0)); -// -// Assertions -// .assertEquals( -// 1, -// resultCommunityId -// .filter("id = '50|dedup_wf_001::3f62cfc27024d564ea86760c494ba93b'") -// .count()); -// -// Assertions -// .assertEquals( -// 3, -// resultCommunityId -// .filter("id = '50|od________18::8887b1df8b563c4ea851eb9c882c9d7b'") -// .count()); -// Assertions -// .assertEquals( -// "beopen", -// resultCommunityId -// .select("community") -// .where( -// resultCommunityId -// .col("id") -// .equalTo( -// "50|od________18::8887b1df8b563c4ea851eb9c882c9d7b")) -// .sort(desc("community")) -// .collectAsList() -// .get(2) -// .getString(0)); -// -// Assertions -// .assertEquals( -// 2, -// resultCommunityId -// .filter("id = '50|doajarticles::8d817039a63710fcf97e30f14662c6c8'") -// .count()); -// Assertions -// .assertEquals( -// "euromarine", -// resultCommunityId -// .select("community") -// .where( -// resultCommunityId -// .col("id") -// .equalTo( -// "50|doajarticles::8d817039a63710fcf97e30f14662c6c8")) -// .sort(desc("community")) -// .collectAsList() -// .get(1) -// .getString(0)); -// -// Assertions -// .assertEquals( -// 3, -// resultCommunityId -// .filter("id = '50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6'") -// .count()); -// Assertions -// .assertEquals( -// "euromarine", -// resultCommunityId -// .select("community") -// .where( -// resultCommunityId -// .col("id") -// .equalTo( -// "50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6")) -// .sort(desc("community")) -// .collectAsList() -// .get(2) -// .getString(0)); -// Assertions -// .assertEquals( -// "ni", -// resultCommunityId -// .select("community") -// .where( -// resultCommunityId -// .col("id") -// .equalTo( -// "50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6")) -// .sort(desc("community")) -// .collectAsList() -// .get(0) -// .getString(0)); + Assertions.assertEquals(10, tmp.count()); + /** + * {"resultId":"50|57a035e5b1ae::d5be548ca7ae489d762f893be67af52f","communityList":["aurora"]} + * {"resultId":"50|57a035e5b1ae::a77232ffca9115fcad51c3503dbc7e3e","communityList":["aurora"]} + * {"resultId":"50|57a035e5b1ae::803aaad4decab7e27cd4b52a1931b3a1","communityList":["sdsn-gr"]} + * {"resultId":"50|57a035e5b1ae::a02e9e4087bca50687731ae5c765b5e1","communityList":["netherlands"]} + */ + List context = tmp.filter(r -> r.getId().equals("50|57a035e5b1ae::d5be548ca7ae489d762f893be67af52f")) + .first().getContext(); + Assertions.assertTrue(context.stream().anyMatch(c -> containsResultCommunityProject(c))); + + context = tmp.filter(r -> r.getId().equals("50|57a035e5b1ae::a77232ffca9115fcad51c3503dbc7e3e")) + .first().getContext(); + Assertions.assertTrue(context.stream().anyMatch(c -> containsResultCommunityProject(c))); + + Assertions.assertEquals(0, tmp.filter(r -> r.getId().equals("50|57a035e5b1ae::803aaad4decab7e27cd4b52a1931b3a1")).count()); + + Assertions.assertEquals(0, tmp.filter(r -> r.getId().equals("50|57a035e5b1ae::a02e9e4087bca50687731ae5c765b5e1")).count()); + + Assertions.assertEquals(2, tmp.filter(r -> r.getContext().stream().anyMatch(c -> c.getId().equals("aurora"))).count()); + + } + + private static boolean containsResultCommunityProject(Context c) { + return c.getDataInfo().stream().anyMatch(di -> di.getProvenanceaction().getClassid().equals("result:community:project")); } } From a737dd47b6268111a18550a3e94241ec8358447c Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 31 Oct 2023 15:54:49 +0100 Subject: [PATCH 13/60] removed not needed test class --- .../dnetlib/dhp/bulktag/BulkTagJobTest.java | 2 +- .../PrepareAssocTest.java | 90 ------------------- 2 files changed, 1 insertion(+), 91 deletions(-) delete mode 100644 dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareAssocTest.java diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java index 7cbbcaafb..5e106ef45 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java @@ -6,7 +6,7 @@ import static eu.dnetlib.dhp.bulktag.community.TaggingConstants.ZENODO_COMMUNITY import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; -import java.util.HashMap; + import java.util.List; import org.apache.commons.io.FileUtils; diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareAssocTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareAssocTest.java deleted file mode 100644 index 172873af3..000000000 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareAssocTest.java +++ /dev/null @@ -1,90 +0,0 @@ - -package eu.dnetlib.dhp.resulttocommunityfromproject; - -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; - -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.*; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.fasterxml.jackson.databind.ObjectMapper; - -import eu.dnetlib.dhp.bulktag.BulkTagJobTest; -import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList; - -/** - * @author miriam.baglioni - * @Date 13/10/23 - */ -public class PrepareAssocTest { - - private static SparkSession spark; - - private static Path workingDir; - - private static final Logger log = LoggerFactory.getLogger(PrepareAssocTest.class); - - @BeforeAll - public static void beforeAll() throws IOException { - workingDir = Files.createTempDirectory(BulkTagJobTest.class.getSimpleName()); - log.info("using work dir {}", workingDir); - - SparkConf conf = new SparkConf(); - conf.setAppName(BulkTagJobTest.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(PrepareAssocTest.class.getSimpleName()) - .config(conf) - .getOrCreate(); - } - - @AfterAll - public static void afterAll() throws IOException { - FileUtils.deleteDirectory(workingDir.toFile()); - spark.stop(); - } - - @Disabled - @Test - void test1() throws Exception { - - PrepareResultCommunitySet - .main( - new String[] { - - "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", - getClass().getResource("/eu/dnetlib/dhp/resulttocommunityfromproject/relation/").getPath(), - "-outputPath", workingDir.toString() + "/prepared", - "-production", Boolean.TRUE.toString() - - }); - - final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - - JavaRDD tmp = sc - .textFile(workingDir.toString() + "/prepared") - .map(item -> new ObjectMapper().readValue(item, ResultProjectList.class)); - - Assertions.assertEquals(4, tmp.count()); - Assertions.assertEquals(2, tmp.filter(rpl -> rpl.getCommunityList().contains("aurora")).count()); - Assertions.assertEquals(1, tmp.filter(rpl -> rpl.getCommunityList().contains("sdsn-gr")).count()); - Assertions.assertEquals(1, tmp.filter(rpl -> rpl.getCommunityList().contains("netherlands")).count()); - } - -} From 937ff6a7c77fa6422463471d8f3aff9c22ff775b Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 31 Oct 2023 15:56:08 +0100 Subject: [PATCH 14/60] - --- .../dhp/api/QueryCommunityAPITest.java | 119 ------------------ .../dnetlib/dhp/bulktag/BulkTagJobTest.java | 2 - 2 files changed, 121 deletions(-) delete mode 100644 dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/api/QueryCommunityAPITest.java diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/api/QueryCommunityAPITest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/api/QueryCommunityAPITest.java deleted file mode 100644 index 6ee01a6f0..000000000 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/api/QueryCommunityAPITest.java +++ /dev/null @@ -1,119 +0,0 @@ - -package eu.dnetlib.dhp.api; - -import java.util.List; - -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; - -import eu.dnetlib.dhp.api.QueryCommunityAPI; -import eu.dnetlib.dhp.api.Utils; -import eu.dnetlib.dhp.api.model.CommunityEntityMap; -import eu.dnetlib.dhp.api.model.CommunityModel; -import eu.dnetlib.dhp.api.model.CommunitySummary; -import eu.dnetlib.dhp.api.model.DatasourceList; -import eu.dnetlib.dhp.bulktag.community.Community; -import eu.dnetlib.dhp.bulktag.community.CommunityConfiguration; - -/** - * @author miriam.baglioni - * @Date 06/10/23 - */ -public class QueryCommunityAPITest { - - @Test - void communityList() throws Exception { - String body = QueryCommunityAPI.communities(true); - new ObjectMapper() - .readValue(body, CommunitySummary.class) - .forEach(p -> { - try { - System.out.println(new ObjectMapper().writeValueAsString(p)); - } catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - }); - } - - @Test - void community() throws Exception { - String id = "dh-ch"; - String body = QueryCommunityAPI.community(id, true); - System.out - .println( - new ObjectMapper() - .writeValueAsString( - new ObjectMapper() - .readValue(body, CommunityModel.class))); - } - - @Test - void communityDatasource() throws Exception { - String id = "dh-ch"; - String body = QueryCommunityAPI.communityDatasource(id, true); - new ObjectMapper() - .readValue(body, DatasourceList.class) - .forEach(ds -> { - try { - System.out.println(new ObjectMapper().writeValueAsString(ds)); - } catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - }); - ; - } - - @Test - void validCommunities() throws Exception { - CommunityConfiguration cc = Utils.getCommunityConfiguration(true); - System.out.println(cc.getCommunities().keySet()); - Community community = cc.getCommunities().get("aurora"); - Assertions.assertEquals(0, community.getSubjects().size()); - Assertions.assertEquals(null, community.getConstraints()); - Assertions.assertEquals(null, community.getRemoveConstraints()); - Assertions.assertEquals(2, community.getZenodoCommunities().size()); - Assertions - .assertTrue( - community.getZenodoCommunities().stream().anyMatch(c -> c.equals("aurora-universities-network"))); - Assertions - .assertTrue(community.getZenodoCommunities().stream().anyMatch(c -> c.equals("university-of-innsbruck"))); - Assertions.assertEquals(35, community.getProviders().size()); - Assertions - .assertEquals( - 35, community.getProviders().stream().filter(p -> p.getSelectionConstraints() == null).count()); - - } - - @Test - void eutopiaCommunityConfiguration() throws Exception { - CommunityConfiguration cc = Utils.getCommunityConfiguration(true); - System.out.println(cc.getCommunities().keySet()); - Community community = cc.getCommunities().get("eutopia"); - community.getProviders().forEach(p -> System.out.println(p.getOpenaireId())); - } - - @Test - void getCommunityProjects() throws Exception { - CommunityEntityMap projectMap = Utils.getCommunityProjects(true); - - Assertions - .assertTrue( - projectMap - .keySet() - .stream() - .allMatch(k -> k.startsWith("40|"))); - - System.out.println(projectMap); - } - - @Test - void getCommunityOrganizations() throws Exception { - CommunityEntityMap organizationMap = Utils.getCommunityOrganization(true); - Assertions.assertTrue(organizationMap.keySet().stream().allMatch(k -> k.startsWith("20|"))); - - } - -} diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java index 5e106ef45..d02faedec 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java @@ -32,8 +32,6 @@ public class BulkTagJobTest { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - public static final String MOCK_IS_LOOK_UP_URL = "BASEURL:8280/is/services/isLookUp"; - public static final String pathMap = "{ \"author\" : \"$['author'][*]['fullname']\"," + " \"title\" : \"$['title'][*]['value']\"," + " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\"," From edcb17ca9827c39c8662b3083d2246dd91bd26ab Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 3 Nov 2023 13:01:14 +0100 Subject: [PATCH 15/60] refactoring and test --- .../PrepareResultCommunitySet.java | 2 +- .../SparkResultToCommunityFromProject.java | 18 +++++++++-- .../dnetlib/dhp/bulktag/BulkTagJobTest.java | 1 - .../ResultToCommunityJobTest.java | 31 +++++++++++++------ 4 files changed, 39 insertions(+), 13 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java index 9a2f25fa6..7bee1ea0c 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java @@ -56,7 +56,7 @@ public class PrepareResultCommunitySet { log.info("production: {}", production); final CommunityEntityMap projectsMap = Utils.getCommunityProjects(production); - //log.info("projectsMap: {}", new Gson().toJson(projectsMap)); + // log.info("projectsMap: {}", new Gson().toJson(projectsMap)); SparkConf conf = new SparkConf(); diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/SparkResultToCommunityFromProject.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/SparkResultToCommunityFromProject.java index 90c30f812..6e298cf94 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/SparkResultToCommunityFromProject.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/SparkResultToCommunityFromProject.java @@ -112,7 +112,7 @@ public class SparkResultToCommunityFromProject implements Serializable { R ret = value._1(); Optional rcl = Optional.ofNullable(value._2()); if (rcl.isPresent()) { - ArrayList communitySet = rcl.get().getCommunityList(); + // ArrayList communitySet = rcl.get().getCommunityList(); List contextList = ret .getContext() .stream() @@ -124,7 +124,7 @@ public class SparkResultToCommunityFromProject implements Serializable { res.setId(ret.getId()); List propagatedContexts = new ArrayList<>(); - for (String cId : communitySet) { + for (String cId : rcl.get().getCommunityList()) { if (!contextList.contains(cId)) { Context newContext = new Context(); newContext.setId(cId); @@ -138,6 +138,20 @@ public class SparkResultToCommunityFromProject implements Serializable { PROPAGATION_RESULT_COMMUNITY_PROJECT_CLASS_NAME, ModelConstants.DNET_PROVENANCE_ACTIONS))); propagatedContexts.add(newContext); + } else { + ret + .getContext() + .stream() + .filter(c -> c.getId().equals(cId)) + .findFirst() + .get() + .getDataInfo() + .add( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RESULT_COMMUNITY_PROJECT_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_PROJECT_CLASS_NAME, + ModelConstants.DNET_PROVENANCE_ACTIONS)); } } res.setContext(propagatedContexts); diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java index d02faedec..efabcc6bc 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java @@ -6,7 +6,6 @@ import static eu.dnetlib.dhp.bulktag.community.TaggingConstants.ZENODO_COMMUNITY import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; - import java.util.List; import org.apache.commons.io.FileUtils; diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/ResultToCommunityJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/ResultToCommunityJobTest.java index b705af8ac..5642501b3 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/ResultToCommunityJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromproject/ResultToCommunityJobTest.java @@ -8,7 +8,6 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.List; -import eu.dnetlib.dhp.schema.oaf.Context; import org.apache.commons.io.FileUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; @@ -27,6 +26,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import eu.dnetlib.dhp.orcidtoresultfromsemrel.OrcidPropagationJobTest; import eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob; +import eu.dnetlib.dhp.schema.oaf.Context; import eu.dnetlib.dhp.schema.oaf.Dataset; public class ResultToCommunityJobTest { @@ -98,23 +98,36 @@ public class ResultToCommunityJobTest { * {"resultId":"50|57a035e5b1ae::803aaad4decab7e27cd4b52a1931b3a1","communityList":["sdsn-gr"]} * {"resultId":"50|57a035e5b1ae::a02e9e4087bca50687731ae5c765b5e1","communityList":["netherlands"]} */ - List context = tmp.filter(r -> r.getId().equals("50|57a035e5b1ae::d5be548ca7ae489d762f893be67af52f")) - .first().getContext(); + List context = tmp + .filter(r -> r.getId().equals("50|57a035e5b1ae::d5be548ca7ae489d762f893be67af52f")) + .first() + .getContext(); Assertions.assertTrue(context.stream().anyMatch(c -> containsResultCommunityProject(c))); - context = tmp.filter(r -> r.getId().equals("50|57a035e5b1ae::a77232ffca9115fcad51c3503dbc7e3e")) - .first().getContext(); + context = tmp + .filter(r -> r.getId().equals("50|57a035e5b1ae::a77232ffca9115fcad51c3503dbc7e3e")) + .first() + .getContext(); Assertions.assertTrue(context.stream().anyMatch(c -> containsResultCommunityProject(c))); - Assertions.assertEquals(0, tmp.filter(r -> r.getId().equals("50|57a035e5b1ae::803aaad4decab7e27cd4b52a1931b3a1")).count()); + Assertions + .assertEquals( + 0, tmp.filter(r -> r.getId().equals("50|57a035e5b1ae::803aaad4decab7e27cd4b52a1931b3a1")).count()); - Assertions.assertEquals(0, tmp.filter(r -> r.getId().equals("50|57a035e5b1ae::a02e9e4087bca50687731ae5c765b5e1")).count()); + Assertions + .assertEquals( + 0, tmp.filter(r -> r.getId().equals("50|57a035e5b1ae::a02e9e4087bca50687731ae5c765b5e1")).count()); - Assertions.assertEquals(2, tmp.filter(r -> r.getContext().stream().anyMatch(c -> c.getId().equals("aurora"))).count()); + Assertions + .assertEquals( + 2, tmp.filter(r -> r.getContext().stream().anyMatch(c -> c.getId().equals("aurora"))).count()); } private static boolean containsResultCommunityProject(Context c) { - return c.getDataInfo().stream().anyMatch(di -> di.getProvenanceaction().getClassid().equals("result:community:project")); + return c + .getDataInfo() + .stream() + .anyMatch(di -> di.getProvenanceaction().getClassid().equals("result:community:project")); } } From 638ad9e74f09a7ea975e49cc7842746096608cbc Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 3 Nov 2023 15:06:50 +0100 Subject: [PATCH 16/60] changing test for new implementation --- ...kResultToCommunityFromOrganizationJob.java | 2 - .../dnetlib/dhp/bulktag/BulkTagJobTest.java | 4 +- .../PrepareAssocTest.java | 95 ------------------ .../ResultToCommunityJobTest.java | 8 +- .../sample/dataset | 10 ++ .../sample/dataset_10.json.gz | Bin 7128 -> 0 bytes 6 files changed, 16 insertions(+), 103 deletions(-) delete mode 100644 dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareAssocTest.java create mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/sample/dataset delete mode 100644 dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/sample/dataset_10.json.gz diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java index fe79f1be1..df8ca3805 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/SparkResultToCommunityFromOrganizationJob.java @@ -60,8 +60,6 @@ public class SparkResultToCommunityFromOrganizationJob { conf, isSparkSessionManaged, spark -> { - // removeOutputDir(spark, outputPath); - execPropagation(spark, inputPath, outputPath, possibleupdatespath); }); diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java index efabcc6bc..64eb86338 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java @@ -40,7 +40,9 @@ public class BulkTagJobTest { "\"fos\" : \"$['subject'][?(@['qualifier']['classid']=='FOS')].value\"," + "\"sdg\" : \"$['subject'][?(@['qualifier']['classid']=='SDG')].value\"," + "\"hostedby\" : \"$['instance'][*]['hostedby']['key']\" , " + - "\"collectedfrom\" : \"$['instance'][*]['collectedfrom']['key']\"} "; + "\"collectedfrom\" : \"$['instance'][*]['collectedfrom']['key']\"," + + "\"publisher\":\"$['publisher'].value\"," + + "\"publicationyear\":\"$['dateofacceptance'].value\"} "; private static SparkSession spark; diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareAssocTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareAssocTest.java deleted file mode 100644 index 6536ecc85..000000000 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareAssocTest.java +++ /dev/null @@ -1,95 +0,0 @@ - -package eu.dnetlib.dhp.resulttocommunityfromorganization; - -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; - -import org.apache.commons.io.FileUtils; -import org.apache.commons.io.IOUtils; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Encoders; -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 org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.gson.Gson; - -import eu.dnetlib.dhp.api.Utils; -import eu.dnetlib.dhp.api.model.CommunityEntityMap; -import eu.dnetlib.dhp.bulktag.BulkTagJobTest; -import eu.dnetlib.dhp.bulktag.SparkBulkTagJob; -import eu.dnetlib.dhp.schema.oaf.Dataset; - -/** - * @author miriam.baglioni - * @Date 13/10/23 - */ -public class PrepareAssocTest { - - private static SparkSession spark; - - private static Path workingDir; - - private static final Logger log = LoggerFactory.getLogger(PrepareAssocTest.class); - - @BeforeAll - public static void beforeAll() throws IOException { - workingDir = Files.createTempDirectory(BulkTagJobTest.class.getSimpleName()); - log.info("using work dir {}", workingDir); - - SparkConf conf = new SparkConf(); - conf.setAppName(BulkTagJobTest.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(PrepareAssocTest.class.getSimpleName()) - .config(conf) - .getOrCreate(); - } - - @AfterAll - public static void afterAll() throws IOException { - FileUtils.deleteDirectory(workingDir.toFile()); - spark.stop(); - } - - @Test - void test1() throws Exception { - - PrepareResultCommunitySet - .main( - new String[] { - - "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", - getClass().getResource("/eu/dnetlib/dhp/resulttocommunityfromorganization/relation/").getPath(), - "-outputPath", workingDir.toString() + "/prepared", - "-production", Boolean.TRUE.toString(), - "-hive_metastore_uris", "" - }); - - final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - - JavaRDD tmp = sc - .textFile(workingDir.toString() + "/prepared") - .map(item -> new ObjectMapper().readValue(item, ResultCommunityList.class)); - - tmp.foreach(r -> System.out.println(new ObjectMapper().writeValueAsString(r))); - } - -} diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java index da3be8b35..6440ded92 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/resulttocommunityfromorganization/ResultToCommunityJobTest.java @@ -72,15 +72,13 @@ public class ResultToCommunityJobTest { SparkResultToCommunityFromOrganizationJob .main( new String[] { - "-isTest", Boolean.TRUE.toString(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", getClass() - .getResource("/eu/dnetlib/dhp/resulttocommunityfromorganization/sample") + .getResource("/eu/dnetlib/dhp/resulttocommunityfromorganization/sample/") .getPath(), - "-hive_metastore_uris", "", - "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Dataset", - "-outputPath", workingDir.toString() + "/dataset", + "-outputPath", workingDir.toString() + "/", "-preparedInfoPath", preparedInfoPath }); diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/sample/dataset b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/sample/dataset new file mode 100644 index 000000000..61a1da1ac --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/sample/dataset @@ -0,0 +1,10 @@ +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055868909,"id":"50|dedup_wf_001::afaf128022d29872c4dad402b2db04fe","originalId":["od______3989::02dd5d2c222191b0b9bd4f33c8e96529"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[{"value":"10.4185/RLCS-2018-1243","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofcollection":"2020-03-03T13:05:26.091Z","dateoftransformation":"2020-03-03T13:06:53.161Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:26.091Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/562","datestamp":"2018-01-23T15:06:07Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"affiliation":[],"fullname":"Jung, Nicole","name":"Nicole","pid":[{"dataInfo":null,"qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":null,"schemename":null},"value":"0000-0001-9513-2468"}],"rank":1,"surname":"Jung"},{"affiliation":[],"fullname":"Gräßle, Simone","name":"Simone","pid":[],"rank":2,"surname":"Gräßle"}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Comunicación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"yihadismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"viralidad","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"vídeo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"propaganda","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"terrorismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"terrorism","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"jihadism","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"virality","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"video","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"5903 Ideologías Políticas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"6310.13 Terrorismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"El individualismo como estrategia","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Individualism as an improvised strategy","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Se analiza como el Yihadismo distribuye su mensaje propaganístico en You Tube.\nUniversidad Camilo José Cela","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"MediaLab - UCJC","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[{"dataInfo":[{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"propagation:community:organizationtoresult","classname":"Propagation of community result through organization to result association","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"}],"id":"beopen"}],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/562"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-01-01","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055868954,"id":"50|dedup_wf_001::3f62cfc27024d564ea86760c494ba93b","originalId":["od______3989::05d8c751462f9bb8d2b06956dfbc5c7b"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:26.079Z","dateoftransformation":"2020-03-03T13:06:53.239Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:26.079Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/227","datestamp":"2017-11-15T16:29:55Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"affiliation":[],"fullname":"Fischer, Malte","name":"Malte","pid":[{"dataInfo":null,"qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":null,"schemename":null},"value":"0000-0002-2806-1302"}],"rank":1,"surname":"Fischer"},{"affiliation":[],"fullname":"Barbul, Daniel","name":"Daniel","pid":[],"rank":2,"surname":"Barbul"},{"affiliation":[],"fullname":"Schmidtmann, Marc","name":"Marc","pid":[{"dataInfo":null,"qualifier":{"classid":"MAG Identifier","classname":"MAG Identifier","schemeid":null,"schemename":null},"value":"2024269002"}],"rank":3,"surname":"Schmidtmann"},{"affiliation":[],"fullname":"Beckhaus, Ruediger","name":"Ruediger","pid":[],"rank":4,"surname":"Beckhaus"}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Actividad Física y del Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Eventos deportivos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Retos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"I Congreso Nuevos Retos en los Eventos Deportivos","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"I Congreso Nuevos Retos en los Eventos Deportivos, celebrado en la Universidad Camilo José Cela.\n\n\n\nDerivado del proyecto de investigación \"La situación de los eventos deportivos en España a través de su estructuras y de sus herramientas\", tiene como finalidad conocer la situación de los eventos deportivos en España. Se realizó un estudio descriptivo holístico en el que se investigaron elementos culturales, el perfil de los recursos humanos, la estructura de los comités, entre otras.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2015-06-20","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/227"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2015-06-20","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055869097,"id":"50|od________18::8887b1df8b563c4ea851eb9c882c9d7b","originalId":["od______3989::0f89464c4ac4c398fe0c71433b175a62"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:26.069Z","dateoftransformation":"2020-03-03T13:06:53.54Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:26.069Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/802","datestamp":"2018-11-14T15:30:47Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Ciencias de la Actividad Física y del Deporte","name":"Ciencias La Actividad Física Y. Del Deporte","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Actividad Física y del Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Actividades deportivas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Impacto físico","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Proyectos de investigación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"2411.06 Fisiología del Ejercicio","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Investigaciones en Ciencias de la Actividad Física y del Deporte CCAFD - UCJC","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Blanca Romero, investigadora y docente de Ciencias de la Actividad Física y del Deporte en la Universidad Camilo José Cela, describe dos de las líneas de investigación que se han desarrollado en el departamento CCAFD de UCJC: el impacto físico de las diferentes actividades deportivas y los métodos de recuperación más eficaces tras el ejercicio físico.\nBlanca también destaca la importancia en el ámbito de la salud los estudios en Ciencias de la Actividad Física y del Deporte.\nCiencias de la Actividad Física y del Deporte en la Universidad Camilo José Cela","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2016-03-21","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Universidad Camilo José Cela, Ciencias de la Actividad Física y del Deporte","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[{"dataInfo":[{"deletedbyinference":false,"inferenceprovenance":"","inferred":true,"invisible":false,"provenanceaction":{"classid":"propagation:community:organizationtoresult","classname":"Propagation of community result through organization to result association","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"}],"id":"beopen"}],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/802"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2016-03-21","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055869215,"id":"50|doajarticles::8d817039a63710fcf97e30f14662c6c8","originalId":["od______3989::1729c3988199b95d1d566851af7d3c55"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:25.653Z","dateoftransformation":"2020-03-03T13:06:53.764Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:25.653Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/801","datestamp":"2018-11-14T15:19:38Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Ciencias de la Actividad Física y del Deporte","name":"Ciencias La Actividad Física Y. Del Deporte","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Actividad Física y del Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Prevención","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Hidratación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deportistas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Electrolitos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deportes de resistencia","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"2411.06 Fisiología del Ejercicio","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"La importancia de la hidratación en competiciones resistencia, por Juan del Coso","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Juan del Coso, investigador y profesor en el Grado en Ciencias de la Actividad Física y del Deporte UCJC, presenta el proyecto de investigación que ha realizado el departamento CCAFD de UCJC: la prevención de desequilibrios de agua y electrolitos que se dan en los deportes de resistencia para que los deportistas realicen su actividad de una manera más segura.\nMás información: http://www.ucjc.edu/2016/03/investiga...\nUniversidad Camilo José Cela, Ciencias de la Actividad Física y del Deporte","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2016-03-21","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Grado en Ciencias de la Actividad Física y del Deporte UCJC","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[{"dataInfo":[{"deletedbyinference":false,"inferenceprovenance":"","inferred":true,"invisible":false,"provenanceaction":{"classid":"propagation:community:organizationtoresult","classname":"Propagation of community result through organization to result association","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"}],"id":"euromarine"}],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/801"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2016-03-21","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055869487,"id":"50|doajarticles::3c98f0632f1875b4979e552ba3aa01e6","originalId":["od______3989::2e3f34ce90520fae350a7e1148d7dcea"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.912Z","dateoftransformation":"2020-03-03T13:06:54.218Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.912Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/800","datestamp":"2018-11-14T13:43:21Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Actividad Física y del Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Deporte","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Cafeína","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Efectos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Alto rendimiento","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Competiciones","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Beneficios","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"2411.06 Fisiología del Ejercicio","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Investigación - Efectos positivos y negativos del uso de la cafeína en el deporte","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Proyecto de investigación que se están desarrollando en la Universidad Camilo José Cela (UCJC) sobre la cafeína y sus efectos cuando se realiza deporte. https://www.ucjc.edu/2016/08/la-bbc-s...\n\nJuan Del Coso, director del Laboratorio de Fisiología del Ejercicio de la Universidad Camilo José Cela, nos describe cómo está siendo la investigación que desarrolla con deportistas de alto rendimiento y el consumo de cafeína.\n\n\"Nuestra investigación se enmarca dentro del campo de la fisiología del ejercicio y el objetivo que tenemos es investigar los efectos que tiene la cafeína en los deportistas, principalmente saber si es una ayuda ergogénica en la mayor parte de los deportes, pero también conocer qué perjuicios existen para aquellos deportistas que la utilizan para incrementar el rendimiento deportivo\", explica Del Coso.\n\nY es que los datos son muy llamativos: tres de cada cuatro deportistas de alto rendimiento utilizan la cafeína antes de la competición y muchas veces se utiliza sin saber los perjuicios que puede tener para el deportista. \"Lo que estamos investigando es cómo una sustancia, que te puede hacer rendir mejor, cómo puede afectar negativamente cuando termina una competición\".\n\nPero esta investigación no sólo puede extraer conclusiones interesantes y útiles para los deportistas profesionales, \"investigando el deporte de alto rendimiento podemos conocer los beneficios y perjuicios de la cafeína que también afectarían al deportista amateur y hacer un deporte en general más seguro\".\n\nPara el desarrollo de este proyecto de investigación están colaborando el Laboratorio de Fisiología del Ejercicio de la Universidad Camilo José Cela y la Agencia para la Protección de la Salud en el Deporte.\nPara el desarrollo de este proyecto de investigación están colaborando el Laboratorio de Fisiología del Ejercicio de la Universidad Camilo José Cela y la Agencia para la Protección de la Salud en el Deporte.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2017-10-06","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Universidad Camilo José Cela, Vicerrectorado de Innovación","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[{"dataInfo":[{"deletedbyinference":false,"inferenceprovenance":"","inferred":true,"invisible":false,"provenanceaction":{"classid":"propagation:community:organizationtoresult","classname":"Propagation of community result through organization to result association","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"}],"id":"euromarine"},{"dataInfo":[{"deletedbyinference":false,"inferenceprovenance":"","inferred":true,"invisible":false,"provenanceaction":{"classid":"propagation:community:organizationtoresult","classname":"Propagation of community result through organization to result association","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.85"}],"id":"ni"}],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/800"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2017-10-06","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055869498,"id":"50|od______3989::2f4f3c820c450bd08dac08d07cc82dcf","originalId":["od______3989::2f4f3c820c450bd08dac08d07cc82dcf"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.806Z","dateoftransformation":"2020-03-03T13:06:54.229Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.806Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/797","datestamp":"2018-11-14T12:12:23Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Cuesta Cano, Laura","name":"Laura","surname":"Cuesta Cano","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Comunicación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Marketing digital","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Medios sociales","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Marcas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Empresas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Emprendedores","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"II Liga de Debate CICAE - UCJC","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"6114.13 Marketing","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Laura Cuesta - Liga de Debate CICAE - UCJC","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Laura Cuesta, profesora en la Universidad Camilo José Cela (UCJC) y Especialista en Marketing Digital y Medios Sociales para marcas y emprendedores. \n\nEn esta ocasión, Laura Cuesta, nos explica el significado de redes sociales y nos habla sobre el uso que le dan los usuarios.\n\nII Liga de Debate CICAE - UCJC http://ligacicae.ucjc.edu/\n\nLa Universidad Camilo José Cela (UCJC) y la Asociación de Colegios Privados e Independientes –Círculo de Calidad Educativa (CICAE) – organizan la segunda edición de esta gran liga de debate académico. El formato es al estilo inglés, similar al de la Oxford Union, donde algunos jóvenes se reunían a debatir sobre temas de la más candente actualidad y polemizaban con la intención de formarse en tan valoradas habilidades.\n\nLa Liga de Debate CICAE-UCJC es una ocasión única para los alumnos de ponerse en contacto con herramientas tan útiles como la oratoria y la comunicación efectiva. Pero, sobre todo, una oportunidad de fomentar el pensamiento crítico y la socialización entre alumnos con espíritu emprendedor. La Liga de Debate CICAE - Universidad Camilo José Cela se diferencia del resto de torneos en que potenciamos que sea eminentemente formativa. Alumnos y profesores reciben formación en comunicación, oratoria y debate por parte de alguno de los expertos en debate españoles más reconocidos en la escena internacional.\n\nDatos de la III Liga de Debate CICAE - UCJC http://ligacicae.ucjc.edu/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-10-23","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Universidad Camilo José Cela, Vicerrectorado de Innovación","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/797"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-10-23","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055870582,"id":"50|od______3989::752fd0b2bbac1ea1cc50e52fd46eb663","originalId":["od______3989::752fd0b2bbac1ea1cc50e52fd46eb663"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.561Z","dateoftransformation":"2020-03-03T13:06:55.716Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.561Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/798","datestamp":"2018-11-14T12:57:26Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Psicología","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Alcohol","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Dependencia","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Proyectos de investigación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Alcoholismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Realidad virtual","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Psicología conductual","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Investigación - Intervención basada en realidad virtual sobre la dependencia del alcohol","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Proyecto de investigación que se están desarrollando en la Universidad Camilo José Cela (UCJC) con tecnología aplicada al abandono de la dependencia del alcohol. \nRosa Jurado, profesora e investigadora de UCJC https://www.ucjc.edu/, explica cómo se conjuga el alcoholismo con la realidad virtual. \"Mi investigación consiste en desarrollar una técnica de intervención o rehabilitación que tiene como propósito fomentar la capacidad de inhibición para que las personas que tienen dependencia del alcohol sean capaces de controlar su conducta de aproximación hacia aquello que tiene que ver con el consumo de alcohol\".\n\nEn esta investigación, liderada por la Universidad Camilo José Cela, están participando, el Instituto de Investigación Biomédica del Hospital 12 de Octubre y la Universidad Politécnica de Madrid, con el CeDint. \"En nuestro equipo de investigación pensamos que, además del deseo de consumo, existen una carencia clara de capacidad de control inhibitorio en este tipo de personas\", destaca Rosa Jurado.\n\nLa realidad virtual sirve, según explica Gabriel Rubio, jefe de Psiquiatría del Hospital 12 de Octubre, \"para dar un paso más, para ver cómo reacciona un sujeto en un ambiente que no es el real, pero se parece mucho al real\". El objetivo final de estas acciones es \"mejorar la capacidad del individuo para poner en marcha mecanismos de inhibición conductual, para que el sujeto sea capaz de decir que no\".\n\nY no acaba ahí el proceso. La fase final del tratamiento se enfoca en que el paciente \"sea capaz de reconstruir su vida\" a través de programas de entrenamiento de habilidades sociales, asertividad, familias... \"Juntando todas las áreas de intervención, la abstinencia se mantiene mucho más tiempo y las recaídas disminuyen\".","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2017-10-05","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[{"value":"Universidad Camilo José Cela, Vicerrectorado de Innovación","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/798"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2017-10-05","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055870750,"id":"50|od______3989::7fcbe3a03280663cddebfd3cb9203177","originalId":["od______3989::7fcbe3a03280663cddebfd3cb9203177"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:25.652Z","dateoftransformation":"2020-03-03T13:06:55.95Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:25.652Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/796","datestamp":"2018-11-05T13:36:54Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Comunicación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Redes sociales","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Comunicación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Información","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Desinformación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"III Liga de Debate CICAE","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Instantaneidad","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Información periodística","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Fake news","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"5910.02 Medios de Comunicación de Masas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Jorge Gallardo - Liga de Debate CICAE","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Jorge Gallardo, profesor de Derecho en la Universidad Camilo José Cela (UCJC), Doctor en Comunicación Audiovisual y MBA en Empresas Audiovisuales. Subdirector de Espejo Público en Antena 3 Noticias. \n\nEn esta ocasión, Jorge Gallardo habla sobre las redes sociales, cómo han transformado la manera en la que nos comunicamos y nos informamos. \n\nIII Liga de Debate CICAE - UCJC http://ligacicae.ucjc.edu/\n\nLa Universidad Camilo José Cela (UCJC) y la Asociación de Colegios Privados e Independientes –Círculo de Calidad Educativa (CICAE) – organizan la tercera edición de esta gran liga de debate académico. El formato es al estilo inglés, similar al de la Oxford Union, donde algunos jóvenes se reunían a debatir sobre temas de la más candente actualidad y polemizaban con la intención de formarse en tan valoradas habilidades.\n\nLa Liga de Debate CICAE-UCJC es una ocasión única para los alumnos de ponerse en contacto con herramientas tan útiles como la oratoria y la comunicación efectiva. Pero, sobre todo, una oportunidad de fomentar el pensamiento crítico y la socialización entre alumnos con espíritu emprendedor. La Liga de Debate CICAE - Universidad Camilo José Cela se diferencia del resto de torneos en que potenciamos que sea eminentemente formativa. Alumnos y profesores reciben formación en comunicación, oratoria y debate por parte de alguno de los expertos en debate españoles más reconocidos en la escena internacional.\n\nDatos de la III Liga de Debate CICAE - UCJC http://ligacicae.ucjc.edu/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-10-23","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/796"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-10-23","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055872001,"id":"50|od______3989::d791339867bec6d3eb2104deeb4e4961","originalId":["od______3989::d791339867bec6d3eb2104deeb4e4961"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.586Z","dateoftransformation":"2020-03-03T13:06:57.721Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.586Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/645","datestamp":"2018-04-12T07:59:27Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Transferencia de Conocimiento e Innovación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Innovación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Investigación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Ciencia","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Tecnología","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"UCJC Open Science Day 2018","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"El UCJC Open Science Day tiene como objetivo mostrar las actividades de investigación llevadas a cabo por investigadores de la Universidad Camilo José Cela (UCJC) a todos los miembros de la UCJC.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-04-10","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/645"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-04-10","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} +{"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1585055872022,"id":"50|od______3989::d90d3a1f64ad264b5ebed8a35b280343","originalId":["od______3989::d90d3a1f64ad264b5ebed8a35b280343"],"collectedfrom":[{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null}],"pid":[],"dateofcollection":"2020-03-03T13:05:34.912Z","dateoftransformation":"2020-03-03T13:06:57.747Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2020-03-03T13:05:34.912Z","altered":true,"baseURL":"http%3A%2F%2Frepositorio.ucjc.edu%2Foai%2Frequest","identifier":"oai:repositorio.ucjc.edu:20.500.12020/795","datestamp":"2018-11-06T15:43:10Z","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"author":[{"fullname":"Universidad Camilo José Cela, Vicerrectorado de Innovación","name":"Vicerrectorado Innovación","surname":"Universidad Camilo José Cela","rank":1,"pid":null,"affiliation":null}],"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"language":{"classid":"esl/spa","classname":"Spanish","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Ciencias de la Comunicación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Unión Europea","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Desinformación","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Política","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Periodismo","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Ciudadanos europeos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Democracias","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Estado de derecho","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Derechos humanos","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"5910.02 Medios de Comunicación de Masas","qualifier":{"classid":"","classname":"","schemeid":"","schemename":""},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Ramón Luis Varcárcel - La Unión Europea ante el reto de la desinformación","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Conferencia de Ramón Luis Valcárcel, vicepresidente del Parlamento Europeo, en el Campus de Almagro de la Universidad Camilo José Cela (UCJC) bajo el título \"La Unión Europea ante el reto de la desinformación\", en la que ha alertado sobre el alcance de la desinformación en las democracias y sobre cómo pone en peligro los derechos de los ciudadanos europeos. También ha participado el rector de la UCJC, Samuel Martín-Barbero.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2018-10-11","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":null,"embargoenddate":null,"source":[],"fulltext":[],"format":[],"contributor":[],"resourcetype":null,"coverage":[],"bestaccessright":null,"context":[],"externalReference":[],"instance":[{"license":{"value":"http://creativecommons.org/licenses/by-nc-nd/4.0/","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"instancetype":{"classid":"0024","classname":"Film","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"hostedby":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"url":["http://hdl.handle.net/20.500.12020/795"],"distributionlocation":"","collectedfrom":{"key":"opendoar____::3989","value":"Depósito Digital e-UCJC","dataInfo":null},"dateofacceptance":{"value":"2018-10-11","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"sysimport:crosswalk:repository","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null}],"storagedate":null,"device":null,"size":null,"version":null,"lastmetadataupdate":null,"metadataversionnumber":null,"geolocation":[]} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/sample/dataset_10.json.gz b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/sample/dataset_10.json.gz deleted file mode 100644 index dccc28c879579c27eddfa3fd64c2e55a862b82b9..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 7128 zcmV;}8zpT17X4im?zDv9FZ`3?^$o{xt=^b%UqvwSXlvmbgQzYv9pQ{J1+XnkNMcuw`5J-E<(r>qn*$BJ1>i$Xvx zy<%O6Y$a7WV@ejqcQpNGrua%0qLgY4#r3T@&G=nzF4&T1Tt7I;c{yw9ex&Ozw)fa4 zUUS#mxBVW}TUIMj7cWJ|i;`xm-pr3Ckrzdi(PZjPq0v}V^Jg&MG zLDLEwd&*avpHNri6lyPNO8C*APoA8>Esa&=Rhqtq8dlggcy3+T_Xb0M5*@#Idh+rp z@chY<9}Gh%`fWv1F&A9z0-H#&ho1T65=0_-Q?6J18fdmx-}as9-pzCoJv3W6w^pJh z?*(4q9eE-AdF6*QFPa6Tfj9O4tykY&Dw-E_sWN(XiP0I2d@xAjX@7 z9cw>Bt8yuo?rU^D7pbu7N4?IWGg`QxRQaM$z7&8dIuQHo_Iur3cRTX6z4NOVCugU7 zCR+d2SfY7E+v>D;XgrDs{~W=8{?Rn@!=qq0n)J~8z>>ZJX9;C1)si1N@y(6=LT&zc z^FJx?lb0fsd8?t@@1~*KI)Zi`Unu&^e_?qe1TU(zMDMZZnLtGaFZaFE_SgNMXr6;^ zNTzZjuJKft@6uyPzNqM8PXJz|$6zgc0=!(&TolVI1gM{Kb;Zrqw>U0Tc?teRYpAH= zbLjp%Zu~@mqZ70s2`4EfCo-#Y!6*}(pYrRS`JSk|GVV*|T=m^ub-zJoEtWJ9MJ62# zhlJrmD9}Aj*c=Q8g<^)Mvwkkq&HEBwstX6qA%Pi%zBlkg^2$x0 z4!RayiV{`RT7-I>5|JkYoEM@2zz`^A_>IuRv* zvwK5Z`n_8m#J{G?owZRV;0tL^Kn^G+3Z4*)?+tSX2XE)#)hp>Kzo2=U=*SH&Ehp{B zPOE3X-^8TxZtY51vzp;Gm%R2L>8NO=Yc%?Q9B#OeL#HJH1Qfum=~_O(Y(EBB3OjyT})82@?oT zGbD1!Y zWPPRn)fULBEN-Sg3tRzXTVBuGfBC`T|4>}hXm}mhG@OqDHfO;YoL~}-hMZ1Dz6$Z z4|gON>c`gmASo>p`k8S=z2_^9Ac(Hk>=SM}Q$qE%KD+M#AwfU_e@S>nVJa`T*UibQ7 zoIK|y5<16MaNPhVbIG{EcHL8m0eQ)Z;uv4u{6wl8g{TsNZM3StkYtJUqirY^Gy!jv zq-ZHYN&FdM#)=rsO`#Df2eqTxxPp0jEsesZrUE`JE10MwG9Yx*^7=MtpeR8#@C2A+ zaw0)h#lS59K45eBHOw`iOoyW(8&Wo8 z@Y9@oZ0rxiF!sk0HSzQghPg)f4Rb}q4;keebt1|&@vcnHfl;m@Cg%vf+3-Uo=V-+v zZhRf>lY6hE+sAIpJ>4lJe)+pD_4NRXw2L*-jjILXzx=(X$!-H%t0sK26LG0+)P+V| zXyl_CS=#HsTX1-@h~jo)9e7(v2JRQe$hqz=ZX4D?!FX;LMvKcUqYjuu0uu~E?aT=@p*84iYUXMxqL; z6ug|;_#4M?(26Q}+rp5MHC}iGTJH=mi{0_Pt}PK7e0WhK^vYQoGz-H=+2&m-t&|vX zT5)Bi$~Hej_nKpZIw*)yC9ICu4zaFq@H)4;1{q_qd8ZpuQuqP%WQ>ENf^Bm)Kf+z5 zm6H?zS7L&1CY<&wdNfGz@EE$K#Rm+ajc_?5aJv#Z5}DY@XzYT<4;eHL{HV1zQqpsHQ57uZ#cVc7CjQt9r*ssK zeQ(a@(=iXdxj!6@0ybikTY|>^IGAFvc;Zi|@ia>OBpQt-kx%F2BxF(4#h`IC7)9Yd zgT~|0hl?6-`!Om#6^u=)YwIhe=kGJZcq$~_bv!_a3KCwBnq_VkT^IJ z&M$Qa1bCZu6qW<;xW-wxlBsYTeLK-5xQ!MT&^Pon%Yov!D-NDpxhqe1tM?LQXb&NrTV=p}3s%9{_ElsuyqC4KnN2Jr=h`{(t(h;hO@2Do9SjTcylixA^w^B`plU9C$U8+gQrjxlh3WK>n z8AtJOI-c?<3St^k>iK-syP0Yd@Ngau8J~Jl;LRxyBae=`?++)*H~}qokZKYR2U9<| z7u94q2>i*1qM9`F9G+@olfqm{H3?_Ka2EInrJDQ&JP>qwhTci1%OhubE-$uGSNoK` z+v=5w88z=3ekY$&N-NXiqTv(maS z>|q^KJFe=WI3yIG@f`P+3+JF5am2xM2ja-tZYchU*zuic8FZrZHOV=~IWBZ8JSA08 zr&wTiM%JX0xOtZBs3To8in~)sp5M&jC^)9t*!p;`Dm1I&b*?!ur&|t8BFu7#Ht(l2mWt|? zAZ+Kg9?bTETh?T9-Dzyf7Pk2b4^KxOkwW15o=#+PS(A$3?^|WeOSY{6(9wd&MQ4>ApCpH0Wt(23KFOL3?R)4 zqNFMXnxDNB#8ueaN@y}Drj5<^#0NY$f`7ll$k*2NG(cZtyS8K_imECA^sHJFFo4W> z-We&C;VCf+jfZ|<0K3^RwoQ5PKZ@_>EV@}T?NqJT2C z^o3%tDxMhpmC?p_C_upmcrJ%A5xsCKTlS%yLt(oGA`4yJFc|yRrtHI)6v5IhQzYZ( zQuS>qbDg4zf$r@u>JGz=7NzklT5XEr~jbsc(j8>9f! zVQEzu5~av-QMxcHU6akvpfLEP)v#|lhFS;Z%&8o&d%Jqrj-OW|xsr+QUfWYf;B&m4 z>>_cm0cCG+)|#vh-)ahaG<*_^A^B`%dk@mD+RR_?F#tV2*Fu5x%9@X}+bK=Q+Nm z<6Ax`-*P&+d>)h~OwNb%kWB)Q4I?j3yh%bC{N;@q{FtzL?-su0ZN)pux14yR`|>S= z=?COnT6qr5w;WH$z3cgw!3_Qc;fLj0o>Y2W@Py{FPoC0BA@|5$tG_mU$=2OHM~-|@ za^#7`IX-jwgAO0re@4|e;Mem7NsKFWa2ym4u1_}5@3g67A051JFqhHk!2Hw3|^R=5cNaOA_wDH)6Xuv6VH~Og2L`<0&b`A{Thtnu&LVCGh5r z0qMPP`;x|ZW}2p7T8^)#q?vSC$Tq5qieh#xP5QZcP^mLqrJ5MSC~pXvSorCkwTDy} zE$Eioaw1c{Ky5u&;)3Goau~o*IIi9ab12>a{pWw5Y~CwY8P3iLC?BhNoK#Gc!^j`C ztbf8Aw^2fkDe{_fq!uP(TcK9;Lg5Xz!HH=w1$}PbWq3{t*e1X^ z-MlOM00Fd-Dnq%#s=oRj3M8lp3GgK{*Q76~9$Iw1`3X$dFz^(wazoZK^$KM{DtWeD z9?_&MM)O3|-k?WTw#{p@0)1uBzL;v#qv4yDt<-5+Zl$@Pq=cKm`jHtywMmH79PT%; zuw8|@UI?wJnYBfd&Ci;cxh29upUMmu7DURqtq_R-6sc4w+7g@;)pR$g97Uvibqd8&@oh^`WL~Vm(nMLcd+C5&qbckiH`+ZWr!$8unk0_MBrOcl3>uzubA+Mh zTuPIvF59NH1FOwMjs{@Jk9G|iHS)+x++_FD=^vSh;b~1`I-4T0QEI=faclLfHpA8! zY|XU=kHb}IWVCnVdhF&O5<%293Y8*%Q6%OrXfpU-@Y04ynd_5RX{xR5l$Kj6C{o)ewS!I4CT$Sfx3vTV*_=94TM48F0+B$5a`%Ijt%s1Y@o3h zO@iy#K;tNwCte)HF=aldK4X!`@wefK$Dt2_hJJ@gRwvQfNUTu z&*9lXlip2iplCb`MlSoBBN*M0VD!8Y%x=PSF_*)pGGcNm9gFCoAe`#;b9dmyy_v8@ZCFwW)BIH<)Pf ziOlO%DA!mYvS41wf|4f{zC`bkb9N52qxGS2S7!%n5;B=&LeMx6?z~!P+6--m{F*wk z-%l{P(KE5HC(Oi!X{1abyw;M^JcM%dj%j9uR&TpE;tq8~xhPxuHVLyd0|Ix<+jXZc zeL?5>S;765*C6txh%I@gImC|YoK1zyX)6m?ew}uaOpu$$+Kf{ec9htHqN=^TZU6<5 zz85upU`g>oHfdH%c>Hu>5-08DDYF&YL_#&IX;+(l0vhMYNYUJb+cU0N^IYju9K;Ik zsgL!Z&G!(<<=o6Z7&6r3`4k9is5U9HTHcgpMX+hMOmWbmz1>nSIz#pJW2>K-#*f&Hb3XvzVro+ zl>o6{RIvbA&-q-FMPRA`O{b-*^B!HTWlW5vEjQ6U25>B-CXe)uL2FCWy%oe^nlOU5 z71cQ}b<5xjCcDxk7&N_HbFws-hNd+JW-2on1@VZARKJQ}l&JetmTqtVkX__>Z(``1qHmz(+LIis+ zxUvTz^g+}NmoYJ0BUJyS0#~OKxk;p@p*P$72yhjbd}trRpfPl++;jc0&ef z-KpRQOmeu^&w>Wg&3k-u0;q_*T60Vd=R$lBo-8YJI(5vU-zA!kIrQmb>zG3wVGd0? z%^Y$=pkod>=FmejhsKe2EpuqjVjfa23?|-a6tX1Y@jMAxJPo|iA7f&f8<|7574IH% zD2N85=zixmMuTbep@>6`IEN<=+1zcHh(lh4!iJbp$q|PfamZyR`UnS!-N-H2oR1WK`a6JqDaGbw7EV?pqVAxd1Wb?i# zVN`dx91@q$={KC@{5v=E8l zCnarP#bjPDtGIRcs7@AkF3I!F&lqUa@;t(1O_YS>ONn$gdTj= zP%UXrnw4U69BD@Jq(7?*Loh0~d4V!nCk(T1GVp_u=-21y>2yl4k}Iaw;ZlORlwd9; zm`e%vI~!nKK9rA%*yng=pR(aNo>?b&W}{B?%$!GdJTu2LdqAF9;CcRaJhNmx^+WhL z8poWCl90!N?+p{q;~^hTM}F^Sp4qL%yT>!bj8K#NomV#=j068eW~iEU;=H=i(2{$1 ziDusL$PZq5<5@JF1!G4vb40T{63t%ep4GhhlN0@(Y*!`rv2!7Y1I9@m5FG)+u@#Z6q?) zLz)DmVI1+8CleY*aWL`1;cX{-+*-Voag(Wkzy0~+!FV|SfN>Km&!OWcxgyhRgqBKzNq&;md(*HcYJ|y z?1?U#!`;~vCo*rdV0W!1wN}$77$(3?_QLFV#+|aysY+?4>9WR<@Z=P8G7HUCm39%2 z(u^*Yybn<#rsonLQo=hMZYY1<`!xoOXRS1Q?3Z}@5m$P2H~W;$c|54NtIM0h*oN)g zjI3>0v$-EPRpVVNo(gPYs-2=8vFrw2c6q8jAg}QJl+8Ol(Wf~^Cy}N-b*=(S3+a=W zG^^msGq`c{K0o@5s+cS3=py&(4tLqq<@-*jjxN%QdN{g>ql Date: Fri, 3 Nov 2023 15:30:01 +0100 Subject: [PATCH 17/60] refactoring --- .../src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java index 64eb86338..ba0c6b252 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java @@ -41,8 +41,8 @@ public class BulkTagJobTest { "\"sdg\" : \"$['subject'][?(@['qualifier']['classid']=='SDG')].value\"," + "\"hostedby\" : \"$['instance'][*]['hostedby']['key']\" , " + "\"collectedfrom\" : \"$['instance'][*]['collectedfrom']['key']\"," + - "\"publisher\":\"$['publisher'].value\"," + - "\"publicationyear\":\"$['dateofacceptance'].value\"} "; + "\"publisher\":\"$['publisher'].value\"," + + "\"publicationyear\":\"$['dateofacceptance'].value\"} "; private static SparkSession spark; From 5bc97615d50383c9adc3eab2c69f86bacbe4fb4b Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Fri, 3 Nov 2023 15:35:10 +0100 Subject: [PATCH 18/60] - --- .../dhp/actionmanager/bipfinder/SparkAtomicActionScoreJob.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipfinder/SparkAtomicActionScoreJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipfinder/SparkAtomicActionScoreJob.java index 8adc88920..040c89782 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipfinder/SparkAtomicActionScoreJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipfinder/SparkAtomicActionScoreJob.java @@ -95,7 +95,7 @@ public class SparkAtomicActionScoreJob implements Serializable { return projectScores.map((MapFunction) bipProjectScores -> { Project project = new Project(); - // project.setId(bipProjectScores.getProjectId()); + project.setId(bipProjectScores.getProjectId()); project.setMeasures(bipProjectScores.toMeasures()); return project; }, Encoders.bean(Project.class)) From d524e3086630f20e129ae339072e0198b16c7392 Mon Sep 17 00:00:00 2001 From: dimitrispie Date: Tue, 14 Nov 2023 09:46:52 +0200 Subject: [PATCH 19/60] Changes to actionsets Resolve comments from https://code-repo.d4science.org/D-Net/dnet-hadoop/pulls/355 --- dhp-workflows/dhp-stats-actionsets/pom.xml | 39 ---- .../dnetlib/dhp/actionmanager/Constants.java | 102 --------- .../StatsAtomicActionsJob.java | 206 ++++-------------- .../stats_actionsets/StatsDiamondOAModel.java | 29 --- .../stats_actionsets/StatsGreenOAModel.java | 29 --- .../stats_actionsets/StatsOAColourModel.java | 47 ---- .../StatsPubliclyFundedModel.java | 29 --- .../StatsResultEnhancementModel.java | 76 +++++++ .../stats_actionsets/oozie_app/workflow.xml | 28 --- 9 files changed, 116 insertions(+), 469 deletions(-) delete mode 100644 dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java delete mode 100644 dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsDiamondOAModel.java delete mode 100644 dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsGreenOAModel.java delete mode 100644 dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsOAColourModel.java delete mode 100644 dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsPubliclyFundedModel.java create mode 100644 dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsResultEnhancementModel.java diff --git a/dhp-workflows/dhp-stats-actionsets/pom.xml b/dhp-workflows/dhp-stats-actionsets/pom.xml index 670fd4a5e..3daa8f995 100644 --- a/dhp-workflows/dhp-stats-actionsets/pom.xml +++ b/dhp-workflows/dhp-stats-actionsets/pom.xml @@ -8,46 +8,7 @@ dhp-stats-actionsets - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - org.apache.httpcomponents httpclient diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java deleted file mode 100644 index 0df34364d..000000000 --- a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/Constants.java +++ /dev/null @@ -1,102 +0,0 @@ - -package eu.dnetlib.dhp.actionmanager; - -import java.util.Optional; - -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.SparkSession; - -import com.fasterxml.jackson.databind.ObjectMapper; - -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.common.ModelConstants; -import eu.dnetlib.dhp.schema.oaf.StructuredProperty; -import eu.dnetlib.dhp.schema.oaf.Subject; -import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; - -public class Constants { - - public static final String DOI = "doi"; - public static final String DOI_CLASSNAME = "Digital Object Identifier"; - - public static final String DEFAULT_DELIMITER = ","; - public static final String DEFAULT_FOS_DELIMITER = "\t"; - - public static final String UPDATE_DATA_INFO_TYPE = "update"; -// public static final String UPDATE_SUBJECT_FOS_CLASS_ID = "subject:fos"; - public static final String UPDATE_CLASS_NAME = "Inferred by OpenAIRE"; -// public static final String UPDATE_MEASURE_BIP_CLASS_ID = "measure:bip"; -// public static final String UPDATE_SUBJECT_SDG_CLASS_ID = "subject:sdg"; -// public static final String UPDATE_MEASURE_USAGE_COUNTS_CLASS_ID = "measure:usage_counts"; -// public static final String UPDATE_KEY_USAGE_COUNTS = "count"; - - public static final String UPDATE_MEASURE_STATS_MODEL_CLASS_ID = "measure:stats_model"; - public static final String UPDATE_KEY_STATS_MODEL = "stats_model"; - -// public static final String UPDATE_MEASURE_PUBLICLY_FUNDED_CLASS_ID = "measure:publicly_funded"; -// public static final String UPDATE_KEY_PUBLICLY_FUNDED = "publicly_funded"; - -// public static final String FOS_CLASS_ID = "FOS"; -// public static final String FOS_CLASS_NAME = "Fields of Science and Technology classification"; -// -// public static final String SDG_CLASS_ID = "SDG"; -// public static final String SDG_CLASS_NAME = "Sustainable Development Goals"; - - public static final String NULL = "NULL"; - - public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - - private Constants() { - } - - public static Boolean isSparkSessionManaged(ArgumentApplicationParser parser) { - return Optional - .ofNullable(parser.get("isSparkSessionManaged")) - .map(Boolean::valueOf) - .orElse(Boolean.TRUE); - } - - public static Dataset readPath( - SparkSession spark, String inputPath, Class clazz) { - return spark - .read() - .textFile(inputPath) - .map((MapFunction) value -> OBJECT_MAPPER.readValue(value, clazz), Encoders.bean(clazz)); - } - - public static Subject getSubject(String sbj, String classid, String classname, - String diqualifierclassid) { - if (sbj == null || sbj.equals(NULL)) - return null; - Subject s = new Subject(); - s.setValue(sbj); - s - .setQualifier( - OafMapperUtils - .qualifier( - classid, - classname, - ModelConstants.DNET_SUBJECT_TYPOLOGIES, - ModelConstants.DNET_SUBJECT_TYPOLOGIES)); - s - .setDataInfo( - OafMapperUtils - .dataInfo( - false, - UPDATE_DATA_INFO_TYPE, - true, - false, - OafMapperUtils - .qualifier( - diqualifierclassid, - UPDATE_CLASS_NAME, - ModelConstants.DNET_PROVENANCE_ACTIONS, - ModelConstants.DNET_PROVENANCE_ACTIONS), - "")); - - return s; - - } -} diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsAtomicActionsJob.java b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsAtomicActionsJob.java index f6e2cfe01..e984d68fc 100644 --- a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsAtomicActionsJob.java +++ b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsAtomicActionsJob.java @@ -1,7 +1,6 @@ package eu.dnetlib.dhp.actionmanager.stats_actionsets; -import static eu.dnetlib.dhp.actionmanager.Constants.*; import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession; import java.io.Serializable; @@ -12,14 +11,22 @@ import java.util.Optional; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.compress.GzipCodec; import org.apache.hadoop.mapred.SequenceFileOutputFormat; import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; 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.apache.spark.sql.Encoders; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,6 +72,9 @@ public class StatsAtomicActionsJob implements Serializable { SparkConf conf = new SparkConf(); conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + conf.set("spark.speculation", "false"); + conf.set("spark.hadoop.mapreduce.map.speculative", "false"); + conf.set("spark.hadoop.mapreduce.reduce.speculative", "false"); final String dbname = parser.get("statsDB"); @@ -75,75 +85,26 @@ public class StatsAtomicActionsJob implements Serializable { isSparkSessionManaged, spark -> { removeOutputDir(spark, outputPath); - prepareGreenData(dbname, spark, workingPath + "/greenOADB", "indi_pub_green_oa", "id"); - prepareDiamondData(dbname, spark, workingPath + "/diamondOADΒ", "indi_pub_diamond", "id"); - preparePubliclyFundedData( - dbname, spark, workingPath + "/publiclyFundedDΒ", "indi_funded_result_with_fundref", "id"); - prepareOAColourData(dbname, spark, workingPath + "/oacolourDB", "", "id"); + prepareResultEnhancement(dbname, spark, workingPath + "/resultEnhancements", "id"); writeActionSet(spark, workingPath, outputPath); }); } - private static void prepareGreenData(String dbname, SparkSession spark, String workingPath, String tableName, + private static void prepareResultEnhancement(String dbname, SparkSession spark, String workingPath, String resultAttributeName) { spark .sql( String .format( - "select %s as id, green_oa as green_oa " + - "from %s.%s", - resultAttributeName, dbname, tableName)) - .as(Encoders.bean(StatsGreenOAModel.class)) - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .json(workingPath); - } - - private static void prepareDiamondData(String dbname, SparkSession spark, String workingPath, String tableName, - String resultAttributeName) { - spark - .sql( - String - .format( - "select %s as id, in_diamond_journal as in_diamond_journal " + - "from %s.%s", - resultAttributeName, dbname, tableName)) - .as(Encoders.bean(StatsDiamondOAModel.class)) - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .json(workingPath); - } - - private static void preparePubliclyFundedData(String dbname, SparkSession spark, String workingPath, - String tableName, - String resultAttributeName) { - spark - .sql( - String - .format( - "select %s as id, fundref as publicly_funded " + - "from %s.%s", - resultAttributeName, dbname, tableName)) - .as(Encoders.bean(StatsPubliclyFundedModel.class)) - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .json(workingPath); - } - - private static void prepareOAColourData(String dbname, SparkSession spark, String workingPath, String tableName, - String resultAttributeName) { - spark - .sql( - String - .format( - "select b.%s as id, is_gold, is_bronze_oa, is_hybrid from %s.indi_pub_bronze_oa b " + + "select b.%s as id, is_gold, is_bronze_oa, is_hybrid,green_oa, in_diamond_journal,f.fundref as publicly_funded " + + "from %s.indi_pub_bronze_oa b " + "left outer join %s.indi_pub_gold_oa g on g.id=b.id " + - "left outer join %s.indi_pub_hybrid h on b.id=h.id", - resultAttributeName, dbname, dbname, dbname)) - .as(Encoders.bean(StatsOAColourModel.class)) + "left outer join %s.indi_pub_hybrid h on b.id=h.id " + + "left outer join %s.indi_pub_green_oa gr on b.id=gr.id " + + "left outer join %s.indi_pub_diamond d on b.id=d.id " + + "left outer join %s.indi_funded_result_with_fundref f on b.id=f.id ", + resultAttributeName, dbname, dbname, dbname, dbname, dbname, dbname)) + .as(Encoders.bean(StatsResultEnhancementModel.class)) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") @@ -152,125 +113,39 @@ public class StatsAtomicActionsJob implements Serializable { public static void writeActionSet(SparkSession spark, String inputPath, String outputPath) { - getFinalIndicatorsGreenResult(spark, inputPath + "/greenOADB") + getResultEnhancements(spark, inputPath + "/resultEnhancements") .toJavaRDD() .map(p -> new AtomicAction(p.getClass(), p)) - .union( - getFinalIndicatorsDiamondResult(spark, inputPath + "/diamondOADΒ") - .toJavaRDD() - .map(p -> new AtomicAction(p.getClass(), p))) - .union( - getFinalIndicatorsPubliclyFundedResult(spark, inputPath + "/publiclyFundedDΒ") - .toJavaRDD() - .map(p -> new AtomicAction(p.getClass(), p))) - .union( - getFinalIndicatorsOAColourResult(spark, inputPath + "/oacolourDB") - .toJavaRDD() - .map(p -> new AtomicAction(p.getClass(), p))) .mapToPair( aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), new Text(OBJECT_MAPPER.writeValueAsString(aa)))) - .saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class); + .saveAsHadoopFile( + outputPath, + Text.class, + Text.class, + SequenceFileOutputFormat.class, + GzipCodec.class); } - public static Measure newMeasureInstance(String id) { - Measure m = new Measure(); - m.setId(id); - m.setUnit(new ArrayList<>()); - return m; - } + private static Dataset getResultEnhancements(SparkSession spark, String inputPath) { - private static Dataset getFinalIndicatorsGreenResult(SparkSession spark, String inputPath) { - - return readPath(spark, inputPath, StatsGreenOAModel.class) - .map((MapFunction) usm -> { + return readPath(spark, inputPath, StatsResultEnhancementModel.class) + .map((MapFunction) usm -> { Result r = new Result(); r.setId("50|" + usm.getId()); - r.setMeasures(getMeasure(usm.isGreen_oa(), "green_oa")); + r.setIsInDiamondJournal(usm.isIn_diamond_journal()); + r.setIsGreen(usm.isGreen_oa()); + r.setPubliclyFunded(usm.isPublicly_funded()); + if (usm.isIs_bronze_oa()) + r.setOpenAccessColor(OpenAccessColor.bronze); + else if (usm.isIs_gold()) + r.setOpenAccessColor(OpenAccessColor.bronze); + else if (usm.isIs_gold()) + r.setOpenAccessColor(OpenAccessColor.gold); return r; }, Encoders.bean(Result.class)); } - private static Dataset getFinalIndicatorsDiamondResult(SparkSession spark, String inputPath) { - - return readPath(spark, inputPath, StatsDiamondOAModel.class) - .map((MapFunction) usm -> { - Result r = new Result(); - r.setId("50|" + usm.getId()); - r.setMeasures(getMeasure(usm.isIn_diamond_journal(), "in_diamond_journal")); - return r; - }, Encoders.bean(Result.class)); - } - - private static Dataset getFinalIndicatorsPubliclyFundedResult(SparkSession spark, String inputPath) { - - return readPath(spark, inputPath, StatsPubliclyFundedModel.class) - .map((MapFunction) usm -> { - Result r = new Result(); - r.setId("50|" + usm.getId()); - r.setMeasures(getMeasure(usm.isPublicly_funded(), "publicly_funded")); - return r; - }, Encoders.bean(Result.class)); - } - - private static Dataset getFinalIndicatorsOAColourResult(SparkSession spark, String inputPath) { - - return readPath(spark, inputPath, StatsOAColourModel.class) - .map((MapFunction) usm -> { - Result r = new Result(); - r.setId("50|" + usm.getId()); - r.setMeasures(getMeasureOAColour(usm.isIs_gold(), usm.isIs_bronze_oa(), usm.isIs_hybrid())); - return r; - }, Encoders.bean(Result.class)); - } - - private static List getMeasure(Boolean is_model_oa, String model_type) { - DataInfo dataInfo = OafMapperUtils - .dataInfo( - false, - UPDATE_DATA_INFO_TYPE, - true, - false, - OafMapperUtils - .qualifier( - UPDATE_MEASURE_STATS_MODEL_CLASS_ID, - UPDATE_CLASS_NAME, - ModelConstants.DNET_PROVENANCE_ACTIONS, - ModelConstants.DNET_PROVENANCE_ACTIONS), - ""); - - return Arrays - .asList( - OafMapperUtils - .newMeasureInstance(model_type, String.valueOf(is_model_oa), UPDATE_KEY_STATS_MODEL, dataInfo)); - } - - private static List getMeasureOAColour(Boolean is_gold, Boolean is_bronze_oa, Boolean is_hybrid) { - DataInfo dataInfo = OafMapperUtils - .dataInfo( - false, - UPDATE_DATA_INFO_TYPE, - true, - false, - OafMapperUtils - .qualifier( - UPDATE_MEASURE_STATS_MODEL_CLASS_ID, - UPDATE_CLASS_NAME, - ModelConstants.DNET_PROVENANCE_ACTIONS, - ModelConstants.DNET_PROVENANCE_ACTIONS), - ""); - - return Arrays - .asList( - OafMapperUtils - .newMeasureInstance("is_gold", String.valueOf(is_gold), UPDATE_KEY_STATS_MODEL, dataInfo), - OafMapperUtils - .newMeasureInstance("is_bronze_oa", String.valueOf(is_bronze_oa), UPDATE_KEY_STATS_MODEL, dataInfo), - OafMapperUtils - .newMeasureInstance("is_hybrid", String.valueOf(is_hybrid), UPDATE_KEY_STATS_MODEL, dataInfo)); - - } - private static void removeOutputDir(SparkSession spark, String path) { HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); } @@ -282,5 +157,4 @@ public class StatsAtomicActionsJob implements Serializable { .textFile(inputPath) .map((MapFunction) value -> OBJECT_MAPPER.readValue(value, clazz), Encoders.bean(clazz)); } - } diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsDiamondOAModel.java b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsDiamondOAModel.java deleted file mode 100644 index 340790fa8..000000000 --- a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsDiamondOAModel.java +++ /dev/null @@ -1,29 +0,0 @@ - -package eu.dnetlib.dhp.actionmanager.stats_actionsets; - -import java.io.Serializable; - -/** - * @author dimitris.pierrakos - * @Date 30/10/23 - */ -public class StatsDiamondOAModel implements Serializable { - private String id; - private boolean in_diamond_journal; - - public String getId() { - return id; - } - - public void setId(String id) { - this.id = id; - } - - public boolean isIn_diamond_journal() { - return in_diamond_journal; - } - - public void setIn_diamond_journal(boolean in_diamond_journal) { - this.in_diamond_journal = in_diamond_journal; - } -} diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsGreenOAModel.java b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsGreenOAModel.java deleted file mode 100644 index 3d59bdf95..000000000 --- a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsGreenOAModel.java +++ /dev/null @@ -1,29 +0,0 @@ - -package eu.dnetlib.dhp.actionmanager.stats_actionsets; - -import java.io.Serializable; - -/** - * @author dimitris.pierrakos - * @Date 30/10/23 - */ -public class StatsGreenOAModel implements Serializable { - private String id; - private boolean green_oa; - - public String getId() { - return id; - } - - public void setId(String id) { - this.id = id; - } - - public boolean isGreen_oa() { - return green_oa; - } - - public void setGreen_oa(boolean green_oa) { - this.green_oa = green_oa; - } -} diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsOAColourModel.java b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsOAColourModel.java deleted file mode 100644 index a83eb4f60..000000000 --- a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsOAColourModel.java +++ /dev/null @@ -1,47 +0,0 @@ - -package eu.dnetlib.dhp.actionmanager.stats_actionsets; - -import java.io.Serializable; - -/** - * @author dimitris.pierrakos - * @Date 30/10/23 - */ -public class StatsOAColourModel implements Serializable { - private String id; - private boolean is_gold; - private boolean is_bronze_oa; - private boolean is_hybrid; - - public String getId() { - return id; - } - - public void setId(String id) { - this.id = id; - } - - public boolean isIs_gold() { - return is_gold; - } - - public void setIs_gold(boolean is_gold) { - this.is_gold = is_gold; - } - - public boolean isIs_bronze_oa() { - return is_bronze_oa; - } - - public void setIs_bronze_oa(boolean is_bronze_oa) { - this.is_bronze_oa = is_bronze_oa; - } - - public boolean isIs_hybrid() { - return is_hybrid; - } - - public void setIs_hybrid(boolean is_hybrid) { - this.is_hybrid = is_hybrid; - } -} diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsPubliclyFundedModel.java b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsPubliclyFundedModel.java deleted file mode 100644 index 6bdc78220..000000000 --- a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsPubliclyFundedModel.java +++ /dev/null @@ -1,29 +0,0 @@ - -package eu.dnetlib.dhp.actionmanager.stats_actionsets; - -import java.io.Serializable; - -/** - * @author dimitris.pierrakos - * @Date 30/10/23 - */ -public class StatsPubliclyFundedModel implements Serializable { - private String id; - private boolean publicly_funded; - - public String getId() { - return id; - } - - public void setId(String id) { - this.id = id; - } - - public boolean isPublicly_funded() { - return publicly_funded; - } - - public void setPublicly_funded(boolean publicly_funded) { - this.publicly_funded = publicly_funded; - } -} diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsResultEnhancementModel.java b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsResultEnhancementModel.java new file mode 100644 index 000000000..f794c5591 --- /dev/null +++ b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsResultEnhancementModel.java @@ -0,0 +1,76 @@ + +package eu.dnetlib.dhp.actionmanager.stats_actionsets; + +import java.io.Serializable; + +import eu.dnetlib.dhp.schema.oaf.*; + +/** + * @author dimitris.pierrakos + * @Date 30/10/23 + */ +public class StatsResultEnhancementModel implements Serializable { + private String id; + private Boolean is_gold; + private Boolean is_bronze_oa; + private Boolean is_hybrid; + private boolean in_diamond_journal; + private boolean green_oa; + private boolean publicly_funded; + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public Boolean isIs_gold() { + return is_gold; + } + + public void setIs_gold(Boolean is_gold) { + this.is_gold = is_gold; + } + + public Boolean isIs_bronze_oa() { + return is_bronze_oa; + } + + public void setIs_bronze_oa(Boolean is_bronze_oa) { + this.is_bronze_oa = is_bronze_oa; + } + + public Boolean isIs_hybrid() { + return is_hybrid; + } + + public void setIs_hybrid(Boolean is_hybrid) { + this.is_hybrid = is_hybrid; + } + + public boolean isIn_diamond_journal() { + return in_diamond_journal; + } + + public void setIn_diamond_journal(boolean in_diamond_journal) { + this.in_diamond_journal = in_diamond_journal; + } + + public boolean isGreen_oa() { + return green_oa; + } + + public void setGreen_oa(boolean green_oa) { + this.green_oa = green_oa; + } + + public boolean isPublicly_funded() { + return publicly_funded; + } + + public void setPublicly_funded(boolean publicly_funded) { + this.publicly_funded = publicly_funded; + } +} diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/resources/eu/dnetlib/dhp/actionmanager/stats_actionsets/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-actionsets/src/main/resources/eu/dnetlib/dhp/actionmanager/stats_actionsets/oozie_app/workflow.xml index 1aa4e1050..79fd3f56e 100644 --- a/dhp-workflows/dhp-stats-actionsets/src/main/resources/eu/dnetlib/dhp/actionmanager/stats_actionsets/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-actionsets/src/main/resources/eu/dnetlib/dhp/actionmanager/stats_actionsets/oozie_app/workflow.xml @@ -68,39 +68,11 @@ Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]
- - - yarn - cluster - Produces the atomic action with the usage stats count for results - - eu.dnetlib.dhp.actionmanager.stats_actionsets.SparkAtomicActionGreenOAJob - dhp-stats-actionsets-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} - - --hive_metastore_uris${hiveMetastoreUris} - --outputPath${outputPath} - --statsDB${statsDB} - --workingPath${workingDir} - - - - - yarn cluster Produces the atomic action with the stats green_oa for results - eu.dnetlib.dhp.actionmanager.stats_actionsets.StatsAtomicActionsJob dhp-stats-actionsets-${projectVersion}.jar From 6ce36b3e412020619afd8a242d863ec3a4f632b5 Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Tue, 14 Nov 2023 12:04:29 +0100 Subject: [PATCH 20/60] Implemented ORCID Workflow on DHP-Aggregation for retrieving ORCID DUMP and generating tables --- .../orcid/DownloadORCIDDumpApplication.java | 102 +++ .../collection/orcid/ExtractORCIDDump.java | 71 +++ .../dhp/collection/orcid/ORCIDExtractor.java | 171 ++++++ .../dhp/collection/orcid/OrcidParser.java | 251 ++++++++ .../dhp/collection/orcid/model/Author.java | 83 +++ .../collection/orcid/model/Employment.java | 54 ++ .../dhp/collection/orcid/model/ORCIDItem.java | 14 + .../dhp/collection/orcid/model/Pid.java | 33 + .../dhp/collection/orcid/model/Work.java | 35 ++ .../orcid/download_orcid_parameter.json | 21 + .../orcid/extract_orcid_parameter.json | 21 + .../orcid/generate_orcid_table_parameter.json | 21 + .../orcid/oozie_app/config-default.xml | 23 + .../collection/orcid/oozie_app/workflow.xml | 81 +++ .../preprocess_orcid_dump_parameter.json | 21 + .../orcid/SparkGenerateORCIDTable.scala | 101 +++ .../collection/orcid/DownloadORCIDTest.java | 158 +++++ .../activity_work_0000-0002-2536-4498.xml | 69 +++ .../activity_work_0000-0002-5982-8983.xml | 79 +++ ...ty_work_0000-0003-2760-1191-similarity.xml | 113 ++++ .../activity_work_0000-0003-2760-1191.xml | 106 ++++ ..._work_0000-0003-2760-1191_contributors.xml | 101 +++ .../dhp/collection/orcid/employment.xml | 50 ++ .../dhp/collection/orcid/employment_2.xml | 55 ++ .../dhp/collection/orcid/employment_3.xml | 62 ++ .../dnetlib/dhp/collection/orcid/summary.xml | 581 ++++++++++++++++++ 26 files changed, 2477 insertions(+) create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/DownloadORCIDDumpApplication.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ExtractORCIDDump.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ORCIDExtractor.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/OrcidParser.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Author.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Employment.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/ORCIDItem.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Pid.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Work.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/download_orcid_parameter.json create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/extract_orcid_parameter.json create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/generate_orcid_table_parameter.json create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/oozie_app/workflow.xml create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/preprocess_orcid_dump_parameter.json create mode 100644 dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/orcid/SparkGenerateORCIDTable.scala create mode 100644 dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/orcid/DownloadORCIDTest.java create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0002-2536-4498.xml create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0002-5982-8983.xml create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0003-2760-1191-similarity.xml create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0003-2760-1191.xml create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0003-2760-1191_contributors.xml create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/employment.xml create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/employment_2.xml create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/employment_3.xml create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/summary.xml diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/DownloadORCIDDumpApplication.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/DownloadORCIDDumpApplication.java new file mode 100644 index 000000000..69661bbf6 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/DownloadORCIDDumpApplication.java @@ -0,0 +1,102 @@ + +package eu.dnetlib.dhp.collection.orcid; + +import static eu.dnetlib.dhp.utils.DHPUtils.getHadoopConfiguration; + +import java.io.InputStream; +import java.net.URL; +import java.net.URLConnection; +import java.util.Objects; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClientBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; + +public class DownloadORCIDDumpApplication { + private static final Logger log = LoggerFactory.getLogger(DownloadORCIDDumpApplication.class); + + private final FileSystem fileSystem; + + public DownloadORCIDDumpApplication(FileSystem fileSystem) { + this.fileSystem = fileSystem; + } + + public static void main(String[] args) throws Exception { + final ArgumentApplicationParser argumentParser = new ArgumentApplicationParser( + IOUtils + .toString( + Objects + .requireNonNull( + DownloadORCIDDumpApplication.class + .getResourceAsStream( + "/eu/dnetlib/dhp/collection/orcid/download_orcid_parameter.json")))); + argumentParser.parseArgument(args); + + final String hdfsuri = argumentParser.get("namenode"); + log.info("hdfsURI is {}", hdfsuri); + + final String targetPath = argumentParser.get("targetPath"); + log.info("targetPath is {}", targetPath); + + final String apiURL = argumentParser.get("apiURL"); + log.info("apiURL is {}", apiURL); + + final FileSystem fileSystem = FileSystem.get(getHadoopConfiguration(hdfsuri)); + + new DownloadORCIDDumpApplication(fileSystem).run(targetPath, apiURL); + + } + + private void downloadItem(final String name, final String itemURL, final String basePath) { + try { + final Path hdfsWritePath = new Path(String.format("%s/%s", basePath, name)); + final FSDataOutputStream fsDataOutputStream = fileSystem.create(hdfsWritePath, true); + final HttpGet request = new HttpGet(itemURL); + final int timeout = 60; // seconds + final RequestConfig config = RequestConfig + .custom() + .setConnectTimeout(timeout * 1000) + .setConnectionRequestTimeout(timeout * 1000) + .setSocketTimeout(timeout * 1000) + .build(); + log.info("Downloading url {} into {}", itemURL, hdfsWritePath.getName()); + try (CloseableHttpClient client = HttpClientBuilder.create().setDefaultRequestConfig(config).build(); + CloseableHttpResponse response = client.execute(request)) { + int responseCode = response.getStatusLine().getStatusCode(); + log.info("Response code is {}", responseCode); + if (responseCode >= 200 && responseCode < 400) { + IOUtils.copy(response.getEntity().getContent(), fsDataOutputStream); + } + } catch (Throwable eu) { + throw new RuntimeException(eu); + } + } catch (Throwable e) { + throw new RuntimeException(e); + } + } + + protected void run(final String targetPath, final String apiURL) throws Exception { + final ObjectMapper mapper = new ObjectMapper(); + final URL url = new URL(apiURL); + URLConnection conn = url.openConnection(); + InputStream is = conn.getInputStream(); + final String json = IOUtils.toString(is); + JsonNode jsonNode = mapper.readTree(json); + jsonNode + .get("files") + .forEach(i -> downloadItem(i.get("name").asText(), i.get("download_url").asText(), targetPath)); + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ExtractORCIDDump.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ExtractORCIDDump.java new file mode 100644 index 000000000..4aefdb5e9 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ExtractORCIDDump.java @@ -0,0 +1,71 @@ + +package eu.dnetlib.dhp.collection.orcid; + +import static eu.dnetlib.dhp.utils.DHPUtils.getHadoopConfiguration; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; + +public class ExtractORCIDDump { + private static final Logger log = LoggerFactory.getLogger(ExtractORCIDDump.class); + + private final FileSystem fileSystem; + + public ExtractORCIDDump(FileSystem fileSystem) { + this.fileSystem = fileSystem; + } + + public static void main(String[] args) throws Exception { + final ArgumentApplicationParser argumentParser = new ArgumentApplicationParser( + IOUtils + .toString( + Objects + .requireNonNull( + DownloadORCIDDumpApplication.class + .getResourceAsStream( + "/eu/dnetlib/dhp/collection/orcid/extract_orcid_parameter.json")))); + argumentParser.parseArgument(args); + + final String hdfsuri = argumentParser.get("namenode"); + log.info("hdfsURI is {}", hdfsuri); + + final String sourcePath = argumentParser.get("sourcePath"); + log.info("sourcePath is {}", sourcePath); + + final String targetPath = argumentParser.get("targetPath"); + log.info("targetPath is {}", targetPath); + + final FileSystem fileSystem = FileSystem.get(getHadoopConfiguration(hdfsuri)); + + new ExtractORCIDDump(fileSystem).run(sourcePath, targetPath); + + } + + public void run(final String sourcePath, final String targetPath) throws IOException, InterruptedException { + RemoteIterator ls = fileSystem.listFiles(new Path(sourcePath), false); + final List workers = new ArrayList<>(); + int i = 0; + while (ls.hasNext()) { + LocatedFileStatus current = ls.next(); + if (current.getPath().getName().endsWith("tar.gz")) { + workers.add(new ORCIDExtractor(fileSystem, "" + i++, current.getPath(), targetPath)); + } + } + workers.forEach(Thread::start); + for (ORCIDExtractor worker : workers) { + worker.join(); + } + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ORCIDExtractor.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ORCIDExtractor.java new file mode 100644 index 000000000..11f4c55d8 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ORCIDExtractor.java @@ -0,0 +1,171 @@ + +package eu.dnetlib.dhp.collection.orcid; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.util.HashMap; +import java.util.Map; + +import org.apache.commons.compress.archivers.tar.TarArchiveEntry; +import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.CompressionCodecFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/**\ + * The ORCIDExtractor class extracts ORCID data from a TAR archive. + * The class creates a map of SequenceFile.Writer objects, one for each type of data that is to be extracted (e.g., employments, works, summaries). + * Then, it iterates over the TAR archive and writes each entry to the appropriate SequenceFile.Writer object. + * Finally, it closes all the SequenceFile.Writer objects. + */ +public class ORCIDExtractor extends Thread { + + private static final Logger log = LoggerFactory.getLogger(ORCIDExtractor.class); + + private final FileSystem fileSystem; + + private final String id; + + private final Path sourcePath; + + private final String baseOutputPath; + + public ORCIDExtractor(FileSystem fileSystem, String id, Path sourcePath, String baseOutputPath) { + this.fileSystem = fileSystem; + this.id = id; + this.sourcePath = sourcePath; + this.baseOutputPath = baseOutputPath; + } + + /** + * creates a map of SequenceFile.Writer objects, + * one for each type of data that is to be extracted. The map is created based on the filename in the TAR archive. + * For example, if the filename is employments.json, the map will contain an entry for the SequenceFile.Writer + * object that writes employment data. + * @return the Map + */ + private Map createMap() { + try { + log.info("Thread {} Creating sequence files starting from this input Path {}", id, sourcePath.getName()); + Map res = new HashMap<>(); + if (sourcePath.getName().contains("summaries")) { + + final String summaryPath = String.format("%s/summaries_%s", baseOutputPath, id); + final SequenceFile.Writer summary_file = SequenceFile + .createWriter( + fileSystem.getConf(), + SequenceFile.Writer.file(new Path(summaryPath)), + SequenceFile.Writer.keyClass(Text.class), + SequenceFile.Writer.valueClass(Text.class)); + + log.info("Thread {} Creating only summary path here {}", id, summaryPath); + res.put("summary", summary_file); + return res; + } else { + String employmentsPath = String.format("%s/employments_%s", baseOutputPath, id); + final SequenceFile.Writer employments_file = SequenceFile + .createWriter( + fileSystem.getConf(), + SequenceFile.Writer.file(new Path(employmentsPath)), + SequenceFile.Writer.keyClass(Text.class), + SequenceFile.Writer.valueClass(Text.class)); + res.put("employments", employments_file); + log.info("Thread {} Creating employments path here {}", id, employmentsPath); + + final String worksPath = String.format("%s/works_%s", baseOutputPath, id); + final SequenceFile.Writer works_file = SequenceFile + .createWriter( + fileSystem.getConf(), + SequenceFile.Writer.file(new Path(worksPath)), + SequenceFile.Writer.keyClass(Text.class), + SequenceFile.Writer.valueClass(Text.class)); + res.put("works", works_file); + log.info("Thread {} Creating works path here {}", id, worksPath); + + return res; + } + } catch (Throwable e) { + throw new RuntimeException(e); + } + } + + @Override + public void run() { + + CompressionCodecFactory factory = new CompressionCodecFactory(fileSystem.getConf()); + CompressionCodec codec = factory.getCodec(sourcePath); + if (codec == null) { + System.err.println("No codec found for " + sourcePath.getName()); + System.exit(1); + } + + InputStream gzipInputStream = null; + try { + gzipInputStream = codec.createInputStream(fileSystem.open(sourcePath)); + final Map fileMap = createMap(); + iterateTar(fileMap, gzipInputStream); + + } catch (IOException e) { + throw new RuntimeException(e); + } finally { + log.info("Closing gzip stream"); + IOUtils.closeStream(gzipInputStream); + } + + } + + private SequenceFile.Writer retrieveFile(Map fileMap, final String path) { + if (sourcePath.getName().contains("summaries")) { + return fileMap.get("summary"); + } + + if (path.contains("works")) { + return fileMap.get("works"); + } + if (path.contains("employments")) + return fileMap.get("employments"); + return null; + } + + private void iterateTar(Map fileMap, InputStream gzipInputStream) throws IOException { + + int extractedItem = 0; + try (final TarArchiveInputStream tais = new TarArchiveInputStream(gzipInputStream)) { + + TarArchiveEntry entry; + while ((entry = tais.getNextTarEntry()) != null) { + + if (entry.isFile()) { + + final SequenceFile.Writer fl = retrieveFile(fileMap, entry.getName()); + if (fl != null) { + final Text key = new Text(entry.getName()); + final Text value = new Text( + org.apache.commons.io.IOUtils.toString(new BufferedReader(new InputStreamReader(tais)))); + fl.append(key, value); + extractedItem++; + if (extractedItem % 100000 == 0) { + log.info("Thread {}: Extracted {} items", id, extractedItem); + break; + } + } + } + } + } finally { + for (SequenceFile.Writer k : fileMap.values()) { + log.info("Thread {}: Completed processed {} items", id, extractedItem); + k.hflush(); + k.close(); + } + } + + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/OrcidParser.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/OrcidParser.java new file mode 100644 index 000000000..159b8a5fc --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/OrcidParser.java @@ -0,0 +1,251 @@ + +package eu.dnetlib.dhp.collection.orcid; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.ximpleware.*; + +import eu.dnetlib.dhp.collection.orcid.model.*; +import eu.dnetlib.dhp.parser.utility.VtdException; +import eu.dnetlib.dhp.parser.utility.VtdUtilityParser; + +public class OrcidParser { + + final Logger log = LoggerFactory.getLogger(OrcidParser.class); + private VTDNav vn; + + private AutoPilot ap; + private static final String NS_COMMON_URL = "http://www.orcid.org/ns/common"; + private static final String NS_COMMON = "common"; + private static final String NS_PERSON_URL = "http://www.orcid.org/ns/person"; + private static final String NS_PERSON = "person"; + private static final String NS_DETAILS_URL = "http://www.orcid.org/ns/personal-details"; + private static final String NS_DETAILS = "personal-details"; + private static final String NS_OTHER_URL = "http://www.orcid.org/ns/other-name"; + private static final String NS_OTHER = "other-name"; + private static final String NS_RECORD_URL = "http://www.orcid.org/ns/record"; + private static final String NS_RECORD = "record"; + private static final String NS_ERROR_URL = "http://www.orcid.org/ns/error"; + private static final String NS_ACTIVITIES = "activities"; + private static final String NS_ACTIVITIES_URL = "http://www.orcid.org/ns/activities"; + private static final String NS_WORK = "work"; + private static final String NS_WORK_URL = "http://www.orcid.org/ns/work"; + + private static final String NS_ERROR = "error"; + private static final String NS_HISTORY = "history"; + private static final String NS_HISTORY_URL = "http://www.orcid.org/ns/history"; + private static final String NS_BULK_URL = "http://www.orcid.org/ns/bulk"; + private static final String NS_BULK = "bulk"; + private static final String NS_EXTERNAL = "external-identifier"; + private static final String NS_EXTERNAL_URL = "http://www.orcid.org/ns/external-identifier"; + + private void generateParsedDocument(final String xml) throws ParseException { + final VTDGen vg = new VTDGen(); + vg.setDoc(xml.getBytes()); + vg.parse(true); + this.vn = vg.getNav(); + this.ap = new AutoPilot(vn); + ap.declareXPathNameSpace(NS_COMMON, NS_COMMON_URL); + ap.declareXPathNameSpace(NS_PERSON, NS_PERSON_URL); + ap.declareXPathNameSpace(NS_DETAILS, NS_DETAILS_URL); + ap.declareXPathNameSpace(NS_OTHER, NS_OTHER_URL); + ap.declareXPathNameSpace(NS_RECORD, NS_RECORD_URL); + ap.declareXPathNameSpace(NS_ERROR, NS_ERROR_URL); + ap.declareXPathNameSpace(NS_HISTORY, NS_HISTORY_URL); + ap.declareXPathNameSpace(NS_WORK, NS_WORK_URL); + ap.declareXPathNameSpace(NS_EXTERNAL, NS_EXTERNAL_URL); + ap.declareXPathNameSpace(NS_ACTIVITIES, NS_ACTIVITIES_URL); + } + + public Author parseSummary(final String xml) { + + try { + final Author author = new Author(); + generateParsedDocument(xml); + List recordNodes = VtdUtilityParser + .getTextValuesWithAttributes( + ap, vn, "//record:record", Arrays.asList("path")); + if (!recordNodes.isEmpty()) { + final String oid = (recordNodes.get(0).getAttributes().get("path")).substring(1); + author.setOrcid(oid); + } else { + return null; + } + List personNodes = VtdUtilityParser + .getTextValuesWithAttributes( + ap, vn, "//person:name", Arrays.asList("visibility")); + final String visibility = (personNodes.get(0).getAttributes().get("visibility")); + author.setVisibility(visibility); + final String name = VtdUtilityParser.getSingleValue(ap, vn, "//personal-details:given-names"); + author.setGivenName(name); + + final String surnames = VtdUtilityParser.getSingleValue(ap, vn, "//personal-details:family-name"); + author.setFamilyName(surnames); + + final String creditNames = VtdUtilityParser.getSingleValue(ap, vn, "//personal-details:credit-name"); + author.setCreditName(creditNames); + + final String biography = VtdUtilityParser + .getSingleValue(ap, vn, "//person:biography/personal-details:content"); + author.setBiography(biography); + + final List otherNames = VtdUtilityParser.getTextValue(ap, vn, "//other-name:content"); + if (!otherNames.isEmpty()) { + author.setOtherNames(otherNames); + } + + ap.selectXPath("//external-identifier:external-identifier"); + + while (ap.evalXPath() != -1) { + final Pid pid = new Pid(); + + final AutoPilot ap1 = new AutoPilot(ap.getNav()); + + ap1.selectXPath("./common:external-id-type"); + while (ap1.evalXPath() != -1) { + int it = vn.getText(); + pid.setSchema(vn.toNormalizedString(it)); + } + ap1.selectXPath("./common:external-id-value"); + while (ap1.evalXPath() != -1) { + int it = vn.getText(); + pid.setValue(vn.toNormalizedString(it)); + } + + author.addOtherPid(pid); + } + + return author; + } catch (Throwable e) { + log.error("Error on parsing {}", xml); + log.error(e.getMessage()); + return null; + } + } + + public Work parseWork(final String xml) { + + try { + final Work work = new Work(); + generateParsedDocument(xml); + List workNodes = VtdUtilityParser + .getTextValuesWithAttributes(ap, vn, "//work:work", Arrays.asList("path", "visibility")); + if (!workNodes.isEmpty()) { + final String oid = (workNodes.get(0).getAttributes().get("path")).split("/")[1]; + work.setOrcid(oid); + } else { + return null; + } + + ap.selectXPath("//common:external-id"); + + while (ap.evalXPath() != -1) { + final Pid pid = new Pid(); + + final AutoPilot ap1 = new AutoPilot(ap.getNav()); + + ap1.selectXPath("./common:external-id-type"); + while (ap1.evalXPath() != -1) { + int it = vn.getText(); + pid.setSchema(vn.toNormalizedString(it)); + } + ap1.selectXPath("./common:external-id-value"); + while (ap1.evalXPath() != -1) { + int it = vn.getText(); + pid.setValue(vn.toNormalizedString(it)); + } + + work.addPid(pid); + } + + work.setTitle(VtdUtilityParser.getSingleValue(ap, vn, "//work:title/common:title")); + + return work; + } catch (Throwable e) { + log.error("Error on parsing {}", xml); + log.error(e.getMessage()); + return null; + } + + } + + private String extractEmploymentDate(final String xpath) throws Exception { + + ap.selectXPath(xpath); + StringBuilder sb = new StringBuilder(); + while (ap.evalXPath() != -1) { + final AutoPilot ap1 = new AutoPilot(ap.getNav()); + ap1.selectXPath("./common:year"); + while (ap1.evalXPath() != -1) { + int it = vn.getText(); + sb.append(vn.toNormalizedString(it)); + } + ap1.selectXPath("./common:month"); + while (ap1.evalXPath() != -1) { + int it = vn.getText(); + sb.append("-"); + sb.append(vn.toNormalizedString(it)); + } + ap1.selectXPath("./common:day"); + while (ap1.evalXPath() != -1) { + int it = vn.getText(); + sb.append("-"); + sb.append(vn.toNormalizedString(it)); + } + } + return sb.toString(); + + } + + public Employment parseEmployment(final String xml) { + try { + final Employment employment = new Employment(); + generateParsedDocument(xml); + final String oid = VtdUtilityParser + .getSingleValue(ap, vn, "//common:source-orcid/common:path"); + if (StringUtils.isNotBlank(oid)) { + employment.setOrcid(oid); + } else { + return null; + } + final String depName = VtdUtilityParser + .getSingleValue(ap, vn, "//common:department-name"); + final String rolTitle = VtdUtilityParser + .getSingleValue(ap, vn, "//common:role-title"); + if (StringUtils.isNotBlank(rolTitle)) + employment.setRoleTitle(rolTitle); + if (StringUtils.isNotBlank(depName)) + employment.setDepartmentName(depName); + else + employment + .setDepartmentName( + VtdUtilityParser + .getSingleValue(ap, vn, "//common:organization/common:name")); + + employment.setStartDate(extractEmploymentDate("//common:start-date")); + employment.setEndDate(extractEmploymentDate("//common:end-date")); + + final String affiliationId = VtdUtilityParser + .getSingleValue(ap, vn, "//common:disambiguated-organization-identifier"); + final String affiliationIdType = VtdUtilityParser + .getSingleValue(ap, vn, "//common:disambiguation-source"); + + if (StringUtils.isNotBlank(affiliationId) || StringUtils.isNotBlank(affiliationIdType)) + employment.setAffiliationId(new Pid(affiliationId, affiliationIdType)); + + return employment; + } catch (Throwable e) { + log.error("Error on parsing {}", xml); + log.error(e.getMessage()); + return null; + } + + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Author.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Author.java new file mode 100644 index 000000000..32c321b41 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Author.java @@ -0,0 +1,83 @@ + +package eu.dnetlib.dhp.collection.orcid.model; + +import java.util.ArrayList; +import java.util.List; + +public class Author extends ORCIDItem { + private String givenName; + private String familyName; + + private String visibility; + + private String creditName; + + private List otherNames; + + private List otherPids; + + private String biography; + + public String getBiography() { + return biography; + } + + public void setBiography(String biography) { + this.biography = biography; + } + + public String getGivenName() { + return givenName; + } + + public void setGivenName(String givenName) { + this.givenName = givenName; + } + + public String getFamilyName() { + return familyName; + } + + public void setFamilyName(String familyName) { + this.familyName = familyName; + } + + public String getCreditName() { + return creditName; + } + + public void setCreditName(String creditName) { + this.creditName = creditName; + } + + public List getOtherNames() { + return otherNames; + } + + public void setOtherNames(List otherNames) { + this.otherNames = otherNames; + } + + public String getVisibility() { + return visibility; + } + + public void setVisibility(String visibility) { + this.visibility = visibility; + } + + public List getOtherPids() { + return otherPids; + } + + public void setOtherPids(List otherPids) { + this.otherPids = otherPids; + } + + public void addOtherPid(final Pid pid) { + + if (otherPids == null) + otherPids = new ArrayList<>(); + otherPids.add(pid); + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Employment.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Employment.java new file mode 100644 index 000000000..baee67d46 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Employment.java @@ -0,0 +1,54 @@ + +package eu.dnetlib.dhp.collection.orcid.model; + +public class Employment extends ORCIDItem { + + private String startDate; + private String EndDate; + + private Pid affiliationId; + + private String departmentName; + + private String roleTitle; + + public String getStartDate() { + return startDate; + } + + public void setStartDate(String startDate) { + this.startDate = startDate; + } + + public String getEndDate() { + return EndDate; + } + + public void setEndDate(String endDate) { + EndDate = endDate; + } + + public Pid getAffiliationId() { + return affiliationId; + } + + public void setAffiliationId(Pid affiliationId) { + this.affiliationId = affiliationId; + } + + public String getDepartmentName() { + return departmentName; + } + + public void setDepartmentName(String departmentName) { + this.departmentName = departmentName; + } + + public String getRoleTitle() { + return roleTitle; + } + + public void setRoleTitle(String roleTitle) { + this.roleTitle = roleTitle; + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/ORCIDItem.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/ORCIDItem.java new file mode 100644 index 000000000..6bc47bc26 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/ORCIDItem.java @@ -0,0 +1,14 @@ + +package eu.dnetlib.dhp.collection.orcid.model; + +public class ORCIDItem { + private String orcid; + + public String getOrcid() { + return orcid; + } + + public void setOrcid(String orcid) { + this.orcid = orcid; + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Pid.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Pid.java new file mode 100644 index 000000000..077dc2550 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Pid.java @@ -0,0 +1,33 @@ + +package eu.dnetlib.dhp.collection.orcid.model; + +public class Pid { + + private String value; + + private String schema; + + public Pid() { + } + + public Pid(String value, String schema) { + this.value = value; + this.schema = schema; + } + + public String getValue() { + return value; + } + + public void setValue(String value) { + this.value = value; + } + + public String getSchema() { + return schema; + } + + public void setSchema(String schema) { + this.schema = schema; + } +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Work.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Work.java new file mode 100644 index 000000000..670170323 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/model/Work.java @@ -0,0 +1,35 @@ + +package eu.dnetlib.dhp.collection.orcid.model; + +import java.util.ArrayList; +import java.util.List; + +public class Work extends ORCIDItem { + + private String title; + + private List pids; + + public String getTitle() { + return title; + } + + public void setTitle(String title) { + this.title = title; + } + + public List getPids() { + return pids; + } + + public void setPids(List pids) { + this.pids = pids; + } + + public void addPid(Pid pid) { + if (pids == null) + pids = new ArrayList<>(); + pids.add(pid); + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/download_orcid_parameter.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/download_orcid_parameter.json new file mode 100644 index 000000000..4a84cbbb9 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/download_orcid_parameter.json @@ -0,0 +1,21 @@ +[ + { + "paramName": "n", + "paramLongName": "namenode", + "paramDescription": "the Name Node URI", + "paramRequired": true + }, + { + "paramName": "t", + "paramLongName": "targetPath", + "paramDescription": "the target PATH where download the files", + "paramRequired": true + }, + { + "paramName": "a", + "paramLongName": "apiURL", + "paramDescription": "the FIGSHARE API id URL to retrieve all the dump files", + "paramRequired": true + } + +] \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/extract_orcid_parameter.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/extract_orcid_parameter.json new file mode 100644 index 000000000..4af371875 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/extract_orcid_parameter.json @@ -0,0 +1,21 @@ +[ + { + "paramName": "n", + "paramLongName": "namenode", + "paramDescription": "the Name Node URI", + "paramRequired": true + }, + { + "paramName": "t", + "paramLongName": "targetPath", + "paramDescription": "the target PATH to extract files", + "paramRequired": true + }, + { + "paramName": "s", + "paramLongName": "sourcePath", + "paramDescription": "the PATH where the tar.gz files were downloaded", + "paramRequired": true + } + +] \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/generate_orcid_table_parameter.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/generate_orcid_table_parameter.json new file mode 100644 index 000000000..01d81ea97 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/generate_orcid_table_parameter.json @@ -0,0 +1,21 @@ +[ + { + "paramName": "m", + "paramLongName": "master", + "paramDescription": "the master name", + "paramRequired": true + }, + { + "paramName": "t", + "paramLongName": "targetPath", + "paramDescription": "the target PATH of the DF tables", + "paramRequired": true + }, + { + "paramName": "s", + "paramLongName": "sourcePath", + "paramDescription": "the PATH of the ORCID sequence file", + "paramRequired": true + } + +] \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/oozie_app/config-default.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/oozie_app/config-default.xml new file mode 100644 index 000000000..dd3c32c62 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/oozie_app/config-default.xml @@ -0,0 +1,23 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + + + oozie.launcher.mapreduce.user.classpath.first + true + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/oozie_app/workflow.xml new file mode 100644 index 000000000..1a5f425e5 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/oozie_app/workflow.xml @@ -0,0 +1,81 @@ + + + + targetPath + the path to store the original ORCID dump + + + apiURL + The figshare API URL to retrieve the list file to download + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + oozie.launcher.mapreduce.user.classpath.first + true + + + eu.dnetlib.dhp.collection.orcid.DownloadORCIDDumpApplication + --namenode${nameNode} + --targetPath${targetPath} + --apiURL${apiURL} + + + + + + + + + + oozie.launcher.mapreduce.user.classpath.first + true + + + + eu.dnetlib.dhp.collection.orcid.ExtractORCIDDump + -Xmx6g + --namenode${nameNode} + --sourcePath${targetPath} + --targetPath${targetPath}/extracted + + + + + + + + yarn + cluster + Generate ORCID Tables + eu.dnetlib.dhp.collection.orcid.SparkGenerateORCIDTable + dhp-aggregation-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=2g + --conf spark.sql.shuffle.partitions=3000 + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${targetPath}/extracted + --targetPath${targetPath}/tables + --masteryarn + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/preprocess_orcid_dump_parameter.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/preprocess_orcid_dump_parameter.json new file mode 100644 index 000000000..4a84cbbb9 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/preprocess_orcid_dump_parameter.json @@ -0,0 +1,21 @@ +[ + { + "paramName": "n", + "paramLongName": "namenode", + "paramDescription": "the Name Node URI", + "paramRequired": true + }, + { + "paramName": "t", + "paramLongName": "targetPath", + "paramDescription": "the target PATH where download the files", + "paramRequired": true + }, + { + "paramName": "a", + "paramLongName": "apiURL", + "paramDescription": "the FIGSHARE API id URL to retrieve all the dump files", + "paramRequired": true + } + +] \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/orcid/SparkGenerateORCIDTable.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/orcid/SparkGenerateORCIDTable.scala new file mode 100644 index 000000000..f0c4cd214 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/orcid/SparkGenerateORCIDTable.scala @@ -0,0 +1,101 @@ +package eu.dnetlib.dhp.collection.orcid + +import eu.dnetlib.dhp.application.AbstractScalaApplication +import eu.dnetlib.dhp.collection.orcid.model.{Author, Employment, Pid, Work} +import org.apache.hadoop.io.Text +import org.apache.spark.SparkContext +import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession} +import org.slf4j.{Logger, LoggerFactory} + +class SparkGenerateORCIDTable(propertyPath: String, args: Array[String], log: Logger) + extends AbstractScalaApplication(propertyPath, args, log: Logger) { + + /** Here all the spark applications runs this method + * where the whole logic of the spark node is defined + */ + override def run(): Unit = { + val sourcePath: String = parser.get("sourcePath") + log.info("found parameters sourcePath: {}", sourcePath) + val targetPath: String = parser.get("targetPath") + log.info("found parameters targetPath: {}", targetPath) + extractORCIDTable(spark, sourcePath, targetPath) + extractORCIDEmploymentsTable(spark, sourcePath, targetPath) + extractORCIDWorksTable(spark, sourcePath, targetPath) + } + + def extractORCIDTable(spark: SparkSession, sourcePath: String, targetPath: String): Unit = { + val sc: SparkContext = spark.sparkContext + import spark.implicits._ + val df = sc + .sequenceFile(sourcePath, classOf[Text], classOf[Text]) + .map { case (x, y) => (x.toString, y.toString) } + .toDF + .as[(String, String)] + implicit val orcidAuthor: Encoder[Author] = Encoders.bean(classOf[Author]) +// implicit val orcidPID:Encoder[Pid] = Encoders.bean(classOf[Pid]) + df.filter(r => r._1.contains("summaries")) + .map { r => + val p = new OrcidParser + p.parseSummary(r._2) + } + .filter(p => p != null) + .write + .mode(SaveMode.Overwrite) + .save(s"$targetPath/Authors") + } + + def extractORCIDWorksTable(spark: SparkSession, sourcePath: String, targetPath: String): Unit = { + val sc: SparkContext = spark.sparkContext + import spark.implicits._ + val df = sc + .sequenceFile(sourcePath, classOf[Text], classOf[Text]) + .map { case (x, y) => (x.toString, y.toString) } + .toDF + .as[(String, String)] + implicit val orcidWorkAuthor: Encoder[Work] = Encoders.bean(classOf[Work]) + implicit val orcidPID: Encoder[Pid] = Encoders.bean(classOf[Pid]) + df.filter(r => r._1.contains("works")) + .map { r => + val p = new OrcidParser + p.parseWork(r._2) + } + .filter(p => p != null) + .write + .mode(SaveMode.Overwrite) + .save(s"$targetPath/Works") + } + + def extractORCIDEmploymentsTable(spark: SparkSession, sourcePath: String, targetPath: String): Unit = { + val sc: SparkContext = spark.sparkContext + import spark.implicits._ + val df = sc + .sequenceFile(sourcePath, classOf[Text], classOf[Text]) + .map { case (x, y) => (x.toString, y.toString) } + .toDF + .as[(String, String)] + implicit val orcidEmploymentAuthor: Encoder[Employment] = Encoders.bean(classOf[Employment]) + implicit val orcidPID: Encoder[Pid] = Encoders.bean(classOf[Pid]) + df.filter(r => r._1.contains("employments")) + .map { r => + val p = new OrcidParser + p.parseEmployment(r._2) + } + .filter(p => p != null) + .write + .mode(SaveMode.Overwrite) + .save(s"$targetPath/Employments") + } +} + +object SparkGenerateORCIDTable { + + val log: Logger = LoggerFactory.getLogger(SparkGenerateORCIDTable.getClass) + + def main(args: Array[String]): Unit = { + + new SparkGenerateORCIDTable("/eu/dnetlib/dhp/collection/orcid/generate_orcid_table_parameter.json", args, log) + .initialize() + .run() + + } +} diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/orcid/DownloadORCIDTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/orcid/DownloadORCIDTest.java new file mode 100644 index 000000000..be5555fc0 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/orcid/DownloadORCIDTest.java @@ -0,0 +1,158 @@ + +package eu.dnetlib.dhp.collection.orcid; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; + +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.io.Text; +import org.apache.spark.SparkContext; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.ximpleware.NavException; +import com.ximpleware.ParseException; +import com.ximpleware.XPathEvalException; +import com.ximpleware.XPathParseException; + +import eu.dnetlib.dhp.collection.orcid.model.Author; +import eu.dnetlib.dhp.collection.orcid.model.ORCIDItem; +import eu.dnetlib.dhp.parser.utility.VtdException; + +public class DownloadORCIDTest { + private final Logger log = LoggerFactory.getLogger(DownloadORCIDTest.class); + +// public void test() throws Exception { +// +// Configuration conf = new Configuration(); +// // Set FileSystem URI +//// conf.set("fs.defaultFS", "file://"); +// // Because of Maven +// conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); +// conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); +// +// System.setProperty("hadoop.home.dir", "file:///Users/sandro/orcid/"); +// +// final FileSystem fileSystem = FileSystem.get(conf); +// +// new ExtractORCIDDump(fileSystem).run("/Users/sandro/orcid/", "/Users/sandro/orcid/extracted"); +// +//// final GZIPInputStream gzip = new GZIPInputStream(Files.newInputStream(Paths.get("/Users/sandro/orcid/ORCID_2023_10_activities_1.tar.gz"))); +//// try(final TarArchiveInputStream tais = new TarArchiveInputStream(gzip)) { +//// +//// TarArchiveEntry entry; +//// while ((entry = tais.getNextTarEntry()) != null) { +//// +//// if (entry.isFile() && entry.getName().contains("employments")) { +//// +//// System.out.println(entry.getName()); +//// final String [] items = entry.getName().split("/"); +//// +//// final String res = IOUtils.toString(new BufferedReader(new InputStreamReader(tais))); +//// System.out.println("res = " + res); +//// +//// System.out.println(items[items.length-2]); +//// break; +//// } +//// +//// +//// } +//// } +// +// } + + @Test + public void testSummary() throws Exception { + final String xml = IOUtils + .toString( + Objects.requireNonNull(getClass().getResourceAsStream("/eu/dnetlib/dhp/collection/orcid/summary.xml"))); + + final OrcidParser parser = new OrcidParser(); + ORCIDItem orcidItem = parser.parseSummary(xml); + + final ObjectMapper mapper = new ObjectMapper(); + System.out.println(mapper.writeValueAsString(orcidItem)); + + } + + @Test + public void testParsingWork() throws Exception { + + final List works_path = Arrays + .asList( + "/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0002-2536-4498.xml", + "/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0002-5982-8983.xml", + "/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0003-2760-1191.xml", + "/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0003-2760-1191-similarity.xml", + "/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0003-2760-1191_contributors.xml" + + ); + + final OrcidParser parser = new OrcidParser(); + final ObjectMapper mapper = new ObjectMapper(); + works_path.stream().map(s -> { + try { + return IOUtils + .toString( + Objects + .requireNonNull( + getClass() + .getResourceAsStream( + s))); + } catch (IOException e) { + throw new RuntimeException(e); + } + }).forEach(s -> { + try { + System.out.println(mapper.writeValueAsString(parser.parseWork(s))); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + + @Test + public void testParsingEmployments() throws Exception { + + final List works_path = Arrays + .asList( + "/eu/dnetlib/dhp/collection/orcid/employment.xml", + "/eu/dnetlib/dhp/collection/orcid/employment_2.xml", + "/eu/dnetlib/dhp/collection/orcid/employment_3.xml" + + ); + + final OrcidParser parser = new OrcidParser(); + final ObjectMapper mapper = new ObjectMapper(); + works_path.stream().map(s -> { + try { + return IOUtils + .toString( + Objects + .requireNonNull( + getClass() + .getResourceAsStream( + s))); + } catch (IOException e) { + throw new RuntimeException(e); + } + }).forEach(s -> { + try { + System.out.println(mapper.writeValueAsString(parser.parseEmployment(s))); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0002-2536-4498.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0002-2536-4498.xml new file mode 100644 index 000000000..2c89b83f6 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0002-2536-4498.xml @@ -0,0 +1,69 @@ + + + 2016-09-01T19:22:46.768Z + 2022-05-25T03:48:56.968Z + + + https://orcid.org/client/0000-0002-5982-8983 + 0000-0002-5982-8983 + orcid.org + + Scopus - Elsevier + + https://orcid.org/0000-0001-5010-5001 + 0000-0001-5010-5001 + orcid.org + + Quang Nguyen + + + Vision outcomes and major complications after endovascular coil embolization of ophthalmic segment aneurysms + + American Journal of Neuroradiology + + bibtex + @article{Nguyen2014,title = {Vision outcomes and major complications after endovascular coil embolization of ophthalmic segment aneurysms},journal = {American Journal of Neuroradiology},year = {2014},volume = {35},number = {11},pages = {2140-2145},author = {Durst, C. and Starke, R.M. and Gaughen, J. and Nguyen, Q. and Patrie, J. and Jensen, M.E. and Evans, A.J.}} + + journal-article + + 2014 + + + + doi + 10.3174/ajnr.A4032 + 10.3174/ajnr.a4032 + self + + + eid + 2-s2.0-84911865199 + 2-s2.0-84911865199 + self + + + http://www.scopus.com/inward/record.url?eid=2-s2.0-84911865199&partnerID=MN8TOARS + + + Durst, C. + + + Starke, R.M. + + + Gaughen, J. + + + Nguyen, Q. + + + Patrie, J. + + + Jensen, M.E. + + + Evans, A.J. + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0002-5982-8983.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0002-5982-8983.xml new file mode 100644 index 000000000..63b4405f1 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0002-5982-8983.xml @@ -0,0 +1,79 @@ + + + 2018-11-01T19:49:45.562Z + 2018-11-01T19:49:45.562Z + + + https://orcid.org/client/0000-0002-5982-8983 + 0000-0002-5982-8983 + orcid.org + + Scopus - Elsevier + + + "Calling Out" in class: Degrees of candor in addressing social injustices in + racially homogenous and heterogeneous U.S. history classrooms + + Journal of Social Studies Research + + bibtex + @article{Massaro2018,title = {{"}Calling Out{"} in class: Degrees of + candor in addressing social injustices in racially homogenous and heterogeneous U.S. + history classrooms},journal = {Journal of Social Studies Research},year = {2018},author + = {Parkhouse, H. and Massaro, V.R.}} + + journal-article + + 2018 + + + + doi + 10.1016/j.jssr.2018.01.004 + 10.1016/j.jssr.2018.01.004 + self + + + eid + 2-s2.0-85041949043 + 2-s2.0-85041949043 + self + + + http://www.scopus.com/inward/record.url?eid=2-s2.0-85041949043&partnerID=MN8TOARS + + + Parkhouse, H. + + + Massaro, V.R. + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0003-2760-1191-similarity.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0003-2760-1191-similarity.xml new file mode 100644 index 000000000..650d5a4cb --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0003-2760-1191-similarity.xml @@ -0,0 +1,113 @@ + + + 2016-12-12T23:02:05.233Z + 2016-12-13T09:08:16.412Z + + + https://orcid.org/0000-0002-9157-3431 + 0000-0002-9157-3431 + orcid.org + + Europe PubMed Central + + + Cutoff Value of Admission N-Terminal Pro-Brain Natriuretic Peptide Which + Predicts Poor Myocardial Perfusion after Primary Percutaneous Coronary Intervention for + ST-Segment-Elevation Myocardial Infarction. + + + formatted-unspecified + Abdel-Dayem K, Eweda II, El-Sherbiny A, Dimitry MO, Nammas W, Acta + Cardiologica Sinica, 2016, vol. 32, no. 6, pp. 649-655, 2016 + + journal-article + + 2016 + 11 + + + + pmid + 27899851 + 27899851 + self + + + pmc + PMC5126442 + PMC5126442 + self + + + http://europepmc.org/abstract/med/27899851 + + + Abdel-Dayem K + + first + author + + + + Abdel-Dayem Fake + + first + author + + + + Eweda II + + first + author + + + + El-Sherbiny A + + first + author + + + + Dimitry MO + + first + author + + + + Nammas W + + first + author + + + + diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0003-2760-1191.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0003-2760-1191.xml new file mode 100644 index 000000000..83752b145 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0003-2760-1191.xml @@ -0,0 +1,106 @@ + + + 2016-12-12T23:02:05.233Z + 2016-12-13T09:08:16.412Z + + + https://orcid.org/0000-0002-9157-3431 + 0000-0002-9157-3431 + orcid.org + + Europe PubMed Central + + + Cutoff Value of Admission N-Terminal Pro-Brain Natriuretic Peptide Which + Predicts Poor Myocardial Perfusion after Primary Percutaneous Coronary Intervention for + ST-Segment-Elevation Myocardial Infarction. + + + formatted-unspecified + Abdel-Dayem K, Eweda II, El-Sherbiny A, Dimitry MO, Nammas W, Acta + Cardiologica Sinica, 2016, vol. 32, no. 6, pp. 649-655, 2016 + + journal-article + + 2016 + 11 + + + + pmid + 27899851 + 27899851 + self + + + pmc + PMC5126442 + PMC5126442 + self + + + http://europepmc.org/abstract/med/27899851 + + + Khair Abde Daye + + first + author + + + + Eweda II + + first + author + + + + El-Sherbiny A + + first + author + + + + Dimitry MO + + first + author + + + + Nammas W + + first + author + + + + diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0003-2760-1191_contributors.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0003-2760-1191_contributors.xml new file mode 100644 index 000000000..26e64aeda --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/activity_work_0000-0003-2760-1191_contributors.xml @@ -0,0 +1,101 @@ + + + 2016-12-12T23:02:05.233Z + 2016-12-13T09:08:16.412Z + + + https://orcid.org/0000-0002-9157-3431 + 0000-0002-9157-3431 + orcid.org + + Europe PubMed Central + + + Cutoff Value of Admission N-Terminal Pro-Brain Natriuretic Peptide Which + Predicts Poor Myocardial Perfusion after Primary Percutaneous Coronary Intervention for + ST-Segment-Elevation Myocardial Infarction. + + + formatted-unspecified + Abdel-Dayem K, Eweda II, El-Sherbiny A, Dimitry MO, Nammas W, Acta + Cardiologica Sinica, 2016, vol. 32, no. 6, pp. 649-655, 2016 + + journal-article + + 2016 + 11 + + + + pmid + 27899851 + 27899851 + self + + + pmc + PMC5126442 + PMC5126442 + self + + + http://europepmc.org/abstract/med/27899851 + + + + seq0 + role0 + + + + creditname1 + + + creditname2 + + seq2 + + + + + creditname3 + + + role3 + + + + + + seq4 + role4 + + + + diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/employment.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/employment.xml new file mode 100644 index 000000000..89c7f7020 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/employment.xml @@ -0,0 +1,50 @@ + + + 2016-09-01T19:21:05.791Z + 2016-09-01T19:21:05.791Z + + + https://orcid.org/0000-0001-5010-5001 + 0000-0001-5010-5001 + orcid.org + + Quang Nguyen + + + Beth Israel Deaconess Medical Center + + Boston + MA + US + + + 1859 + RINGGOLD + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/employment_2.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/employment_2.xml new file mode 100644 index 000000000..c0e88e236 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/employment_2.xml @@ -0,0 +1,55 @@ + + + 2018-09-03T01:46:19.474Z + 2018-09-03T01:46:19.474Z + + + https://orcid.org/0000-0001-5011-3001 + 0000-0001-5011-3001 + orcid.org + + zhengyan li + + + 2008 + 09 + 01 + + + Anhui Academy of Agricultural Sciences + + Hefei + Anhui + CN + + + 125385 + RINGGOLD + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/employment_3.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/employment_3.xml new file mode 100644 index 000000000..8e7857fb8 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/employment_3.xml @@ -0,0 +1,62 @@ + + + 2021-03-11T14:48:29.603Z + 2021-03-11T14:48:29.603Z + + + https://orcid.org/0000-0001-5012-1001 + 0000-0001-5012-1001 + orcid.org + + Asma Bazzi + + Pathology and Laboratory Medicine + Medical Laboratory Technologist + + 1994 + 10 + 01 + + + 2000 + 06 + 30 + + + American University of Beirut + + Hamra + Beirut + LB + + + 11238 + RINGGOLD + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/summary.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/summary.xml new file mode 100644 index 000000000..1f5a1bff1 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/summary.xml @@ -0,0 +1,581 @@ + + + + https://orcid.org/0000-0001-5045-1000 + 0000-0001-5045-1000 + orcid.org + + + es + + + Direct + 2023-01-17T23:50:40.215Z + 2023-09-04T17:51:57.749Z + true + true + true + + + + 2023-01-17T23:50:40.472Z + 2023-01-17T23:50:40.472Z + Patricio + Sánchez Quinchuela + + + + 2023-01-19T13:47:33.653Z + 2023-01-19T13:47:33.653Z + Especialista de vinculación con la sociedad y docente de la Universidad de las Artes. Magister en Economía Social y Solidaria por el IAEN; Magister en Proyectos Sociales y Productivos por la UNACH. Licenciado en Artes UCE. Licenciado en Castellano y Literatura por la UNACH. Doctorando del programa de Sociología de la UNED España. Larga trayectoria vinculado a las organizaciones sociales acompañando procesos de gestión cultural, formación de liderazgos y economía solidaria. + + + + + + + 2018-02-05T23:27:36.636Z + + 2013-03-08T03:20:39.347Z + 2018-02-05T23:27:36.636Z + + + https://orcid.org/client/0000-0002-5982-8983 + 0000-0002-5982-8983 + orcid.org + + Scopus - Elsevier + + https://orcid.org/0000-0001-7291-3210 + 0000-0001-7291-3210 + orcid.org + + Paolo Manghi + + Scopus Author ID + 6602255248 + http://www.scopus.com/inward/authorDetails.url?authorID=6602255248&partnerID=MN8TOARS + self + + + + + 2023-09-04T17:51:57.749Z + + 2023-01-19T13:49:48.482Z + + 2023-01-19T13:49:48.482Z + + + 2023-01-19T13:49:48.482Z + 2023-01-19T13:49:48.482Z + + + https://orcid.org/0000-0001-5045-1000 + 0000-0001-5045-1000 + orcid.org + + Patricio Sánchez Quinchuela + + Programa de Maestría + Becario del programa de Maestría en Economía Social y Solidaria + + 2014 + 10 + 20 + + + Instituto de Altos Estudios Nacionales + + Quito + Pichincha + EC + + + https://ror.org/011g3me54 + ROR + + + + + + + 2023-01-18T21:41:03.175Z + + 2023-01-18T21:41:03.175Z + + + 2023-01-18T21:41:03.175Z + 2023-01-18T21:41:03.175Z + + + https://orcid.org/0000-0001-5045-1000 + 0000-0001-5045-1000 + orcid.org + + Patricio Sánchez Quinchuela + + Programa de Doctorado en Sociología + Doctorando del Programa de Sociología + + 2020 + 11 + 06 + + + Universidad Nacional de Educación a Distancia Facultad de Ciencias Políticas y Sociología + + Madrid + Comunidad de Madrid + ES + + + 223339 + RINGGOLD + + + + + + + 2023-01-18T21:25:07.138Z + + 2023-01-18T21:22:21.513Z + + + 2023-01-17T23:57:08.246Z + 2023-01-18T21:22:21.513Z + + + https://orcid.org/0000-0001-5045-1000 + 0000-0001-5045-1000 + orcid.org + + Patricio Sánchez Quinchuela + + Dirección de Vinculación con la Sociedad + Especialista de Proyectos y docente + + 2021 + 11 + 01 + + + Universidad de las Artes + + Guayaquil + Guayas + EC + + + https://ror.org/016drwn73 + ROR + + + + + + 2023-01-18T21:25:07.138Z + + + 2023-01-18T21:25:07.138Z + 2023-01-18T21:25:07.138Z + + + https://orcid.org/0000-0001-5045-1000 + 0000-0001-5045-1000 + orcid.org + + Patricio Sánchez Quinchuela + + Dirección de Vinculación con la Sociedad + Director + + 2019 + 11 + 05 + + + 2021 + 10 + 31 + + + Universidad Regional Amazónica IKIAM + + Tena + Napo + EC + + + https://ror.org/05xedqd83 + ROR + + + http://ikiam.edu.ec + + + + + + + 2023-03-24T18:16:09.131Z + + 2023-03-24T18:16:09.131Z + + + 2023-03-24T18:16:09.131Z + 2023-03-24T18:16:09.131Z + + + https://orcid.org/0000-0001-5045-1000 + 0000-0001-5045-1000 + orcid.org + + Patricio Sánchez Quinchuela + + Artes Escénicas + Miembro + + 2000 + 07 + 15 + + + Casa de la Cultura Ecuatoriana + + Riobamba + Sierra Centro + EC + + + + + + + + 2023-01-18T21:45:07.379Z + + 2023-01-18T21:29:11.300Z + + + 2023-01-18T21:29:11.300Z + 2023-01-18T21:29:11.300Z + + + https://orcid.org/0000-0001-5045-1000 + 0000-0001-5045-1000 + orcid.org + + Patricio Sánchez Quinchuela + + Programa de Gobernabilidad + Magister en Economïa Social y Solidaria + + 2014 + 10 + 20 + + + 2017 + 01 + 26 + + + Instituto de Altos Estudios Nacionales + + Quito + Pichincha + EC + + + https://ror.org/011g3me54 + ROR + + + + + + 2023-01-18T21:34:32.093Z + + + 2023-01-18T21:34:32.093Z + 2023-01-18T21:34:32.093Z + + + https://orcid.org/0000-0001-5045-1000 + 0000-0001-5045-1000 + orcid.org + + Patricio Sánchez Quinchuela + + Posgrados + Magister en Proyectos Sociales y Productivos + + 2001 + 03 + 09 + + + 2003 + 02 + 27 + + + Universidad Nacional de Chimborazo + + Riobamba + Chimborazo + EC + + + https://ror.org/059wmd288 + ROR + + + + + + 2023-01-18T21:45:07.379Z + + + 2023-01-18T21:45:07.379Z + 2023-01-18T21:45:07.379Z + + + https://orcid.org/0000-0001-5045-1000 + 0000-0001-5045-1000 + orcid.org + + Patricio Sánchez Quinchuela + + Ciencias de la Educación + Licenciado en Ciencias de la Educación en Castellano y Literatura + + 1994 + 10 + 03 + + + 2000 + 01 + 31 + + + Universidad Nacional de Chimborazo + + Riobamba + Chimborazo + EC + + + https://ror.org/059wmd288 + ROR + + + + + + 2023-01-18T21:37:42.186Z + + + 2023-01-18T21:37:42.186Z + 2023-01-18T21:37:42.186Z + + + https://orcid.org/0000-0001-5045-1000 + 0000-0001-5045-1000 + orcid.org + + Patricio Sánchez Quinchuela + + Facultad de Artes + Licenciado en Artes + + 1989 + 09 + 05 + + + 1997 + 08 + 07 + + + Universidad Central del Ecuador + + Quito + Pichincha + EC + + + http://dx.doi.org/10.13039/100019134 + FUNDREF + + + + + + + + + 2023-09-04T17:51:57.749Z + + 2023-06-09T22:15:12.910Z + + + 2023-03-24T18:36:56.180Z + 2023-06-09T22:15:12.910Z + + + https://orcid.org/0000-0001-5045-1000 + 0000-0001-5045-1000 + orcid.org + + Patricio Sánchez Quinchuela + + + Experience in a non-capitalist way: solidarity funds that do not tax interest on the use of money + + + + isbn + 978-9942-29-089-2 + 9789942290892 + part-of + + + book-chapter + + 2023 + 06 + 07 + + Finanzas éticas y solidarias en América Latina: diagnósticos, debates y propuestas + + + + 2023-03-24T19:05:36.384Z + + + 2023-03-24T19:05:36.384Z + 2023-03-24T19:05:36.384Z + + + https://orcid.org/0000-0001-5045-1000 + 0000-0001-5045-1000 + orcid.org + + Patricio Sánchez Quinchuela + + + Incidence of artistic practices in the social transformation of the territory. study of case: Hilarte Association, Guayaquil-Ecuador + + + conference-abstract + + 2022 + 10 + 06 + + + + + 2023-09-04T17:40:30.215Z + + + other-id + 2018 + 2018 + self + + + + 2023-09-04T17:40:30.215Z + 2023-09-04T17:40:30.215Z + + + https://orcid.org/0000-0001-5045-1000 + 0000-0001-5045-1000 + orcid.org + + Patricio Sánchez Quinchuela + + + Más allá de la transferencia de conocimientos, un espacio para el interaprendizaje y el diálogo de saberes + + + + other-id + 2018 + 2018 + self + + + https://drive.google.com/drive/folders/1Tclz6isxGzSjTq-hfTnxe6M1nux-88wF?usp=drive_link + conference-poster + + 2018 + 11 + 30 + + Más allá de la transferencia de conocimientos, un espacio para el interaprendizaje y el diálogo de saberes + + + + 2023-03-24T18:57:10.095Z + + + 2023-03-24T18:57:10.095Z + 2023-03-24T18:57:10.095Z + + + https://orcid.org/0000-0001-5045-1000 + 0000-0001-5045-1000 + orcid.org + + Patricio Sánchez Quinchuela + + + Promotion of the popular and solidarity economy from the state: principles and challenges in the experience of Ecuador + + + dissertation-thesis + + 2017 + 01 + 26 + + + + + 2023-09-04T17:51:57.749Z + + + 2023-09-04T17:51:57.749Z + 2023-09-04T17:51:57.749Z + + + https://orcid.org/0000-0001-5045-1000 + 0000-0001-5045-1000 + orcid.org + + Patricio Sánchez Quinchuela + + + La Rebelión de los Dioses + + + https://drive.google.com/drive/folders/1Tclz6isxGzSjTq-hfTnxe6M1nux-88wF?usp=drive_link + registered-copyright + + 2001 + 08 + 28 + + Editorial pedagógica freire + + + + + \ No newline at end of file From eaf0a702dea01c77a01bfa38acb4c9b1b6c72627 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Tue, 14 Nov 2023 14:53:34 +0100 Subject: [PATCH 21/60] - --- .../eu/dnetlib/dhp/api/QueryCommunityAPI.java | 50 +++++++++---------- .../main/java/eu/dnetlib/dhp/api/Utils.java | 25 +++++----- .../dnetlib/dhp/bulktag/SparkBulkTagJob.java | 6 +-- .../PrepareResultCommunitySet.java | 6 +-- .../PrepareResultCommunitySet.java | 7 ++- 5 files changed, 47 insertions(+), 47 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java index 262ca0290..dc10fce15 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java @@ -14,8 +14,8 @@ import org.jetbrains.annotations.NotNull; * @Date 06/10/23 */ public class QueryCommunityAPI { - private static final String PRODUCTION_BASE_URL = "https://services.openaire.eu/openaire/"; - private static final String BETA_BASE_URL = "https://beta.services.openaire.eu/openaire/"; + + private static String get(String geturl) throws IOException { URL url = new URL(geturl); @@ -32,35 +32,35 @@ public class QueryCommunityAPI { return body; } - public static String communities(boolean production) throws IOException { - if (production) - return get(PRODUCTION_BASE_URL + "community/communities"); - return get(BETA_BASE_URL + "community/communities"); - } + public static String communities(String baseURL) throws IOException { - public static String community(String id, boolean production) throws IOException { - if (production) - return get(PRODUCTION_BASE_URL + "community/" + id); - return get(BETA_BASE_URL + "community/" + id); - } - - public static String communityDatasource(String id, boolean production) throws IOException { - if (production) - return get(PRODUCTION_BASE_URL + "community/" + id + "/contentproviders"); - return (BETA_BASE_URL + "community/" + id + "/contentproviders"); + return get(baseURL + "community/communities"); } - public static String communityPropagationOrganization(String id, boolean production) throws IOException { - if (production) - return get(PRODUCTION_BASE_URL + "community/" + id + "/propagationOrganizations"); - return get(BETA_BASE_URL + "community/" + id + "/propagationOrganizations"); + public static String community(String id, String baseURL ) throws IOException { + + return get(baseURL + "community/" + id); + } - public static String communityProjects(String id, String page, String size, boolean production) throws IOException { - if (production) - return get(PRODUCTION_BASE_URL + "community/" + id + "/projects/" + page + "/" + size); - return get(BETA_BASE_URL + "community/" + id + "/projects/" + page + "/" + size); + public static String communityDatasource(String id, String baseURL ) throws IOException { + + return get(baseURL + "community/" + id + "/contentproviders"); + + + } + + public static String communityPropagationOrganization(String id, String baseURL ) throws IOException { + + return get(baseURL + "community/" + id + "/propagationOrganizations"); + + } + + public static String communityProjects(String id, String page, String size, String baseURL ) throws IOException { + + return get(baseURL + "community/" + id + "/projects/" + page + "/" + size); + } @NotNull diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java index 43d5e7e98..d121b8b7e 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/Utils.java @@ -36,14 +36,14 @@ public class Utils implements Serializable { private static final Logger log = LoggerFactory.getLogger(Utils.class); - public static CommunityConfiguration getCommunityConfiguration(boolean production) throws IOException { + public static CommunityConfiguration getCommunityConfiguration(String baseURL) throws IOException { final Map communities = Maps.newHashMap(); List validCommunities = new ArrayList<>(); - getValidCommunities(production) + getValidCommunities(baseURL) .forEach(community -> { try { CommunityModel cm = MAPPER - .readValue(QueryCommunityAPI.community(community.getId(), production), CommunityModel.class); + .readValue(QueryCommunityAPI.community(community.getId(), baseURL), CommunityModel.class); validCommunities.add(getCommunity(cm)); } catch (IOException e) { throw new RuntimeException(e); @@ -53,7 +53,7 @@ public class Utils implements Serializable { try { DatasourceList dl = MAPPER .readValue( - QueryCommunityAPI.communityDatasource(community.getId(), production), DatasourceList.class); + QueryCommunityAPI.communityDatasource(community.getId(), baseURL), DatasourceList.class); community.setProviders(dl.stream().map(d -> { if (d.getEnabled() == null || Boolean.FALSE.equals(d.getEnabled())) return null; @@ -98,9 +98,9 @@ public class Utils implements Serializable { return c; } - public static List getValidCommunities(boolean production) throws IOException { + public static List getValidCommunities(String baseURL) throws IOException { return MAPPER - .readValue(QueryCommunityAPI.communities(production), CommunitySummary.class) + .readValue(QueryCommunityAPI.communities(baseURL), CommunitySummary.class) .stream() .filter( community -> !community.getStatus().equals("hidden") && @@ -111,15 +111,15 @@ public class Utils implements Serializable { /** * it returns for each organization the list of associated communities */ - public static CommunityEntityMap getCommunityOrganization(boolean production) throws IOException { + public static CommunityEntityMap getCommunityOrganization(String baseURL) throws IOException { CommunityEntityMap organizationMap = new CommunityEntityMap(); - getValidCommunities(production) + getValidCommunities(baseURL) .forEach(community -> { String id = community.getId(); try { List associatedOrgs = MAPPER .readValue( - QueryCommunityAPI.communityPropagationOrganization(id, production), OrganizationList.class); + QueryCommunityAPI.communityPropagationOrganization(id, baseURL), OrganizationList.class); associatedOrgs.forEach(o -> { if (!organizationMap .keySet() @@ -136,9 +136,10 @@ public class Utils implements Serializable { return organizationMap; } - public static CommunityEntityMap getCommunityProjects(boolean production) throws IOException { + public static CommunityEntityMap getCommunityProjects(String baseURL) throws IOException { CommunityEntityMap projectMap = new CommunityEntityMap(); - getValidCommunities(production) + + getValidCommunities(baseURL) .forEach(community -> { int page = -1; int size = 100; @@ -150,7 +151,7 @@ public class Utils implements Serializable { .readValue( QueryCommunityAPI .communityProjects( - community.getId(), String.valueOf(page), String.valueOf(size), production), + community.getId(), String.valueOf(page), String.valueOf(size), baseURL), ContentModel.class); if (cm.getContent().size() > 0) { cm.getContent().forEach(p -> { diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java index 15712ad66..d972cecac 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java @@ -62,8 +62,8 @@ public class SparkBulkTagJob { final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); - final boolean production = Boolean.valueOf(parser.get("production")); - log.info("production: {}", production); + final String baseURL = parser.get("baseURL"); + log.info("baseURL: {}", baseURL); ProtoMap protoMappingParams = new Gson().fromJson(parser.get("pathMap"), ProtoMap.class); log.info("pathMap: {}", new Gson().toJson(protoMappingParams)); @@ -79,7 +79,7 @@ public class SparkBulkTagJob { if (taggingConf != null) { cc = CommunityConfigurationFactory.newInstance(taggingConf); } else { - cc = Utils.getCommunityConfiguration(production); + cc = Utils.getCommunityConfiguration(baseURL); } runWithSparkSession( diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java index e32e94a4b..54fa60168 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromorganization/PrepareResultCommunitySet.java @@ -48,10 +48,10 @@ public class PrepareResultCommunitySet { final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); - final boolean production = Boolean.valueOf(parser.get("production")); - log.info("production: {}", production); + final String baseURL = parser.get("baseURL"); + log.info("baseURL: {}", baseURL); - final CommunityEntityMap organizationMap = Utils.getCommunityOrganization(production); + final CommunityEntityMap organizationMap = Utils.getCommunityOrganization(baseURL); log.info("organizationMap: {}", new Gson().toJson(organizationMap)); SparkConf conf = new SparkConf(); diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java index 7bee1ea0c..2c2eb69dd 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java @@ -52,11 +52,10 @@ public class PrepareResultCommunitySet { final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); - final boolean production = Boolean.valueOf(parser.get("production")); - log.info("production: {}", production); + final String baseURL = parser.get("baseURL"); + log.info("baseUEL: {}", baseURL); - final CommunityEntityMap projectsMap = Utils.getCommunityProjects(production); - // log.info("projectsMap: {}", new Gson().toJson(projectsMap)); + final CommunityEntityMap projectsMap = Utils.getCommunityProjects(baseURL); SparkConf conf = new SparkConf(); From a94a54a2d0a9bdd9e6178f7f63b5369bd639d440 Mon Sep 17 00:00:00 2001 From: dimitrispie Date: Wed, 15 Nov 2023 14:32:18 +0200 Subject: [PATCH 22/60] Changes for tables and creation of the new indicator indi_is_result_accessible - Drop table statements for all tables to avoid duplicates in case of wf rerun - Add pdfsaggregated step to create the indi_is_result_accessible table. This step is executed on the new impala cluster only, since the pdfaggregation_i is updated on this cluster. --- .../oozie_app/copyDataToImpalaCluster.sh | 1 - .../stats/oozie_app/createPDFsAggregated.sh | 42 +++++ .../graph/stats/oozie_app/scripts/step10.sql | 2 + .../graph/stats/oozie_app/scripts/step11.sql | 1 + .../graph/stats/oozie_app/scripts/step12.sql | 10 ++ .../graph/stats/oozie_app/scripts/step13.sql | 17 +- .../graph/stats/oozie_app/scripts/step14.sql | 16 +- .../graph/stats/oozie_app/scripts/step15.sql | 12 ++ .../stats/oozie_app/scripts/step15_5.sql | 10 ++ .../scripts/step16-createIndicatorsTables.sql | 18 +- .../scripts/step16_1-definitions.sql | 6 + .../stats/oozie_app/scripts/step16_5.sql | 1 + .../graph/stats/oozie_app/scripts/step2.sql | 17 ++ .../scripts/step20-createMonitorDB.sql | 155 +++--------------- .../scripts/step20-createMonitorDBAll.sql | 85 +++------- .../scripts/step20-createMonitorDB_RIs.sql | 3 +- .../step20-createMonitorDB_RIs_tail.sql | 3 +- .../scripts/step20-createMonitorDB_funded.sql | 4 +- .../step20-createMonitorDB_institutions.sql | 4 +- .../scripts/step21-createObservatoryDB.sql | 33 ---- .../graph/stats/oozie_app/scripts/step7.sql | 6 + .../graph/stats/oozie_app/scripts/step8.sql | 11 ++ .../graph/stats/oozie_app/scripts/step9.sql | 2 + .../dhp/oa/graph/stats/oozie_app/workflow.xml | 17 ++ 24 files changed, 234 insertions(+), 242 deletions(-) create mode 100644 dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/createPDFsAggregated.sh diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh index 431978997..18ff6dca8 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/copyDataToImpalaCluster.sh @@ -6,7 +6,6 @@ then ln -sfn ${PYTHON_EGG_CACHE}${link_folder} ${link_folder} fi -#export HADOOP_USER_NAME="dimitris.pierrakos" export HADOOP_USER_NAME=$6 export PROD_USAGE_STATS_DB="openaire_prod_usage_stats" function copydb() { diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/createPDFsAggregated.sh b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/createPDFsAggregated.sh new file mode 100644 index 000000000..46631a0c2 --- /dev/null +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/createPDFsAggregated.sh @@ -0,0 +1,42 @@ +export PYTHON_EGG_CACHE=/home/$(whoami)/.python-eggs +export link_folder=/tmp/impala-shell-python-egg-cache-$(whoami) +if ! [ -L $link_folder ] +then + rm -Rf "$link_folder" + ln -sfn ${PYTHON_EGG_CACHE}${link_folder} ${link_folder} +fi + +function createPDFsAggregated() { + db=$1 + +impala-shell --user $HADOOP_USER_NAME -i impala-cluster-dn1.openaire.eu -d ${db} -q "drop table if exists indi_is_result_accessible"; + +impala-shell --user $HADOOP_USER_NAME -i impala-cluster-dn1.openaire.eu -d ${db} -q "create table indi_is_result_accessible stored as parquet as + select distinct p.id, coalesce(is_result_accessible, 0) as is_result_accessible from result p + left outer join + (select id, 1 as is_result_accessible from (select pl.* from result r + join pdfaggregation_i.publication p on r.id=p.id + join pdfaggregation_i.payload pl on pl.id=p.id + union all + select pl.* from result r + join pdfaggregation_i.publication p on r.id=p.dedupid + join pdfaggregation_i.payload pl on pl.id=p.id) foo) tmp on p.id=tmp.id"; +} + +STATS_DB=$1 +MONITOR_DB=$2 +HADOOP_USER_NAME=$3 + +createPDFsAggregated $STATS_DB +createPDFsAggregated $MONITOR_DB + +createPDFsAggregated $MONITOR_DB'_funded' +createPDFsAggregated $MONITOR_DB'_institutions' +createPDFsAggregated $MONITOR_DB'_ris_tail' + +contexts="knowmad::other dh-ch::other enermaps::other gotriple::other neanias-atmospheric::other rural-digital-europe::other covid-19::other aurora::other neanias-space::other north-america-studies::other north-american-studies::other eutopia::other" +for i in ${contexts} +do + tmp=`echo "$i" | sed 's/'-'/'_'/g' | sed 's/'::'/'_'/g'` + createPDFsAggregated ${MONITOR_DB}'_'${tmp} +done \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql index 92dedf243..bbd7b3bbc 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step10.sql @@ -49,5 +49,7 @@ select * from openaire_prod_usage_stats.views_stats; -- Creation date of the database ------------------------------------------------------------------------------------------------ ------------------------------------------------------------------------------------------------ +DROP TABLE IF EXISTS ${stats_db_name}.creation_date purge; + create table ${stats_db_name}.creation_date STORED AS PARQUET as select date_format(current_date(), 'dd-MM-yyyy') as date; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql index 41c3ed751..638fb0f7a 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step11.sql @@ -20,6 +20,7 @@ WHERE project_tmp.id IN (SELECT pr.id ${stats_db_name}.result r WHERE pr.result = r.id AND r.type = 'publication'); +DROP TABLE IF EXISTS ${stats_db_name}.stored purge; CREATE TABLE ${stats_db_name}.project stored as parquet as SELECT p.id, diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql index 47d147f75..0a1904de7 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step12.sql @@ -1,22 +1,32 @@ ------------------------------------------------------------------------------------------------------ -- Creating parquet tables from the updated temporary tables and removing unnecessary temporary tables ------------------------------------------------------------------------------------------------------ +DROP TABLE IF EXISTS ${stats_db_name}.datasource purge; + CREATE TABLE ${stats_db_name}.datasource stored AS parquet AS SELECT * FROM ${stats_db_name}.datasource_tmp; +DROP TABLE IF EXISTS ${stats_db_name}.publication purge; + CREATE TABLE ${stats_db_name}.publication stored AS parquet AS SELECT * FROM ${stats_db_name}.publication_tmp; +DROP TABLE IF EXISTS ${stats_db_name}.dataset purge; + CREATE TABLE ${stats_db_name}.dataset stored AS parquet AS SELECT * FROM ${stats_db_name}.dataset_tmp; +DROP TABLE IF EXISTS ${stats_db_name}.software purge; + CREATE TABLE ${stats_db_name}.software stored AS parquet AS SELECT * FROM ${stats_db_name}.software_tmp; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct purge; + CREATE TABLE ${stats_db_name}.otherresearchproduct stored AS parquet AS SELECT * FROM ${stats_db_name}.otherresearchproduct_tmp; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql index 24e1a1355..6493fa7d0 100755 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step13.sql @@ -5,6 +5,8 @@ -- Sources related tables/views ------------------------------------------------------ ------------------------------------------------------ +DROP TABLE IF EXISTS ${stats_db_name}.publication_sources purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_sources STORED AS PARQUET as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM ( @@ -16,6 +18,8 @@ LEFT OUTER JOIN from ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on p.datasource = d.id; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_sources purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_sources STORED AS PARQUET as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM ( @@ -27,6 +31,8 @@ LEFT OUTER JOIN from ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on p.datasource = d.id; +DROP TABLE IF EXISTS ${stats_db_name}.software_sources purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_sources STORED AS PARQUET as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM ( @@ -38,6 +44,8 @@ LEFT OUTER JOIN from ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on p.datasource = d.id; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_sources purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_sources STORED AS PARQUET as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM ( @@ -48,7 +56,7 @@ LEFT OUTER JOIN SELECT substr(d.id, 4) id from ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on p.datasource = d.id; - + CREATE VIEW IF NOT EXISTS ${stats_db_name}.result_sources AS SELECT * FROM ${stats_db_name}.publication_sources UNION ALL @@ -58,6 +66,7 @@ SELECT * FROM ${stats_db_name}.software_sources UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_sources; +DROP TABLE IF EXISTS ${stats_db_name}.result_orcid purge; CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_orcid STORED AS PARQUET as select distinct res.id, regexp_replace(res.orcid, 'http://orcid.org/' ,'') as orcid @@ -69,6 +78,8 @@ from ( LATERAL VIEW explode(auth.pid.qualifier.classid) apt as author_pid_type WHERE res.datainfo.deletedbyinference = FALSE and res.datainfo.invisible = FALSE and author_pid_type = 'orcid') as res; +DROP TABLE IF EXISTS ${stats_db_name}.result_result purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_result stored as parquet as select substr(rel.source, 4) as source, substr(rel.target, 4) as target, relclass, subreltype from ${openaire_db_name}.relation rel @@ -82,6 +93,8 @@ where reltype='resultResult' and r2.resulttype.classname != 'other' and rel.datainfo.deletedbyinference=false and rel.datainfo.invisible = FALSE; +DROP TABLE IF EXISTS ${stats_db_name}.result_citations_oc purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_citations_oc stored as parquet as select substr(target, 4) as id, count(distinct substr(source, 4)) as citations from ${openaire_db_name}.relation rel @@ -97,6 +110,8 @@ where relClass='Cites' and rel.datainfo.provenanceaction.classid = 'sysimport:cr and rel.datainfo.deletedbyinference=false and rel.datainfo.invisible = FALSE group by substr(target, 4); +DROP TABLE IF EXISTS ${stats_db_name}.result_references_oc purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_references_oc stored as parquet as select substr(source, 4) as id, count(distinct substr(target, 4)) as references from ${openaire_db_name}.relation rel diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql index 39755d68e..f50c13521 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step14.sql @@ -5,21 +5,29 @@ -- Licences related tables/views ------------------------------------------------------ ------------------------------------------------------ +DROP TABLE IF EXISTS ${stats_db_name}.publication_licenses purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_licenses STORED AS PARQUET AS -SELECT substr(p.id, 4) as id, licenses.value as type +SELECT substr(p.id, 4) as id, licenses.value as type from ${openaire_db_name}.publication p LATERAL VIEW explode(p.instance.license) instances as licenses where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false and p.datainfo.invisible = FALSE; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_licenses purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_licenses STORED AS PARQUET AS SELECT substr(p.id, 4) as id, licenses.value as type from ${openaire_db_name}.dataset p LATERAL VIEW explode(p.instance.license) instances as licenses where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false and p.datainfo.invisible = FALSE; +DROP TABLE IF EXISTS ${stats_db_name}.software_licenses purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_licenses STORED AS PARQUET AS SELECT substr(p.id, 4) as id, licenses.value as type from ${openaire_db_name}.software p LATERAL VIEW explode(p.instance.license) instances as licenses where licenses.value is not null and licenses.value != '' and p.datainfo.deletedbyinference=false and p.datainfo.invisible = FALSE; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_licenses purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_licenses STORED AS PARQUET AS SELECT substr(p.id, 4) as id, licenses.value as type from ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.instance.license) instances as licenses @@ -34,10 +42,14 @@ SELECT * FROM ${stats_db_name}.software_licenses UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_licenses; +DROP TABLE IF EXISTS ${stats_db_name}.organization_pids purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.organization_pids STORED AS PARQUET AS select substr(o.id, 4) as id, ppid.qualifier.classname as type, ppid.value as pid from ${openaire_db_name}.organization o lateral view explode(o.pid) pids as ppid; +DROP TABLE IF EXISTS ${stats_db_name}.organization_sources purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.organization_sources STORED AS PARQUET as SELECT o.id, case when d.id is null then 'other' else o.datasource end as datasource FROM ( @@ -48,6 +60,8 @@ FROM ( from ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference=false and d.datainfo.invisible = FALSE) d on o.datasource = d.id; +DROP TABLE IF EXISTS ${stats_db_name}.result_accessroute purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.result_accessroute STORED AS PARQUET as select distinct substr(id,4) as id, accessroute from ${openaire_db_name}.result lateral view explode (instance.accessright.openaccessroute) openaccessroute as accessroute; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql index 4a8f81943..066b197e6 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15.sql @@ -6,21 +6,29 @@ ------------------------------------------------------ ------------------------------------------------------ +DROP TABLE IF EXISTS ${stats_db_name}.publication_refereed purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.publication_refereed STORED AS PARQUET as select substr(r.id, 4) as id, inst.refereed.classname as refereed from ${openaire_db_name}.publication r lateral view explode(r.instance) instances as inst where r.datainfo.deletedbyinference=false and r.datainfo.invisible = FALSE; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_refereed purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.dataset_refereed STORED AS PARQUET as select substr(r.id, 4) as id, inst.refereed.classname as refereed from ${openaire_db_name}.dataset r lateral view explode(r.instance) instances as inst where r.datainfo.deletedbyinference=false and r.datainfo.invisible = FALSE; +DROP TABLE IF EXISTS ${stats_db_name}.software_refereed purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.software_refereed STORED AS PARQUET as select substr(r.id, 4) as id, inst.refereed.classname as refereed from ${openaire_db_name}.software r lateral view explode(r.instance) instances as inst where r.datainfo.deletedbyinference=false and r.datainfo.invisible = FALSE; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_refereed purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.otherresearchproduct_refereed STORED AS PARQUET as select substr(r.id, 4) as id, inst.refereed.classname as refereed from ${openaire_db_name}.otherresearchproduct r lateral view explode(r.instance) instances as inst @@ -35,12 +43,16 @@ select * from ${stats_db_name}.software_refereed union all select * from ${stats_db_name}.otherresearchproduct_refereed; +DROP TABLE IF EXISTS ${stats_db_name}.indi_impact_measures purge; + create table if not exists ${stats_db_name}.indi_impact_measures STORED AS PARQUET as select substr(id, 4) as id, measures_ids.id impactmetric, cast(measures_ids.unit.value[0] as double) score, cast(measures_ids.unit.value[0] as decimal(6,3)) score_dec, measures_ids.unit.value[1] impact_class from ${openaire_db_name}.result lateral view explode(measures) measures as measures_ids where measures_ids.id!='views' and measures_ids.id!='downloads'; +DROP TABLE IF EXISTS ${stats_db_name}.result_apc_affiliations purge; + create table if not exists ${stats_db_name}.result_apc_affiliations STORED AS PARQUET as select distinct substr(rel.target,4) id, substr(rel.source,4) organization, o.legalname.value name, cast(rel.properties[0].value as double) apc_amount, diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql index 615f523ce..2c606fb92 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step15_5.sql @@ -1,6 +1,8 @@ ------------------------------------------- --- Extra tables, mostly used by indicators +DROP TABLE IF EXISTS ${stats_db_name}.result_projectcount purge; + create table if not exists ${stats_db_name}.result_projectcount STORED AS PARQUET as select r.id, count(distinct p.id) as count from ${stats_db_name}.result r @@ -8,6 +10,8 @@ left outer join ${stats_db_name}.result_projects rp on rp.id=r.id left outer join ${stats_db_name}.project p on p.id=rp.project group by r.id; +DROP TABLE IF EXISTS ${stats_db_name}.result_fundercount purge; + create table if not exists ${stats_db_name}.result_fundercount STORED AS PARQUET as select r.id, count(distinct p.funder) as count from ${stats_db_name}.result r @@ -15,6 +19,8 @@ left outer join ${stats_db_name}.result_projects rp on rp.id=r.id left outer join ${stats_db_name}.project p on p.id=rp.project group by r.id; +DROP TABLE IF EXISTS ${stats_db_name}.project_resultcount purge; + create table if not exists ${stats_db_name}.project_resultcount STORED AS PARQUET as with rcount as ( select p.id as pid, count(distinct r.id) as `count`, r.type as type @@ -37,6 +43,8 @@ create or replace view ${stats_db_name}.totalresearchersft as select * from stat create or replace view ${stats_db_name}.hrrst as select * from stats_ext.hrrst; create or replace view ${stats_db_name}.graduatedoctorates as select * from stats_ext.graduatedoctorates; +DROP TABLE IF EXISTS ${stats_db_name}.result_instance purge; + create table if not exists ${stats_db_name}.result_instance stored as parquet as select distinct r.* from ( @@ -45,6 +53,8 @@ from ( from ${openaire_db_name}.result r lateral view explode(r.instance) instances as inst lateral view explode(inst.pid) pids as p) r join ${stats_db_name}.result res on res.id=r.id; +DROP TABLE IF EXISTS ${stats_db_name}.result_apc purge; + create table if not exists ${stats_db_name}.result_apc STORED AS PARQUET as select r.id, r.amount, r.currency from ( diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql index 6af486340..8180e6527 100755 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql @@ -104,7 +104,7 @@ from ${stats_db_name}.tmp as o1 join ${stats_db_name}.tmp as o2 where o1.id=o2.id and o1.organization!=o2.organization and o1.name!=o2.name group by o1.organization, o2.organization, o1.name, o2.name; -drop table ${stats_db_name}.tmp purge; +drop table if exists ${stats_db_name}.tmp purge; create TEMPORARY TABLE ${stats_db_name}.tmp AS select distinct ro.organization organization, ro.id, o.name, o.country from ${stats_db_name}.result_organization ro @@ -118,7 +118,7 @@ from ${stats_db_name}.tmp as o1 join ${stats_db_name}.tmp as o2 on o1.id=o2.id where o1.id=o2.id and o1.country!=o2.country group by o1.organization, o1.id, o1.name, o2.country; -drop table ${stats_db_name}.tmp purge; +drop table if exists ${stats_db_name}.tmp purge; create TEMPORARY TABLE ${stats_db_name}.tmp AS select o.id organization, o.name, ro.project as project from ${stats_db_name}.organization o @@ -133,7 +133,7 @@ from ${stats_db_name}.tmp as o1 where o1.organization<>o2.organization and o1.name<>o2.name group by o1.name,o2.name, o1.organization, o2.organization; -drop table ${stats_db_name}.tmp purge; +drop table if exists ${stats_db_name}.tmp purge; create TEMPORARY TABLE ${stats_db_name}.tmp AS select o.id organization, o.name, o.country , ro.project as project from ${stats_db_name}.organization o @@ -149,7 +149,7 @@ from ${stats_db_name}.tmp as o1 where o1.organization<>o2.organization and o1.country<>o2.country group by o1.organization, o2.country, o1.name; -drop table ${stats_db_name}.tmp purge; +drop table if exists ${stats_db_name}.tmp purge; drop table if exists ${stats_db_name}.indi_funder_country_collab purge; @@ -178,7 +178,7 @@ from ${stats_db_name}.tmp as o1 where o1.country<>o2.country group by o1.country, o2.country; -drop table ${stats_db_name}.tmp purge; +drop table if exists ${stats_db_name}.tmp purge; ---- Sprint 4 ---- drop table if exists ${stats_db_name}.indi_pub_diamond purge; @@ -422,7 +422,7 @@ drop table if exists ${stats_db_name}.indi_pub_hybrid purge; -- on pd.id=tmp.id; create table if not exists ${stats_db_name}.indi_pub_hybrid stored as parquet as -select pd.id,coalesce(is_hybrid,0) is_hybrid from ${stats_db_name}.publication_datasources pd +select distinct pd.id,coalesce(is_hybrid,0) is_hybrid from ${stats_db_name}.publication_datasources pd left outer join (select pd.id, 1 as is_hybrid from ${stats_db_name}.publication_datasources pd join ${stats_db_name}.datasource d on pd.datasource=d.id join ${stats_db_name}.result_instance ri on ri.id=pd.id @@ -492,7 +492,7 @@ CREATE TEMPORARY TABLE ${stats_db_name}.allresults as select year, ro.organizati drop table if exists ${stats_db_name}.indi_org_fairness_pub_year purge; create table if not exists ${stats_db_name}.indi_org_fairness_pub_year stored as parquet as -select allresults.year, allresults.organization, result_fair.no_result_fair/allresults.no_allresults org_fairness +select cast(allresults.year as int) year, allresults.organization, result_fair.no_result_fair/allresults.no_allresults org_fairness from ${stats_db_name}.allresults join ${stats_db_name}.result_fair on result_fair.organization=allresults.organization and result_fair.year=allresults.year; @@ -813,8 +813,8 @@ drop table if exists ${stats_db_name}.indi_pub_bronze_oa purge; --and (d.type='Journal' or d.type='Journal Aggregator/Publisher') --and ri.accessright='Open Access') tmp on tmp.id=p.id; -create table ${stats_db_name}.indi_pub_bronze stored as parquet as -select pd.id,coalesce(is_bronze_oa,0) is_bronze_oa from ${stats_db_name}.publication_datasources pd +create table ${stats_db_name}.indi_pub_bronze_oa stored as parquet as +select distinct pd.id,coalesce(is_bronze_oa,0) is_bronze_oa from ${stats_db_name}.publication_datasources pd left outer join (select pd.id, 1 as is_bronze_oa from ${stats_db_name}.publication_datasources pd join ${stats_db_name}.datasource d on pd.datasource=d.id join ${stats_db_name}.result_instance ri on ri.id=pd.id diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql index 41c95758c..b55af13d4 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_1-definitions.sql @@ -3,6 +3,8 @@ ---------------------------------------------------- -- Peer reviewed: +drop table if exists ${stats_db_name}.result_peerreviewed purge; + create table IF NOT EXISTS ${stats_db_name}.result_peerreviewed STORED AS PARQUET as select r.id as id, case when doi.doi_from_crossref=1 and grey.grey_lit=0 then true else false end as peer_reviewed from ${stats_db_name}.result r @@ -10,12 +12,16 @@ left outer join ${stats_db_name}.indi_pub_doi_from_crossref doi on doi.id=r.id left outer join ${stats_db_name}.indi_pub_grey_lit grey on grey.id=r.id; -- Green OA: +drop table if exists ${stats_db_name}.result_greenoa purge; + create table IF NOT EXISTS ${stats_db_name}.result_greenoa STORED AS PARQUET as select r.id, case when green.green_oa=1 then true else false end as green from ${stats_db_name}.result r left outer join ${stats_db_name}.indi_pub_green_oa green on green.id=r.id; -- GOLD OA: +drop table if exists ${stats_db_name}.result_gold purge; + create table IF NOT EXISTS ${stats_db_name}.result_gold STORED AS PARQUET as select r.id, case when gold.is_gold=1 then true else false end as gold from ${stats_db_name}.result r diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql index f737c1ea6..7faa91697 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16_5.sql @@ -1,6 +1,7 @@ -- replace the creation of the result view to include the boolean fields from the previous tables (green, gold, -- peer reviewed) drop table if exists ${stats_db_name}.result_tmp; + CREATE TABLE ${stats_db_name}.result_tmp ( id STRING, title STRING, diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql index 4ffbd384b..8e56f98fc 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step2.sql @@ -5,6 +5,7 @@ -------------------------------------------------------------- -- Publication temporary table +DROP TABLE IF EXISTS ${stats_db_name}.publication_tmp purge; CREATE TABLE ${stats_db_name}.publication_tmp ( id STRING, @@ -40,12 +41,16 @@ SELECT substr(p.id, 4) as id, from ${openaire_db_name}.publication p where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.publication_classifications purge; + CREATE TABLE ${stats_db_name}.publication_classifications STORED AS PARQUET AS SELECT substr(p.id, 4) as id, instancetype.classname as type from ${openaire_db_name}.publication p LATERAL VIEW explode(p.instance.instancetype) instances as instancetype where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.publication_concepts purge; + CREATE TABLE ${stats_db_name}.publication_concepts STORED AS PARQUET AS SELECT substr(p.id, 4) as id, case when contexts.context.id RLIKE '^[^::]+::[^::]+::.+$' then contexts.context.id @@ -55,6 +60,8 @@ from ${openaire_db_name}.publication p LATERAL VIEW explode(p.context) contexts as context where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.publication_datasources purge; + CREATE TABLE ${stats_db_name}.publication_datasources STORED AS PARQUET as SELECT p.id, case when d.id is null then 'other' else p.datasource end as datasource FROM ( @@ -66,29 +73,39 @@ FROM ( from ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference = false and d.datainfo.invisible=false) d on p.datasource = d.id; +DROP TABLE IF EXISTS ${stats_db_name}.publication_languages purge; + CREATE TABLE ${stats_db_name}.publication_languages STORED AS PARQUET AS select substr(p.id, 4) as id, p.language.classname as language FROM ${openaire_db_name}.publication p where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.publication_oids purge; + CREATE TABLE ${stats_db_name}.publication_oids STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.publication p LATERAL VIEW explode(p.originalid) oids AS ids where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.publication_pids purge; + CREATE TABLE ${stats_db_name}.publication_pids STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value as pid FROM ${openaire_db_name}.publication p LATERAL VIEW explode(p.pid) pids AS ppid where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.publication_topics purge; + CREATE TABLE ${stats_db_name}.publication_topics STORED AS PARQUET as select substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS TYPE, subjects.subject.value AS topic FROM ${openaire_db_name}.publication p LATERAL VIEW explode(p.subject) subjects AS subject where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.publication_citations purge; + CREATE TABLE ${stats_db_name}.publication_citations STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites FROM ${openaire_db_name}.publication p diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql index d5d242230..b52abd865 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql @@ -1,79 +1,3 @@ ---drop database if exists TARGET cascade; ---create database if not exists TARGET; --- ---create view if not exists TARGET.category as select * from SOURCE.category; ---create view if not exists TARGET.concept as select * from SOURCE.concept; ---create view if not exists TARGET.context as select * from SOURCE.context; ---create view if not exists TARGET.country as select * from SOURCE.country; ---create view if not exists TARGET.countrygdp as select * from SOURCE.countrygdp; ---create view if not exists TARGET.creation_date as select * from SOURCE.creation_date; ---create view if not exists TARGET.funder as select * from SOURCE.funder; ---create view if not exists TARGET.fundref as select * from SOURCE.fundref; ---create view if not exists TARGET.rndexpenditure as select * from SOURCE.rndexpediture; ---create view if not exists TARGET.rndgdpexpenditure as select * from SOURCE.rndgdpexpenditure; ---create view if not exists TARGET.doctoratestudents as select * from SOURCE.doctoratestudents; ---create view if not exists TARGET.totalresearchers as select * from SOURCE.totalresearchers; ---create view if not exists TARGET.totalresearchersft as select * from SOURCE.totalresearchersft; ---create view if not exists TARGET.hrrst as select * from SOURCE.hrrst; --- ---create table TARGET.result stored as parquet as --- select distinct * from ( --- select * from SOURCE.result r where exists (select 1 from SOURCE.result_projects rp join SOURCE.project p on rp.project=p.id where rp.id=r.id) --- union all --- select * from SOURCE.result r where exists (select 1 from SOURCE.result_concepts rc where rc.id=r.id) --- union all --- select * from SOURCE.result r where exists (select 1 from SOURCE.result_organization ro where ro.id=r.id and ro.organization in ( --- 'openorgs____::b84450f9864182c67b8611b5593f4250', --"Athena Research and Innovation Center In Information Communication & Knowledge Technologies', --ARC" --- 'openorgs____::d41cf6bd4ab1b1362a44397e0b95c975', --National Research Council --- 'openorgs____::d2a09b9d5eabb10c95f9470e172d05d2', --??? Not exists ?? --- 'openorgs____::d169c7407dd417152596908d48c11460', --Masaryk University --- 'openorgs____::1ec924b1759bb16d0a02f2dad8689b21', --University of Belgrade --- 'openorgs____::0ae431b820e4c33db8967fbb2b919150', --University of Helsinki --- 'openorgs____::759d59f05d77188faee99b7493b46805', --University of Minho --- 'openorgs____::cad284878801b9465fa51a95b1d779db', --Universidad Politécnica de Madrid --- 'openorgs____::eadc8da90a546e98c03f896661a2e4d4', --University of Göttingen --- 'openorgs____::c0286313e36479eff8676dba9b724b40', --National and Kapodistrian University of Athens --- -- 'openorgs____::c80a8243a5e5c620d7931c88d93bf17a', --Université Paris Diderot --- 'openorgs____::c08634f0a6b0081c3dc6e6c93a4314f3', --Bielefeld University --- 'openorgs____::6fc85e4a8f7ecaf4b0c738d010e967ea', --University of Southern Denmark --- 'openorgs____::3d6122f87f9a97a99d8f6e3d73313720', --Humboldt-Universität zu Berlin --- 'openorgs____::16720ada63d0fa8ca41601feae7d1aa5', --TU Darmstadt --- 'openorgs____::ccc0a066b56d2cfaf90c2ae369df16f5', --KU Leuven --- 'openorgs____::4c6f119632adf789746f0a057ed73e90', --University of the Western Cape --- 'openorgs____::ec3665affa01aeafa28b7852c4176dbd', --Rudjer Boskovic Institute --- 'openorgs____::5f31346d444a7f06a28c880fb170b0f6', --Ghent University --- 'openorgs____::2dbe47117fd5409f9c61620813456632', --University of Luxembourg --- 'openorgs____::6445d7758d3a40c4d997953b6632a368', --National Institute of Informatics (NII) --- 'openorgs____::b77c01aa15de3675da34277d48de2ec1', -- Valencia Catholic University Saint Vincent Martyr --- 'openorgs____::7fe2f66cdc43983c6b24816bfe9cf6a0', -- Unviersity of Warsaw --- 'openorgs____::15e7921fc50d9aa1229a82a84429419e', -- University Of Thessaly --- 'openorgs____::11f7919dadc8f8a7251af54bba60c956', -- Technical University of Crete --- 'openorgs____::84f0c5f5dbb6daf42748485924efde4b', -- University of Piraeus --- 'openorgs____::4ac562f0376fce3539504567649cb373', -- University of Patras --- 'openorgs____::3e8d1f8c3f6cd7f418b09f1f58b4873b', -- Aristotle University of Thessaloniki --- 'openorgs____::3fcef6e1c469c10f2a84b281372c9814', -- World Bank --- 'openorgs____::1698a2eb1885ef8adb5a4a969e745ad3', -- École des Ponts ParisTech --- 'openorgs____::e15adb13c4dadd49de4d35c39b5da93a', -- Nanyang Technological University --- 'openorgs____::4b34103bde246228fcd837f5f1bf4212', -- Autonomous University of Barcelona --- 'openorgs____::72ec75fcfc4e0df1a76dc4c49007fceb', -- McMaster University --- 'openorgs____::51c7fc556e46381734a25a6fbc3fd398', -- University of Modena and Reggio Emilia --- 'openorgs____::235d7f9ad18ecd7e6dc62ea4990cb9db', -- Bilkent University --- 'openorgs____::31f2fa9e05b49d4cf40a19c3fed8eb06', -- Saints Cyril and Methodius University of Skopje --- 'openorgs____::db7686f30f22cbe73a4fde872ce812a6', -- University of Milan --- 'openorgs____::b8b8ca674452579f3f593d9f5e557483', -- University College Cork --- 'openorgs____::38d7097854736583dde879d12dacafca' -- Brown University --- 'openorgs____::57784c9e047e826fefdb1ef816120d92', --Arts et Métiers ParisTech --- 'openorgs____::2530baca8a15936ba2e3297f2bce2e7e', -- University of Cape Town --- 'openorgs____::d11f981828c485cd23d93f7f24f24db1', -- Technological University Dublin --- 'openorgs____::5e6bf8962665cdd040341171e5c631d8', -- Delft University of Technology --- 'openorgs____::846cb428d3f52a445f7275561a7beb5d', -- University of Manitoba --- 'openorgs____::eb391317ed0dc684aa81ac16265de041', -- Universitat Rovira i Virgili --- 'openorgs____::66aa9fc2fceb271423dfabcc38752dc0', -- Lund University --- 'openorgs____::3cff625a4370d51e08624cc586138b2f' -- IMT Atlantique --- ) )) foo; --- ---ANALYZE TABLE TARGET.result COMPUTE STATISTICS; - create view if not exists TARGET.category as select * from SOURCE.category; create view if not exists TARGET.concept as select * from SOURCE.concept; create view if not exists TARGET.context as select * from SOURCE.context; @@ -91,76 +15,52 @@ create view if not exists TARGET.hrrst as select * from SOURCE.hrrst; create view if not exists TARGET.graduatedoctorates as select * from SOURCE.graduatedoctorates; create table TARGET.result_citations stored as parquet as select * from SOURCE.result_citations orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_citations COMPUTE STATISTICS; create table TARGET.result_references_oc stored as parquet as select * from SOURCE.result_references_oc orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_references_oc COMPUTE STATISTICS; create table TARGET.result_citations_oc stored as parquet as select * from SOURCE.result_citations_oc orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_citations_oc COMPUTE STATISTICS; create table TARGET.result_classifications stored as parquet as select * from SOURCE.result_classifications orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_classifications COMPUTE STATISTICS; create table TARGET.result_apc stored as parquet as select * from SOURCE.result_apc orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_apc COMPUTE STATISTICS; create table TARGET.result_concepts stored as parquet as select * from SOURCE.result_concepts orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_concepts COMPUTE STATISTICS; create table TARGET.result_datasources stored as parquet as select * from SOURCE.result_datasources orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_datasources COMPUTE STATISTICS; create table TARGET.result_fundercount stored as parquet as select * from SOURCE.result_fundercount orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_fundercount COMPUTE STATISTICS; create table TARGET.result_gold stored as parquet as select * from SOURCE.result_gold orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_gold COMPUTE STATISTICS; create table TARGET.result_greenoa stored as parquet as select * from SOURCE.result_greenoa orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_greenoa COMPUTE STATISTICS; create table TARGET.result_languages stored as parquet as select * from SOURCE.result_languages orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_languages COMPUTE STATISTICS; create table TARGET.result_licenses stored as parquet as select * from SOURCE.result_licenses orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_licenses COMPUTE STATISTICS; create table TARGET.licenses_normalized STORED AS PARQUET as select * from SOURCE.licenses_normalized; ---ANALYZE TABLE TARGET.licenses_normalized COMPUTE STATISTICS; create table TARGET.result_oids stored as parquet as select * from SOURCE.result_oids orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_oids COMPUTE STATISTICS; create table TARGET.result_organization stored as parquet as select * from SOURCE.result_organization orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_organization COMPUTE STATISTICS; create table TARGET.result_peerreviewed stored as parquet as select * from SOURCE.result_peerreviewed orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_peerreviewed COMPUTE STATISTICS; create table TARGET.result_pids stored as parquet as select * from SOURCE.result_pids orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_pids COMPUTE STATISTICS; create table TARGET.result_projectcount stored as parquet as select * from SOURCE.result_projectcount orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_projectcount COMPUTE STATISTICS; create table TARGET.result_projects stored as parquet as select * from SOURCE.result_projects orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_projects COMPUTE STATISTICS; create table TARGET.result_refereed stored as parquet as select * from SOURCE.result_refereed orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_refereed COMPUTE STATISTICS; create table TARGET.result_sources stored as parquet as select * from SOURCE.result_sources orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_sources COMPUTE STATISTICS; create table TARGET.result_topics stored as parquet as select * from SOURCE.result_topics orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_topics COMPUTE STATISTICS; create table TARGET.result_fos stored as parquet as select * from SOURCE.result_fos orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_fos COMPUTE STATISTICS; create table TARGET.result_accessroute stored as parquet as select * from SOURCE.result_accessroute orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_accessroute COMPUTE STATISTICS; create table TARGET.result_orcid stored as parquet as select * from SOURCE.result_orcid orig where exists (select 1 from TARGET.result r where r.id=orig.id); @@ -169,7 +69,6 @@ create view TARGET.foo2 as select * from SOURCE.result_result rr where rr.target create table TARGET.result_result STORED AS PARQUET as select distinct * from (select * from TARGET.foo1 union all select * from TARGET.foo2) foufou; drop view TARGET.foo1; drop view TARGET.foo2; ---ANALYZE TABLE TARGET.result_result COMPUTE STATISTICS; -- datasources create view if not exists TARGET.datasource as select * from SOURCE.datasource; @@ -178,7 +77,6 @@ create view if not exists TARGET.datasource_organizations as select * from SOURC create view if not exists TARGET.datasource_sources as select * from SOURCE.datasource_sources; create table TARGET.datasource_results stored as parquet as select id as result, datasource as id from TARGET.result_datasources; ---ANALYZE TABLE TARGET.datasource_results COMPUTE STATISTICS; -- organizations create view if not exists TARGET.organization as select * from SOURCE.organization; @@ -196,28 +94,27 @@ create view if not exists TARGET.project_classification as select * from SOURCE. create view if not exists TARGET.project_organization_contribution as select * from SOURCE.project_organization_contribution; create table TARGET.project_results stored as parquet as select id as result, project as id from TARGET.result_projects; ---ANALYZE TABLE TARGET.project_results COMPUTE STATISTICS; -- indicators -- Sprint 1 ---- create table TARGET.indi_pub_green_oa stored as parquet as select * from SOURCE.indi_pub_green_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_green_oa COMPUTE STATISTICS; + create table TARGET.indi_pub_grey_lit stored as parquet as select * from SOURCE.indi_pub_grey_lit orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_grey_lit COMPUTE STATISTICS; + create table TARGET.indi_pub_doi_from_crossref stored as parquet as select * from SOURCE.indi_pub_doi_from_crossref orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_doi_from_crossref COMPUTE STATISTICS; + -- Sprint 2 ---- create table TARGET.indi_result_has_cc_licence stored as parquet as select * from SOURCE.indi_result_has_cc_licence orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_result_has_cc_licence COMPUTE STATISTICS; + create table TARGET.indi_result_has_cc_licence_url stored as parquet as select * from SOURCE.indi_result_has_cc_licence_url orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_result_has_cc_licence_url COMPUTE STATISTICS; + create table TARGET.indi_pub_has_abstract stored as parquet as select * from SOURCE.indi_pub_has_abstract orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_has_abstract COMPUTE STATISTICS; + create table TARGET.indi_result_with_orcid stored as parquet as select * from SOURCE.indi_result_with_orcid orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_result_with_orcid COMPUTE STATISTICS; + ---- Sprint 3 ---- create table TARGET.indi_funded_result_with_fundref stored as parquet as select * from SOURCE.indi_funded_result_with_fundref orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_funded_result_with_fundref COMPUTE STATISTICS; + create view TARGET.indi_result_org_collab as select * from SOURCE.indi_result_org_collab; create view TARGET.indi_result_org_country_collab as select * from SOURCE.indi_result_org_country_collab; create view TARGET.indi_project_collab_org as select * from SOURCE.indi_project_collab_org; @@ -226,32 +123,32 @@ create view TARGET.indi_funder_country_collab as select * from SOURCE.indi_funde create view TARGET.indi_result_country_collab as select * from SOURCE.indi_result_country_collab; ---- Sprint 4 ---- create table TARGET.indi_pub_diamond stored as parquet as select * from SOURCE.indi_pub_diamond orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_diamond COMPUTE STATISTICS; + create table TARGET.indi_pub_in_transformative stored as parquet as select * from SOURCE.indi_pub_in_transformative orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_in_transformative COMPUTE STATISTICS; + create table TARGET.indi_pub_closed_other_open stored as parquet as select * from SOURCE.indi_pub_closed_other_open orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_closed_other_open COMPUTE STATISTICS; + ---- Sprint 5 ---- create table TARGET.indi_result_no_of_copies stored as parquet as select * from SOURCE.indi_result_no_of_copies orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_result_no_of_copies COMPUTE STATISTICS; + ---- Sprint 6 ---- create table TARGET.indi_pub_hybrid_oa_with_cc stored as parquet as select * from SOURCE.indi_pub_hybrid_oa_with_cc orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_hybrid_oa_with_cc COMPUTE STATISTICS; + create table TARGET.indi_pub_bronze_oa stored as parquet as select * from SOURCE.indi_pub_bronze_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_bronze_oa COMPUTE STATISTICS; + create table TARGET.indi_pub_downloads stored as parquet as select * from SOURCE.indi_pub_downloads orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); ---ANALYZE TABLE TARGET.indi_pub_downloads COMPUTE STATISTICS; + create table TARGET.indi_pub_downloads_datasource stored as parquet as select * from SOURCE.indi_pub_downloads_datasource orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); ---ANALYZE TABLE TARGET.indi_pub_downloads_datasource COMPUTE STATISTICS; + create table TARGET.indi_pub_downloads_year stored as parquet as select * from SOURCE.indi_pub_downloads_year orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); ---ANALYZE TABLE TARGET.indi_pub_downloads_year COMPUTE STATISTICS; + create table TARGET.indi_pub_downloads_datasource_year stored as parquet as select * from SOURCE.indi_pub_downloads_datasource_year orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); ---ANALYZE TABLE TARGET.indi_pub_downloads_datasource_year COMPUTE STATISTICS; + ---- Sprint 7 ---- create table TARGET.indi_pub_gold_oa stored as parquet as select * from SOURCE.indi_pub_gold_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_gold_oa COMPUTE STATISTICS; + create table TARGET.indi_pub_hybrid stored as parquet as select * from SOURCE.indi_pub_hybrid orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_hybrid COMPUTE STATISTICS; + create view TARGET.indi_org_fairness as select * from SOURCE.indi_org_fairness; create view TARGET.indi_org_fairness_pub_pr as select * from SOURCE.indi_org_fairness_pub_pr; create view TARGET.indi_org_fairness_pub_year as select * from SOURCE.indi_org_fairness_pub_year; @@ -262,17 +159,17 @@ create view TARGET.indi_org_findable as select * from SOURCE.indi_org_findable; create view TARGET.indi_org_openess as select * from SOURCE.indi_org_openess; create view TARGET.indi_org_openess_year as select * from SOURCE.indi_org_openess_year; create table TARGET.indi_pub_has_preprint stored as parquet as select * from SOURCE.indi_pub_has_preprint orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_has_preprint COMPUTE STATISTICS; + create table TARGET.indi_pub_in_subscribed stored as parquet as select * from SOURCE.indi_pub_in_subscribed orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_in_subscribed COMPUTE STATISTICS; + create table TARGET.indi_result_with_pid stored as parquet as select * from SOURCE.indi_result_with_pid orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_result_with_pid COMPUTE STATISTICS; + create table TARGET.indi_impact_measures stored as parquet as select * from SOURCE.indi_impact_measures orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_impact_measures COMPUTE STATISTICS; + create table TARGET.indi_pub_interdisciplinarity stored as parquet as select * from SOURCE.indi_pub_interdisciplinarity orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_interdisciplinarity COMPUTE STATISTICS; + create table TARGET.result_apc_affiliations stored as parquet as select * from SOURCE.result_apc_affiliations orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_apc_affiliations COMPUTE STATISTICS; + create table TARGET.indi_is_project_result_after stored as parquet as select * from SOURCE.indi_is_project_result_after orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); create table TARGET.indi_is_funder_plan_s stored as parquet as select * from SOURCE.indi_is_funder_plan_s orig where exists (select 1 from TARGET.result r where r.id=orig.id); create view TARGET.indi_funder_fairness as select * from SOURCE.indi_funder_fairness; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql index df4795e3e..2b6a68514 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql @@ -81,8 +81,6 @@ create table TARGET.result stored as parquet as 'openorgs____::8839b55dae0c84d56fd533f52d5d483a' -- Leibniz Institute of Ecological Urban and Regional Development ) )) foo; ---ANALYZE TABLE TARGET.result COMPUTE STATISTICS; - create view if not exists TARGET.category as select * from SOURCE.category; create view if not exists TARGET.concept as select * from SOURCE.concept; create view if not exists TARGET.context as select * from SOURCE.context; @@ -97,86 +95,63 @@ create view if not exists TARGET.doctoratestudents as select * from SOURCE.docto create view if not exists TARGET.totalresearchers as select * from SOURCE.totalresearchers; create view if not exists TARGET.totalresearchersft as select * from SOURCE.totalresearchersft; create view if not exists TARGET.hrrst as select * from SOURCE.hrrst; ---create view if not exists TARGET.graduatedoctorates as select * from SOURCE.graduatedoctorates; +create view if not exists TARGET.graduatedoctorates as select * from SOURCE.graduatedoctorates; create table TARGET.result_citations stored as parquet as select * from SOURCE.result_citations orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_citations COMPUTE STATISTICS; create table TARGET.result_references_oc stored as parquet as select * from SOURCE.result_references_oc orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_references_oc COMPUTE STATISTICS; create table TARGET.result_citations_oc stored as parquet as select * from SOURCE.result_citations_oc orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_citations_oc COMPUTE STATISTICS; create table TARGET.result_classifications stored as parquet as select * from SOURCE.result_classifications orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_classifications COMPUTE STATISTICS; create table TARGET.result_apc stored as parquet as select * from SOURCE.result_apc orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_apc COMPUTE STATISTICS; create table TARGET.result_concepts stored as parquet as select * from SOURCE.result_concepts orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_concepts COMPUTE STATISTICS; create table TARGET.result_datasources stored as parquet as select * from SOURCE.result_datasources orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_datasources COMPUTE STATISTICS; create table TARGET.result_fundercount stored as parquet as select * from SOURCE.result_fundercount orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_fundercount COMPUTE STATISTICS; create table TARGET.result_gold stored as parquet as select * from SOURCE.result_gold orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_gold COMPUTE STATISTICS; create table TARGET.result_greenoa stored as parquet as select * from SOURCE.result_greenoa orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_greenoa COMPUTE STATISTICS; create table TARGET.result_languages stored as parquet as select * from SOURCE.result_languages orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_languages COMPUTE STATISTICS; create table TARGET.result_licenses stored as parquet as select * from SOURCE.result_licenses orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_licenses COMPUTE STATISTICS; create table TARGET.licenses_normalized STORED AS PARQUET as select * from SOURCE.licenses_normalized; ---ANALYZE TABLE TARGET.licenses_normalized COMPUTE STATISTICS; create table TARGET.result_oids stored as parquet as select * from SOURCE.result_oids orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_oids COMPUTE STATISTICS; create table TARGET.result_organization stored as parquet as select * from SOURCE.result_organization orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_organization COMPUTE STATISTICS; create table TARGET.result_peerreviewed stored as parquet as select * from SOURCE.result_peerreviewed orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_peerreviewed COMPUTE STATISTICS; create table TARGET.result_pids stored as parquet as select * from SOURCE.result_pids orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_pids COMPUTE STATISTICS; create table TARGET.result_projectcount stored as parquet as select * from SOURCE.result_projectcount orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_projectcount COMPUTE STATISTICS; create table TARGET.result_projects stored as parquet as select * from SOURCE.result_projects orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_projects COMPUTE STATISTICS; create table TARGET.result_refereed stored as parquet as select * from SOURCE.result_refereed orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_refereed COMPUTE STATISTICS; create table TARGET.result_sources stored as parquet as select * from SOURCE.result_sources orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_sources COMPUTE STATISTICS; create table TARGET.result_topics stored as parquet as select * from SOURCE.result_topics orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_topics COMPUTE STATISTICS; create table TARGET.result_fos stored as parquet as select * from SOURCE.result_fos orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_fos COMPUTE STATISTICS; create table TARGET.result_accessroute stored as parquet as select * from SOURCE.result_accessroute orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_accessroute COMPUTE STATISTICS; + +create table TARGET.result_orcid stored as parquet as select * from SOURCE.result_orcid orig where exists (select 1 from TARGET.result r where r.id=orig.id); create view TARGET.foo1 as select * from SOURCE.result_result rr where rr.source in (select id from TARGET.result); create view TARGET.foo2 as select * from SOURCE.result_result rr where rr.target in (select id from TARGET.result); create table TARGET.result_result STORED AS PARQUET as select distinct * from (select * from TARGET.foo1 union all select * from TARGET.foo2) foufou; drop view TARGET.foo1; drop view TARGET.foo2; ---ANALYZE TABLE TARGET.result_result COMPUTE STATISTICS; -- datasources create view if not exists TARGET.datasource as select * from SOURCE.datasource; @@ -185,7 +160,6 @@ create view if not exists TARGET.datasource_organizations as select * from SOURC create view if not exists TARGET.datasource_sources as select * from SOURCE.datasource_sources; create table TARGET.datasource_results stored as parquet as select id as result, datasource as id from TARGET.result_datasources; ---ANALYZE TABLE TARGET.datasource_results COMPUTE STATISTICS; -- organizations create view if not exists TARGET.organization as select * from SOURCE.organization; @@ -203,28 +177,26 @@ create view if not exists TARGET.project_classification as select * from SOURCE. create view if not exists TARGET.project_organization_contribution as select * from SOURCE.project_organization_contribution; create table TARGET.project_results stored as parquet as select id as result, project as id from TARGET.result_projects; ---ANALYZE TABLE TARGET.project_results COMPUTE STATISTICS; -- indicators -- Sprint 1 ---- create table TARGET.indi_pub_green_oa stored as parquet as select * from SOURCE.indi_pub_green_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_green_oa COMPUTE STATISTICS; + create table TARGET.indi_pub_grey_lit stored as parquet as select * from SOURCE.indi_pub_grey_lit orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_grey_lit COMPUTE STATISTICS; + create table TARGET.indi_pub_doi_from_crossref stored as parquet as select * from SOURCE.indi_pub_doi_from_crossref orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_doi_from_crossref COMPUTE STATISTICS; + -- Sprint 2 ---- create table TARGET.indi_result_has_cc_licence stored as parquet as select * from SOURCE.indi_result_has_cc_licence orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_result_has_cc_licence COMPUTE STATISTICS; + create table TARGET.indi_result_has_cc_licence_url stored as parquet as select * from SOURCE.indi_result_has_cc_licence_url orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_result_has_cc_licence_url COMPUTE STATISTICS; + create table TARGET.indi_pub_has_abstract stored as parquet as select * from SOURCE.indi_pub_has_abstract orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_has_abstract COMPUTE STATISTICS; + create table TARGET.indi_result_with_orcid stored as parquet as select * from SOURCE.indi_result_with_orcid orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_result_with_orcid COMPUTE STATISTICS; + ---- Sprint 3 ---- create table TARGET.indi_funded_result_with_fundref stored as parquet as select * from SOURCE.indi_funded_result_with_fundref orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_funded_result_with_fundref COMPUTE STATISTICS; create view TARGET.indi_result_org_collab as select * from SOURCE.indi_result_org_collab; create view TARGET.indi_result_org_country_collab as select * from SOURCE.indi_result_org_country_collab; create view TARGET.indi_project_collab_org as select * from SOURCE.indi_project_collab_org; @@ -233,32 +205,29 @@ create view TARGET.indi_funder_country_collab as select * from SOURCE.indi_funde create view TARGET.indi_result_country_collab as select * from SOURCE.indi_result_country_collab; ---- Sprint 4 ---- create table TARGET.indi_pub_diamond stored as parquet as select * from SOURCE.indi_pub_diamond orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_diamond COMPUTE STATISTICS; + create table TARGET.indi_pub_in_transformative stored as parquet as select * from SOURCE.indi_pub_in_transformative orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_in_transformative COMPUTE STATISTICS; + create table TARGET.indi_pub_closed_other_open stored as parquet as select * from SOURCE.indi_pub_closed_other_open orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_closed_other_open COMPUTE STATISTICS; ---- Sprint 5 ---- create table TARGET.indi_result_no_of_copies stored as parquet as select * from SOURCE.indi_result_no_of_copies orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_result_no_of_copies COMPUTE STATISTICS; ---- Sprint 6 ---- create table TARGET.indi_pub_hybrid_oa_with_cc stored as parquet as select * from SOURCE.indi_pub_hybrid_oa_with_cc orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_hybrid_oa_with_cc COMPUTE STATISTICS; + create table TARGET.indi_pub_bronze_oa stored as parquet as select * from SOURCE.indi_pub_bronze_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_bronze_oa COMPUTE STATISTICS; + create table TARGET.indi_pub_downloads stored as parquet as select * from SOURCE.indi_pub_downloads orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); ---ANALYZE TABLE TARGET.indi_pub_downloads COMPUTE STATISTICS; + create table TARGET.indi_pub_downloads_datasource stored as parquet as select * from SOURCE.indi_pub_downloads_datasource orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); ---ANALYZE TABLE TARGET.indi_pub_downloads_datasource COMPUTE STATISTICS; + create table TARGET.indi_pub_downloads_year stored as parquet as select * from SOURCE.indi_pub_downloads_year orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); ---ANALYZE TABLE TARGET.indi_pub_downloads_year COMPUTE STATISTICS; + create table TARGET.indi_pub_downloads_datasource_year stored as parquet as select * from SOURCE.indi_pub_downloads_datasource_year orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); ---ANALYZE TABLE TARGET.indi_pub_downloads_datasource_year COMPUTE STATISTICS; + ---- Sprint 7 ---- create table TARGET.indi_pub_gold_oa stored as parquet as select * from SOURCE.indi_pub_gold_oa orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_gold_oa COMPUTE STATISTICS; + create table TARGET.indi_pub_hybrid stored as parquet as select * from SOURCE.indi_pub_hybrid orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_hybrid COMPUTE STATISTICS; create view TARGET.indi_org_fairness as select * from SOURCE.indi_org_fairness; create view TARGET.indi_org_fairness_pub_pr as select * from SOURCE.indi_org_fairness_pub_pr; create view TARGET.indi_org_fairness_pub_year as select * from SOURCE.indi_org_fairness_pub_year; @@ -269,19 +238,19 @@ create view TARGET.indi_org_findable as select * from SOURCE.indi_org_findable; create view TARGET.indi_org_openess as select * from SOURCE.indi_org_openess; create view TARGET.indi_org_openess_year as select * from SOURCE.indi_org_openess_year; create table TARGET.indi_pub_has_preprint stored as parquet as select * from SOURCE.indi_pub_has_preprint orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_has_preprint COMPUTE STATISTICS; + create table TARGET.indi_pub_in_subscribed stored as parquet as select * from SOURCE.indi_pub_in_subscribed orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_in_subscribed COMPUTE STATISTICS; + create table TARGET.indi_result_with_pid stored as parquet as select * from SOURCE.indi_result_with_pid orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_result_with_pid COMPUTE STATISTICS; + create table TARGET.indi_impact_measures stored as parquet as select * from SOURCE.indi_impact_measures orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_impact_measures COMPUTE STATISTICS; + create table TARGET.indi_pub_interdisciplinarity stored as parquet as select * from SOURCE.indi_pub_interdisciplinarity orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.indi_pub_interdisciplinarity COMPUTE STATISTICS; + create table TARGET.result_apc_affiliations stored as parquet as select * from SOURCE.result_apc_affiliations orig where exists (select 1 from TARGET.result r where r.id=orig.id); ---ANALYZE TABLE TARGET.result_apc_affiliations COMPUTE STATISTICS; -create table TARGET.indi_is_project_result_after stored as parquet as select * from SOURCE.indi_is_project_result_after orig where exists (select 1 from TARGET.result r where r.id=orig.id); -create table TARGET.indi_is_funder_plan_s stored as parquet as select * from SOURCE.indi_is_funder_plan_s orig where exists (select 1 from TARGET.result r where r.id=orig.id); + +create table TARGET.indi_is_project_result_after stored as parquet as select * from SOURCE.indi_is_project_result_after orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); +create view TARGET.indi_is_funder_plan_s as select * from SOURCE.indi_is_funder_plan_s; create view TARGET.indi_funder_fairness as select * from SOURCE.indi_funder_fairness; create view TARGET.indi_funder_openess as select * from SOURCE.indi_funder_openess; create view TARGET.indi_funder_findable as select * from SOURCE.indi_funder_findable; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs.sql index 9a9407c2d..4469782f0 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs.sql @@ -11,5 +11,4 @@ create table TARGET.result stored as parquet as join SOURCE.context cont on cont.id=cat.context -- join SOURCE.result where rc.id=r.id and conc.category like CONTEXT) -) foo; ---ANALYZE TABLE TARGET.result COMPUTE STATISTICS; \ No newline at end of file +) foo; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs_tail.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs_tail.sql index bad18efde..a28206d56 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs_tail.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_RIs_tail.sql @@ -11,5 +11,4 @@ create table TARGET.result stored as parquet as join SOURCE.context cont on cont.id=cat.context -- join SOURCE.result where rc.id=r.id and conc.category not in (CONTEXTS)) -) foo; ---ANALYZE TABLE TARGET.result COMPUTE STATISTICS; \ No newline at end of file +) foo; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_funded.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_funded.sql index b8d3c0242..ce6475c22 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_funded.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_funded.sql @@ -4,6 +4,4 @@ create database if not exists TARGET; create table TARGET.result stored as parquet as select distinct * from ( select * from SOURCE.result r where exists (select 1 from SOURCE.result_projects rp join SOURCE.project p on rp.project=p.id where rp.id=r.id) - ) foo; - ---ANALYZE TABLE TARGET.result COMPUTE STATISTICS; \ No newline at end of file + ) foo; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql index 7bfba92a8..d2f08b391 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql @@ -59,6 +59,4 @@ create table TARGET.result stored as parquet as 'openorgs____::5d55fb216b14691cf68218daf5d78cd9', -- Munster Technological University 'openorgs____::0fccc7640f0cb44d5cd1b06b312a06b9', -- Cardiff University 'openorgs____::8839b55dae0c84d56fd533f52d5d483a' -- Leibniz Institute of Ecological Urban and Regional Development - ))) foo; - ---ANALYZE TABLE TARGET.result COMPUTE STATISTICS; \ No newline at end of file + ))) foo; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql index b7e421813..2e6f0711c 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step21-createObservatoryDB.sql @@ -8,7 +8,6 @@ from ${stats_db_name}.result r group by rl.id ) rln on rln.id=r.id; ---ANALYZE TABLE ${observatory_db_name}.result_cc_licence COMPUTE STATISTICS; create table ${observatory_db_name}.result_affiliated_country stored as parquet as select @@ -39,7 +38,6 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, c.code, c.name; ---ANALYZE TABLE ${observatory_db_name}.result_affiliated_country COMPUTE STATISTICS; create table ${observatory_db_name}.result_affiliated_year stored as parquet as select @@ -70,7 +68,6 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year; ---ANALYZE TABLE ${observatory_db_name}.result_affiliated_year COMPUTE STATISTICS; create table ${observatory_db_name}.result_affiliated_year_country stored as parquet as select @@ -101,7 +98,6 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year, c.code, c.name; ---ANALYZE TABLE ${observatory_db_name}.result_affiliated_year_country COMPUTE STATISTICS; create table ${observatory_db_name}.result_affiliated_datasource stored as parquet as select @@ -134,8 +130,6 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name; ---ANALYZE TABLE ${observatory_db_name}.result_affiliated_datasource COMPUTE STATISTICS; - create table ${observatory_db_name}.result_affiliated_datasource_country stored as parquet as select count(distinct r.id) as total, @@ -167,8 +161,6 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name, c.code, c.name; ---ANALYZE TABLE ${observatory_db_name}.result_affiliated_datasource_country COMPUTE STATISTICS; - create table ${observatory_db_name}.result_affiliated_organization stored as parquet as select count(distinct r.id) as total, @@ -198,8 +190,6 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name; ---ANALYZE TABLE ${observatory_db_name}.result_affiliated_organization COMPUTE STATISTICS; - create table ${observatory_db_name}.result_affiliated_organization_country stored as parquet as select count(distinct r.id) as total, @@ -229,8 +219,6 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name, c.code, c.name; ---ANALYZE TABLE ${observatory_db_name}.result_affiliated_organization_country COMPUTE STATISTICS; - create table ${observatory_db_name}.result_affiliated_funder stored as parquet as select count(distinct r.id) as total, @@ -262,8 +250,6 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder; ---ANALYZE TABLE ${observatory_db_name}.result_affiliated_funder COMPUTE STATISTICS; - create table ${observatory_db_name}.result_affiliated_funder_country stored as parquet as select count(distinct r.id) as total, @@ -295,8 +281,6 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder, c.code, c.name; ---ANALYZE TABLE ${observatory_db_name}.result_affiliated_funder_country COMPUTE STATISTICS; - create table ${observatory_db_name}.result_deposited_country stored as parquet as select count(distinct r.id) as total, @@ -328,8 +312,6 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, c.code, c.name; ---ANALYZE TABLE ${observatory_db_name}.result_deposited_country COMPUTE STATISTICS; - create table ${observatory_db_name}.result_deposited_year stored as parquet as select count(distinct r.id) as total, @@ -361,7 +343,6 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year; ---ANALYZE TABLE ${observatory_db_name}.result_deposited_year COMPUTE STATISTICS; create table ${observatory_db_name}.result_deposited_year_country stored as parquet as select @@ -394,8 +375,6 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, r.year, c.code, c.name; ---ANALYZE TABLE ${observatory_db_name}.result_deposited_year_country COMPUTE STATISTICS; - create table ${observatory_db_name}.result_deposited_datasource stored as parquet as select count(distinct r.id) as total, @@ -427,8 +406,6 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name; ---ANALYZE TABLE ${observatory_db_name}.result_deposited_datasource COMPUTE STATISTICS; - create table ${observatory_db_name}.result_deposited_datasource_country stored as parquet as select count(distinct r.id) as total, @@ -460,8 +437,6 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, d.name, c.code, c.name; ---ANALYZE TABLE ${observatory_db_name}.result_deposited_datasource_country COMPUTE STATISTICS; - create table ${observatory_db_name}.result_deposited_organization stored as parquet as select count(distinct r.id) as total, @@ -493,8 +468,6 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name; ---ANALYZE TABLE ${observatory_db_name}.result_deposited_organization COMPUTE STATISTICS; - create table ${observatory_db_name}.result_deposited_organization_country stored as parquet as select count(distinct r.id) as total, @@ -526,8 +499,6 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, o.name, c.code, c.name; ---ANALYZE TABLE ${observatory_db_name}.result_deposited_organization_country COMPUTE STATISTICS; - create table ${observatory_db_name}.result_deposited_funder stored as parquet as select count(distinct r.id) as total, @@ -561,8 +532,6 @@ group by r.green, r.gold, case when rl.type is not null then true else false end case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder; ---ANALYZE TABLE ${observatory_db_name}.result_deposited_funder COMPUTE STATISTICS; - create table ${observatory_db_name}.result_deposited_funder_country stored as parquet as select count(distinct r.id) as total, @@ -595,5 +564,3 @@ from ${stats_db_name}.result r group by r.green, r.gold, case when rl.type is not null then true else false end, case when pids.pid is not null then true else false end, case when r.access_mode in ('Open Access', 'Open Source') then true else false end, r.peer_reviewed, r.type, abstract, cc_licence, r.authors > 1, rpc.count > 1, rfc.count > 1, p.funder, c.code, c.name; - ---ANALYZE TABLE ${observatory_db_name}.result_deposited_funder_country COMPUTE STATISTICS; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql index 1514ecf52..eb16a161e 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step7.sql @@ -123,6 +123,8 @@ UNION ALL SELECT * FROM ${stats_db_name}.otherresearchproduct_topics; +DROP TABLE IF EXISTS ${stats_db_name}.result_fos purge; + create table ${stats_db_name}.result_fos stored as parquet as with lvl1 as (select id, topic from ${stats_db_name}.result_topics where topic like '__ %' and type='Fields of Science and Technology classification'), @@ -133,6 +135,8 @@ from lvl1 join lvl2 on lvl1.id=lvl2.id and substr(lvl2.topic, 1, 2)=substr(lvl1.topic, 1, 2) join lvl3 on lvl3.id=lvl1.id and substr(lvl3.topic, 1, 4)=substr(lvl2.topic, 1, 4); +DROP TABLE IF EXISTS ${stats_db_name}.result_organization purge; + CREATE TABLE ${stats_db_name}.result_organization STORED AS PARQUET AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS organization FROM ${openaire_db_name}.relation r @@ -140,6 +144,8 @@ WHERE r.reltype = 'resultOrganization' and r.target like '50|%' and r.datainfo.deletedbyinference = false and r.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.result_projects purge; + CREATE TABLE ${stats_db_name}.result_projects STORED AS PARQUET AS select pr.result AS id, pr.id AS project, datediff(p.enddate, p.startdate) AS daysfromend, pr.provenance as provenance FROM ${stats_db_name}.result r diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql index 248716b36..07204db0c 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step8.sql @@ -5,6 +5,8 @@ -- Datasource table/view and Datasource related tables/views ------------------------------------------------------------ ------------------------------------------------------------ +DROP TABLE IF EXISTS ${stats_db_name}.datasource_tmp purge; + CREATE TABLE ${stats_db_name}.datasource_tmp ( `id` string, @@ -48,6 +50,7 @@ WHERE d1.datainfo.deletedbyinference = FALSE and d1.datainfo.invisible=false; -- Updating temporary table with everything that is not based on results -> This is done with the following "dual" table. -- Creating a temporary dual table that will be removed after the following insert + CREATE TABLE ${stats_db_name}.dual ( dummy CHAR(1)); INSERT INTO ${stats_db_name}.dual VALUES ('X'); @@ -74,16 +77,22 @@ DROP TABLE ${stats_db_name}.dual; UPDATE ${stats_db_name}.datasource_tmp SET name='Other' WHERE name = 'Unknown Repository'; UPDATE ${stats_db_name}.datasource_tmp SET yearofvalidation=null WHERE yearofvalidation = '-1'; +DROP TABLE IF EXISTS ${stats_db_name}.datasource_languages purge; + CREATE TABLE ${stats_db_name}.datasource_languages STORED AS PARQUET AS SELECT substr(d.id, 4) AS id, langs.languages AS language FROM ${openaire_db_name}.datasource d LATERAL VIEW explode(d.odlanguages.value) langs AS languages where d.datainfo.deletedbyinference=false and d.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.datasource_oids purge; + CREATE TABLE ${stats_db_name}.datasource_oids STORED AS PARQUET AS SELECT substr(d.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.datasource d LATERAL VIEW explode(d.originalid) oids AS ids where d.datainfo.deletedbyinference=false and d.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.datasource_organizations purge; + CREATE TABLE ${stats_db_name}.datasource_organizations STORED AS PARQUET AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS organization FROM ${openaire_db_name}.relation r @@ -91,6 +100,8 @@ WHERE r.reltype = 'datasourceOrganization' and r.datainfo.deletedbyinference = f -- datasource sources: -- where the datasource info have been collected from. +DROP TABLE IF EXISTS ${stats_db_name}.datasource_sources purge; + create table if not exists ${stats_db_name}.datasource_sources STORED AS PARQUET AS select substr(d.id, 4) as id, substr(cf.key, 4) as datasource from ${openaire_db_name}.datasource d lateral view explode(d.collectedfrom) cfrom as cf diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql index 3da36dfe5..19d301e27 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step9.sql @@ -3,6 +3,8 @@ -- Organization table/view and Organization related tables/views ---------------------------------------------------------------- ---------------------------------------------------------------- +DROP TABLE IF EXISTS ${stats_db_name}.organization purge; + CREATE TABLE IF NOT EXISTS ${stats_db_name}.organization STORED AS PARQUET AS SELECT substr(o.id, 4) as id, o.legalname.value as name, diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml index aa991730b..cbf97944d 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/workflow.xml @@ -114,6 +114,7 @@ ${wf:conf('resumeFrom') eq 'step21-createObservatoryDB'} ${wf:conf('resumeFrom') eq 'step21-createObservatoryDB-post'} ${wf:conf('resumeFrom') eq 'step22-copyDataToImpalaCluster'} + ${wf:conf('resumeFrom') eq 'step22a-createPDFsAggregated'} ${wf:conf('resumeFrom') eq 'step23-finalizeImpalaCluster'} ${wf:conf('resumeFrom') eq 'Step24-updateCache'} @@ -448,6 +449,22 @@ ${hadoop_user_name} copyDataToImpalaCluster.sh + + + + + + + ${jobTracker} + ${nameNode} + createPDFsAggregated.sh + + + ${stats_db_name} + ${monitor_db_name} + ${hadoop_user_name} + createPDFsAggregated.sh + From 11a1207f9c8476db4fc926f707915c3209720106 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 22 Nov 2023 12:22:14 +0100 Subject: [PATCH 23/60] [graph cleaning] applying coar based vocabularies in bulk --- .../dhp/oa/merge/GroupEntitiesSparkJob.java | 40 +++++- .../oaf/utils/GraphCleaningFunctions.java | 95 +++++++++++++ .../dhp/schema/oaf/utils/OafMapperUtils.java | 21 ++- .../group_graph_entities_parameters.json | 6 + .../raw/AbstractMdRecordToOafMapper.java | 63 +-------- .../group/GroupEntitiesSparkJobTest.java | 131 ++++++++++++++++-- .../raw/GenerateEntitiesApplicationTest.java | 5 +- .../dnetlib/dhp/oa/graph/raw/MappersTest.java | 28 ++-- .../dhp/oa/graph/group/dataset/dataset.json | 6 +- .../graph/group/publication/publication.json | 6 +- pom.xml | 2 +- 11 files changed, 294 insertions(+), 109 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/GroupEntitiesSparkJob.java b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/GroupEntitiesSparkJob.java index f5c8eea19..0225a5063 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/GroupEntitiesSparkJob.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/GroupEntitiesSparkJob.java @@ -21,10 +21,15 @@ import org.slf4j.LoggerFactory; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; import eu.dnetlib.dhp.schema.common.EntityType; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.OafEntity; +import eu.dnetlib.dhp.schema.oaf.utils.GraphCleaningFunctions; import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import scala.Tuple2; /** @@ -35,6 +40,12 @@ public class GroupEntitiesSparkJob { private static final Encoder OAFENTITY_KRYO_ENC = Encoders.kryo(OafEntity.class); + private ArgumentApplicationParser parser; + + public GroupEntitiesSparkJob(ArgumentApplicationParser parser) { + this.parser = parser; + } + public static void main(String[] args) throws Exception { String jsonConfiguration = IOUtils @@ -51,6 +62,17 @@ public class GroupEntitiesSparkJob { .orElse(Boolean.TRUE); log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + final String isLookupUrl = parser.get("isLookupUrl"); + log.info("isLookupUrl: {}", isLookupUrl); + + final ISLookUpService isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl); + + new GroupEntitiesSparkJob(parser).run(isSparkSessionManaged, isLookupService); + } + + public void run(Boolean isSparkSessionManaged, ISLookUpService isLookUpService) + throws ISLookUpException { + String graphInputPath = parser.get("graphInputPath"); log.info("graphInputPath: {}", graphInputPath); @@ -60,19 +82,21 @@ public class GroupEntitiesSparkJob { String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); - boolean filterInvisible = Boolean.valueOf(parser.get("filterInvisible")); + boolean filterInvisible = Boolean.parseBoolean(parser.get("filterInvisible")); log.info("filterInvisible: {}", filterInvisible); SparkConf conf = new SparkConf(); conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); conf.registerKryoClasses(ModelSupport.getOafModelClasses()); + final VocabularyGroup vocs = VocabularyGroup.loadVocsFromIS(isLookUpService); + runWithSparkSession( conf, isSparkSessionManaged, spark -> { HdfsSupport.remove(checkpointPath, spark.sparkContext().hadoopConfiguration()); - groupEntities(spark, graphInputPath, checkpointPath, outputPath, filterInvisible); + groupEntities(spark, graphInputPath, checkpointPath, outputPath, filterInvisible, vocs); }); } @@ -81,7 +105,7 @@ public class GroupEntitiesSparkJob { String inputPath, String checkpointPath, String outputPath, - boolean filterInvisible) { + boolean filterInvisible, VocabularyGroup vocs) { Dataset allEntities = spark.emptyDataset(OAFENTITY_KRYO_ENC); @@ -106,10 +130,14 @@ public class GroupEntitiesSparkJob { } Dataset groupedEntities = allEntities - .groupByKey((MapFunction) OafEntity::getId, Encoders.STRING()) - .reduceGroups((ReduceFunction) (b, a) -> OafMapperUtils.mergeEntities(b, a)) .map( - (MapFunction, Tuple2>) t -> new Tuple2( + (MapFunction) entity -> GraphCleaningFunctions + .applyCoarVocabularies(entity, vocs), + OAFENTITY_KRYO_ENC) + .groupByKey((MapFunction) OafEntity::getId, Encoders.STRING()) + .reduceGroups((ReduceFunction) OafMapperUtils::mergeEntities) + .map( + (MapFunction, Tuple2>) t -> new Tuple2<>( t._2().getClass().getName(), t._2()), Encoders.tuple(Encoders.STRING(), OAFENTITY_KRYO_ENC)); diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java index 324e3dd58..b23dade98 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java @@ -1,6 +1,8 @@ package eu.dnetlib.dhp.schema.oaf.utils; +import static eu.dnetlib.dhp.schema.common.ModelConstants.*; +import static eu.dnetlib.dhp.schema.common.ModelConstants.OPENAIRE_META_RESOURCE_TYPE; import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.getProvenance; import java.time.LocalDate; @@ -784,4 +786,97 @@ public class GraphCleaningFunctions extends CleaningFunctions { return s; } + public static OafEntity applyCoarVocabularies(OafEntity entity, VocabularyGroup vocs) { + + if (entity instanceof Result) { + final Result result = (Result) entity; + + Optional + .ofNullable(result.getInstance()) + .ifPresent( + instances -> instances + .forEach( + instance -> { + if (Objects.isNull(instance.getInstanceTypeMapping())) { + List mapping = Lists.newArrayList(); + mapping + .add( + OafMapperUtils + .instanceTypeMapping( + instance.getInstancetype().getClassname(), + OPENAIRE_COAR_RESOURCE_TYPES_3_1)); + instance.setInstanceTypeMapping(mapping); + } + Optional optionalItm = instance + .getInstanceTypeMapping() + .stream() + .filter(GraphCleaningFunctions::originalResourceType) + .findFirst(); + if (optionalItm.isPresent()) { + InstanceTypeMapping coarItm = optionalItm.get(); + Optional + .ofNullable( + vocs + .lookupTermBySynonym( + OPENAIRE_COAR_RESOURCE_TYPES_3_1, coarItm.getOriginalType())) + .ifPresent(type -> { + coarItm.setTypeCode(type.getClassid()); + coarItm.setTypeLabel(type.getClassname()); + }); + final List mappings = Lists.newArrayList(); + if (vocs.vocabularyExists(OPENAIRE_USER_RESOURCE_TYPES)) { + Optional + .ofNullable( + vocs + .lookupTermBySynonym( + OPENAIRE_USER_RESOURCE_TYPES, coarItm.getTypeCode())) + .ifPresent( + type -> mappings + .add( + OafMapperUtils + .instanceTypeMapping(coarItm.getTypeCode(), type))); + } + if (!mappings.isEmpty()) { + instance.getInstanceTypeMapping().addAll(mappings); + } + } + })); + result.setMetaResourceType(getMetaResourceType(result.getInstance(), vocs)); + } + + return entity; + } + + private static boolean originalResourceType(InstanceTypeMapping itm) { + return StringUtils.isNotBlank(itm.getOriginalType()) && + OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(itm.getVocabularyName()) && + StringUtils.isBlank(itm.getTypeCode()) && + StringUtils.isBlank(itm.getTypeLabel()); + } + + private static Qualifier getMetaResourceType(final List instances, final VocabularyGroup vocs) { + + if (vocs.vocabularyExists(OPENAIRE_META_RESOURCE_TYPE)) { + Optional instanceTypeMapping = instances + .stream() + .flatMap( + i -> Optional.ofNullable(i.getInstanceTypeMapping()).map(Collection::stream).orElse(Stream.empty())) + .filter(t -> OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(t.getVocabularyName())) + .findFirst(); + + if (!instanceTypeMapping.isPresent()) { + return null; + } else { + final String typeCode = instanceTypeMapping.get().getTypeCode(); + return Optional + .ofNullable(vocs.lookupTermBySynonym(OPENAIRE_META_RESOURCE_TYPE, typeCode)) + .orElseThrow( + () -> new IllegalStateException("unable to find a synonym for '" + typeCode + "' in " + + OPENAIRE_META_RESOURCE_TYPE)); + } + } else { + throw new IllegalStateException("vocabulary '" + OPENAIRE_META_RESOURCE_TYPE + "' not available"); + } + } + } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/OafMapperUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/OafMapperUtils.java index 1c557c805..4cecd0895 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/OafMapperUtils.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/OafMapperUtils.java @@ -140,15 +140,28 @@ public class OafMapperUtils { .collect(Collectors.toList()); } - public static InstanceTypeMapping instanceTypeMapping(String originalType, Qualifier term) { + public static InstanceTypeMapping instanceTypeMapping(String originalType, String code, String label, + String vocabularyName) { final InstanceTypeMapping m = new InstanceTypeMapping(); - m.setVocabularyName(term.getSchemeid()); + m.setVocabularyName(vocabularyName); m.setOriginalType(originalType); - m.setTypeCode(term.getClassid()); - m.setTypeLabel(term.getClassname()); + m.setTypeCode(code); + m.setTypeLabel(label); return m; } + public static InstanceTypeMapping instanceTypeMapping(String originalType, Qualifier term) { + return instanceTypeMapping(originalType, term.getClassid(), term.getClassname(), term.getSchemeid()); + } + + public static InstanceTypeMapping instanceTypeMapping(String originalType) { + return instanceTypeMapping(originalType, null, null, null); + } + + public static InstanceTypeMapping instanceTypeMapping(String originalType, String vocabularyName) { + return instanceTypeMapping(originalType, null, null, vocabularyName); + } + public static Qualifier unknown(final String schemeid, final String schemename) { return qualifier(UNKNOWN, "Unknown", schemeid, schemename); } diff --git a/dhp-common/src/main/resources/eu/dnetlib/dhp/oa/merge/group_graph_entities_parameters.json b/dhp-common/src/main/resources/eu/dnetlib/dhp/oa/merge/group_graph_entities_parameters.json index 58e3ca711..512878457 100644 --- a/dhp-common/src/main/resources/eu/dnetlib/dhp/oa/merge/group_graph_entities_parameters.json +++ b/dhp-common/src/main/resources/eu/dnetlib/dhp/oa/merge/group_graph_entities_parameters.json @@ -28,5 +28,11 @@ "paramLongName": "filterInvisible", "paramDescription": "if true filters out invisible entities", "paramRequired": true + }, + { + "paramName": "isu", + "paramLongName": "isLookupUrl", + "paramDescription": "url to the ISLookup Service", + "paramRequired": true } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java index 49133cedb..a71908b02 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/AbstractMdRecordToOafMapper.java @@ -125,9 +125,7 @@ public abstract class AbstractMdRecordToOafMapper { final String type = getResultType(doc, instances); - final Qualifier metaResourceType = getMetaResourceType(instances); - - return createOafs(doc, type, metaResourceType, instances, collectedFrom, entityInfo, lastUpdateTimestamp); + return createOafs(doc, type, instances, collectedFrom, entityInfo, lastUpdateTimestamp); } catch (DocumentException e) { log.error("Error with record:\n" + xml); return Lists.newArrayList(); @@ -153,30 +151,6 @@ public abstract class AbstractMdRecordToOafMapper { return type; } - protected Qualifier getMetaResourceType(final List instances) { - - if (vocs.vocabularyExists(OPENAIRE_META_RESOURCE_TYPE)) { - Optional instanceTypeMapping = instances - .stream() - .flatMap(i -> i.getInstanceTypeMapping().stream()) - .filter(t -> OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(t.getVocabularyName())) - .findFirst(); - - if (!instanceTypeMapping.isPresent()) { - return null; - } else { - final String typeCode = instanceTypeMapping.get().getTypeCode(); - return Optional - .ofNullable(vocs.lookupTermBySynonym(OPENAIRE_META_RESOURCE_TYPE, typeCode)) - .orElseThrow( - () -> new IllegalStateException("unable to find a synonym for '" + typeCode + "' in " + - OPENAIRE_META_RESOURCE_TYPE)); - } - } else { - throw new IllegalStateException("vocabulary '" + OPENAIRE_META_RESOURCE_TYPE + "' not available"); - } - } - private KeyValue getProvenanceDatasource(final Document doc, final String xpathId, final String xpathName) { final String dsId = doc.valueOf(xpathId); final String dsName = doc.valueOf(xpathName); @@ -191,14 +165,13 @@ public abstract class AbstractMdRecordToOafMapper { protected List createOafs( final Document doc, final String type, - final Qualifier metaResourceType, final List instances, final KeyValue collectedFrom, final DataInfo info, final long lastUpdateTimestamp) { final OafEntity entity = createEntity( - doc, type, metaResourceType, instances, collectedFrom, info, lastUpdateTimestamp); + doc, type, instances, collectedFrom, info, lastUpdateTimestamp); final Set originalId = Sets.newHashSet(entity.getOriginalId()); originalId.add(entity.getId()); @@ -231,7 +204,6 @@ public abstract class AbstractMdRecordToOafMapper { private OafEntity createEntity(final Document doc, final String type, - final Qualifier metaResourceType, final List instances, final KeyValue collectedFrom, final DataInfo info, @@ -239,12 +211,12 @@ public abstract class AbstractMdRecordToOafMapper { switch (type.toLowerCase()) { case "publication": final Publication p = new Publication(); - populateResultFields(p, metaResourceType, doc, instances, collectedFrom, info, lastUpdateTimestamp); + populateResultFields(p, doc, instances, collectedFrom, info, lastUpdateTimestamp); p.setJournal(prepareJournal(doc, info)); return p; case "dataset": final Dataset d = new Dataset(); - populateResultFields(d, metaResourceType, doc, instances, collectedFrom, info, lastUpdateTimestamp); + populateResultFields(d, doc, instances, collectedFrom, info, lastUpdateTimestamp); d.setStoragedate(prepareDatasetStorageDate(doc, info)); d.setDevice(prepareDatasetDevice(doc, info)); d.setSize(prepareDatasetSize(doc, info)); @@ -255,7 +227,7 @@ public abstract class AbstractMdRecordToOafMapper { return d; case "software": final Software s = new Software(); - populateResultFields(s, metaResourceType, doc, instances, collectedFrom, info, lastUpdateTimestamp); + populateResultFields(s, doc, instances, collectedFrom, info, lastUpdateTimestamp); s.setDocumentationUrl(prepareSoftwareDocumentationUrls(doc, info)); s.setLicense(prepareSoftwareLicenses(doc, info)); s.setCodeRepositoryUrl(prepareSoftwareCodeRepositoryUrl(doc, info)); @@ -265,7 +237,7 @@ public abstract class AbstractMdRecordToOafMapper { case "otherresearchproducts": default: final OtherResearchProduct o = new OtherResearchProduct(); - populateResultFields(o, metaResourceType, doc, instances, collectedFrom, info, lastUpdateTimestamp); + populateResultFields(o, doc, instances, collectedFrom, info, lastUpdateTimestamp); o.setContactperson(prepareOtherResearchProductContactPersons(doc, info)); o.setContactgroup(prepareOtherResearchProductContactGroups(doc, info)); o.setTool(prepareOtherResearchProductTools(doc, info)); @@ -402,13 +374,11 @@ public abstract class AbstractMdRecordToOafMapper { private void populateResultFields( final Result r, - final Qualifier metaResourceType, final Document doc, final List instances, final KeyValue collectedFrom, final DataInfo info, final long lastUpdateTimestamp) { - r.setMetaResourceType(metaResourceType); r.setDataInfo(info); r.setLastupdatetimestamp(lastUpdateTimestamp); r.setId(createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false)); @@ -555,26 +525,7 @@ public abstract class AbstractMdRecordToOafMapper { .ofNullable(findOriginalType(doc)) .map(originalType -> { final List mappings = Lists.newArrayList(); - - if (vocs.vocabularyExists(OPENAIRE_COAR_RESOURCE_TYPES_3_1)) { - - // TODO verify what the vocabs return when a synonym is not defined - Optional - .ofNullable(vocs.lookupTermBySynonym(OPENAIRE_COAR_RESOURCE_TYPES_3_1, originalType)) - .ifPresent(coarTerm -> { - mappings.add(OafMapperUtils.instanceTypeMapping(originalType, coarTerm)); - if (vocs.vocabularyExists(OPENAIRE_USER_RESOURCE_TYPES)) { - - // TODO verify what the vocabs return when a synonym is not defined - Optional - .ofNullable( - vocs.lookupTermBySynonym(OPENAIRE_USER_RESOURCE_TYPES, coarTerm.getClassid())) - .ifPresent( - type -> mappings.add(OafMapperUtils.instanceTypeMapping(originalType, type))); - } - }); - } - + mappings.add(OafMapperUtils.instanceTypeMapping(originalType, OPENAIRE_COAR_RESOURCE_TYPES_3_1)); return mappings; }) .orElse(new ArrayList<>()); diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/group/GroupEntitiesSparkJobTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/group/GroupEntitiesSparkJobTest.java index 0887adf45..242aed88e 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/group/GroupEntitiesSparkJobTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/group/GroupEntitiesSparkJobTest.java @@ -1,15 +1,23 @@ package eu.dnetlib.dhp.oa.graph.group; +import static eu.dnetlib.dhp.schema.common.ModelConstants.*; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.lenient; import java.io.IOException; import java.net.URISyntaxException; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import org.apache.commons.cli.ParseException; import org.apache.commons.io.FileUtils; +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.MapFunction; @@ -17,20 +25,36 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; import org.junit.jupiter.api.*; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.common.HdfsSupport; +import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; import eu.dnetlib.dhp.oa.merge.GroupEntitiesSparkJob; +import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.Instance; +import eu.dnetlib.dhp.schema.oaf.InstanceTypeMapping; import eu.dnetlib.dhp.schema.oaf.OafEntity; import eu.dnetlib.dhp.schema.oaf.Result; import eu.dnetlib.dhp.utils.DHPUtils; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +@ExtendWith(MockitoExtension.class) @TestMethodOrder(MethodOrderer.OrderAnnotation.class) public class GroupEntitiesSparkJobTest { + @Mock + private ISLookUpService isLookUpService; + + private VocabularyGroup vocabularies; + private static SparkSession spark; private static ObjectMapper mapper = new ObjectMapper() @@ -45,10 +69,10 @@ public class GroupEntitiesSparkJobTest { @BeforeAll public static void beforeAll() throws IOException { - workingDir = Files.createTempDirectory(GroupEntitiesSparkJob.class.getSimpleName()); + workingDir = Files.createTempDirectory(GroupEntitiesSparkJobTest.class.getSimpleName()); SparkConf conf = new SparkConf(); - conf.setAppName(GroupEntitiesSparkJob.class.getSimpleName()); + conf.setAppName(GroupEntitiesSparkJobTest.class.getSimpleName()); conf.setMaster("local"); conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); conf.registerKryoClasses(ModelSupport.getOafModelClasses()); @@ -56,10 +80,17 @@ public class GroupEntitiesSparkJobTest { } @BeforeEach - public void beforeEach() throws IOException, URISyntaxException { + public void beforeEach() throws IOException, URISyntaxException, ISLookUpException { dataInputPath = Paths.get(ClassLoader.getSystemResource("eu/dnetlib/dhp/oa/graph/group").toURI()); checkpointPath = workingDir.resolve("grouped_entity"); outputPath = workingDir.resolve("dispatched_entity"); + + lenient().when(isLookUpService.quickSearchProfile(VocabularyGroup.VOCABULARIES_XQUERY)).thenReturn(vocs()); + lenient() + .when(isLookUpService.quickSearchProfile(VocabularyGroup.VOCABULARY_SYNONYMS_XQUERY)) + .thenReturn(synonyms()); + + vocabularies = VocabularyGroup.loadVocsFromIS(isLookUpService); } @AfterAll @@ -71,18 +102,17 @@ public class GroupEntitiesSparkJobTest { @Test @Order(1) void testGroupEntities() throws Exception { - GroupEntitiesSparkJob.main(new String[] { - "-isSparkSessionManaged", - Boolean.FALSE.toString(), - "-graphInputPath", - dataInputPath.toString(), - "-checkpointPath", - checkpointPath.toString(), - "-outputPath", - outputPath.toString(), - "-filterInvisible", - Boolean.FALSE.toString() - }); + new GroupEntitiesSparkJob( + args( + "/eu/dnetlib/dhp/oa/merge/group_graph_entities_parameters.json", + new String[] { + "--isSparkSessionManaged", Boolean.FALSE.toString(), + "--graphInputPath", dataInputPath.toString(), + "--checkpointPath", checkpointPath.toString(), + "--outputPath", outputPath.toString(), + "--filterInvisible", Boolean.FALSE.toString(), + "--isLookupUrl", "lookupurl" + })).run(false, isLookUpService); Dataset checkpointTable = spark .read() @@ -109,6 +139,14 @@ public class GroupEntitiesSparkJobTest { .map((MapFunction) s -> mapper.readValue(s, Result.class), Encoders.bean(Result.class)); assertEquals(3, output.count()); + + List resultTypes = output + .map((MapFunction) value -> value.getResulttype().getClassid(), Encoders.STRING()) + .distinct() + .collectAsList(); + + assertEquals(2, resultTypes.size()); + assertEquals( 2, output @@ -121,5 +159,68 @@ public class GroupEntitiesSparkJobTest { .map((MapFunction) r -> r.getResulttype().getClassid(), Encoders.STRING()) .filter((FilterFunction) s -> s.equals("dataset")) .count()); + + Result result = output + .filter("id = '50|doi_________::09821844208a5cd6300b2bfb13bca1b9'") + .first(); + + result.getInstance().forEach(instance -> { + Optional coarType = instance + .getInstanceTypeMapping() + .stream() + .filter(itm -> OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(itm.getVocabularyName())) + .filter(itm -> "journal-article".equals(itm.getOriginalType())) + .findFirst(); + + assertTrue(coarType.isPresent()); + assertEquals("http://purl.org/coar/resource_type/c_2df8fbb1", coarType.get().getTypeCode()); + assertEquals("research article", coarType.get().getTypeLabel()); + }); + + final Dataset filtered = output.filter("id = '50|DansKnawCris::203a27996ddc0fd1948258e5b7dec61c'"); + assertEquals(1, filtered.count()); + result = filtered.first(); + + result + .getInstance() + .stream() + .flatMap(instance -> instance.getInstanceTypeMapping().stream()) + .filter(itm -> OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(itm.getVocabularyName())) + .filter(itm -> "Patent".equals(itm.getOriginalType())) + .forEach(itm -> { + assertEquals("http://purl.org/coar/resource_type/c_15cd", itm.getTypeCode()); + assertEquals("patent", itm.getTypeLabel()); + }); } + + private List vocs() throws IOException { + return IOUtils + .readLines( + Objects + .requireNonNull( + getClass().getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/terms.txt"))); + } + + private List synonyms() throws IOException { + return IOUtils + .readLines( + Objects + .requireNonNull( + getClass().getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/synonyms.txt"))); + } + + private ArgumentApplicationParser args(String paramSpecs, String[] args) throws IOException, ParseException { + ArgumentApplicationParser parser = new ArgumentApplicationParser(classPathResourceAsString(paramSpecs)); + parser.parseArgument(args); + return parser; + } + + private static String classPathResourceAsString(String path) throws IOException { + return IOUtils + .toString( + Objects + .requireNonNull( + GroupEntitiesSparkJobTest.class.getResourceAsStream(path))); + } + } diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplicationTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplicationTest.java index 53b3f8432..6d6b2ffbd 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplicationTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplicationTest.java @@ -92,13 +92,14 @@ class GenerateEntitiesApplicationTest { private List vocs() throws IOException { return IOUtils .readLines( - GraphCleaningFunctionsTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/terms.txt")); + GenerateEntitiesApplicationTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/terms.txt")); } private List synonyms() throws IOException { return IOUtils .readLines( - GraphCleaningFunctionsTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/synonyms.txt")); + GenerateEntitiesApplicationTest.class + .getResourceAsStream("/eu/dnetlib/dhp/oa/graph/clean/synonyms.txt")); } } diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java index e6997ac1d..1f2defa01 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/MappersTest.java @@ -122,7 +122,7 @@ class MappersTest { assertTrue(instance.getPid().isEmpty()); assertNotNull(instance.getInstanceTypeMapping()); - assertEquals(2, instance.getInstanceTypeMapping().size()); + assertEquals(1, instance.getInstanceTypeMapping().size()); Optional coarType = instance .getInstanceTypeMapping() @@ -131,8 +131,8 @@ class MappersTest { .findFirst(); assertTrue(coarType.isPresent()); - assertEquals("http://purl.org/coar/resource_type/c_5794", coarType.get().getTypeCode()); - assertEquals("conference paper", coarType.get().getTypeLabel()); + assertNull(coarType.get().getTypeCode()); + assertNull(coarType.get().getTypeLabel()); Optional userType = instance .getInstanceTypeMapping() @@ -140,9 +140,7 @@ class MappersTest { .filter(itm -> ModelConstants.OPENAIRE_USER_RESOURCE_TYPES.equals(itm.getVocabularyName())) .findFirst(); - assertTrue(userType.isPresent()); - assertEquals("Article", userType.get().getTypeCode()); - assertEquals("Article", userType.get().getTypeLabel()); + assertFalse(userType.isPresent()); assertFalse(instance.getAlternateIdentifier().isEmpty()); assertEquals("doi", instance.getAlternateIdentifier().get(0).getQualifier().getClassid()); @@ -710,14 +708,10 @@ class MappersTest { assertEquals("0001", p_cleaned.getInstance().get(0).getRefereed().getClassid()); assertEquals("peerReviewed", p_cleaned.getInstance().get(0).getRefereed().getClassname()); - assertNotNull(p_cleaned.getMetaResourceType()); - assertEquals("Research Literature", p_cleaned.getMetaResourceType().getClassid()); - assertEquals("Research Literature", p_cleaned.getMetaResourceType().getClassname()); - assertEquals(ModelConstants.OPENAIRE_META_RESOURCE_TYPE, p_cleaned.getMetaResourceType().getSchemeid()); - assertEquals(ModelConstants.OPENAIRE_META_RESOURCE_TYPE, p_cleaned.getMetaResourceType().getSchemename()); + assertNull(p_cleaned.getMetaResourceType()); assertNotNull(p_cleaned.getInstance().get(0).getInstanceTypeMapping()); - assertEquals(2, p_cleaned.getInstance().get(0).getInstanceTypeMapping().size()); + assertEquals(1, p_cleaned.getInstance().get(0).getInstanceTypeMapping().size()); assertTrue( p_cleaned @@ -728,8 +722,7 @@ class MappersTest { .anyMatch( t -> "journal-article".equals(t.getOriginalType()) && ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(t.getVocabularyName()) && - "http://purl.org/coar/resource_type/c_2df8fbb1".equals(t.getTypeCode()) && - "research article".equals(t.getTypeLabel()))); + Objects.isNull(t.getTypeCode()) && Objects.isNull(t.getTypeLabel()))); assertTrue( p_cleaned @@ -737,11 +730,8 @@ class MappersTest { .get(0) .getInstanceTypeMapping() .stream() - .anyMatch( - t -> "journal-article".equals(t.getOriginalType()) && - ModelConstants.OPENAIRE_USER_RESOURCE_TYPES.equals(t.getVocabularyName()) && - "Article".equals(t.getTypeCode()) && - "Article".equals(t.getTypeLabel()))); + .noneMatch( + t -> ModelConstants.OPENAIRE_USER_RESOURCE_TYPES.equals(t.getVocabularyName()))); } @Test diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/group/dataset/dataset.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/group/dataset/dataset.json index e30be47e9..efbabb3c8 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/group/dataset/dataset.json +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/group/dataset/dataset.json @@ -1,3 +1,3 @@ -{"author":[{"affiliation":[],"fullname":"Greenough, B","name":"B","pid":[],"rank":1,"surname":"Greenough"}],"bestaccessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|opendoar____::358aee4cc897452c00244351e4d91f69","value":"Zenodo"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":true,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofcollection":"2021-09-25T10:55:00.639Z","dateoftransformation":"2021-09-25T11:00:04.201Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Heritage Education"}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"geolocation":[],"id":"50|doi_________::09821844208a5cd6300b2bfb13bca1b9","instance":[{"accessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"urn","classname":"urn","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"urn:nbn:nl:ui:13-59-cjhf"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.17632/96bpgw5j9d.1"}],"collectedfrom":{"key":"10|opendoar____::358aee4cc897452c00244351e4d91f69","value":"Zenodo"},"hostedby":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"instancetype":{"classid":"0021","classname":"Dataset","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"alternateIdentifier":[],"refereed":{"classid":"0000","classname":"Unknown","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["","http://dx.doi.org/10.17632/96bpgw5j9d.1"]}],"language":{"classid":"und","classname":"Undetermined","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1635434801681,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"http%3A%2F%2Fservices.nod.dans.knaw.nl%2Foa-cerif","datestamp":"2021-08-16T15:29:45Z","harvestDate":"2021-09-25T10:55:00.639Z","identifier":"oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:211323","metadataNamespace":""}},"originalId":["50|DansKnawCris::09821844208a5cd6300b2bfb13bca1b9","oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:211323"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"0021","classname":"0021","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Interdisciplinary sciences"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Interdisciplinary sciences"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Heritage Education"}]} -{"author":[{"affiliation":[],"fullname":"Keijers, D.M.G.","name":"D.M.G.","pid":[],"rank":1,"surname":"Keijers"}],"bestaccessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":true,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofcollection":"2021-09-25T10:41:59.767Z","dateoftransformation":"2021-09-25T11:00:19.238Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"onderzoeksrapport"}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"geolocation":[],"id":"50|DansKnawCris::0dd644304b7116e8e58da3a5e3adc37a","instance":[{"accessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"urn","classname":"urn","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"urn:nbn:nl:ui:13-das-fkq"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.17026/dans-xsw-qtnx"}],"collectedfrom":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"hostedby":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"instancetype":{"classid":"0021","classname":"Dataset","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"alternateIdentifier":[],"refereed":{"classid":"0000","classname":"Unknown","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["","http://dx.doi.org/10.17026/dans-xsw-qtnx"]}],"language":{"classid":"dut/nld","classname":"Dutch; Flemish","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1635434847381,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"http%3A%2F%2Fservices.nod.dans.knaw.nl%2Foa-cerif","datestamp":"2021-08-16T13:53:29Z","harvestDate":"2021-09-25T10:41:59.767Z","identifier":"oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:20759","metadataNamespace":""}},"originalId":["oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:20759","50|DansKnawCris::0dd644304b7116e8e58da3a5e3adc37a"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"0021","classname":"0021","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"PROSPECTIE"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Archaeology"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Plangebied Lange Ekker te Vessem, gemeente Eersel"}]} -{"author":[],"bestaccessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofcollection":"2021-09-25T10:43:13.768Z","dateoftransformation":"2021-09-25T11:01:22.863Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"This find is registered at Portable Antiquities of the Netherlands with number PAN-00054604"}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"geolocation":[],"id":"50|DansKnawCris::203a27996ddc0fd1948258e5b7dec61c","instance":[{"accessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"urn","classname":"urn","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"urn:nbn:nl:ui:13-a7-hwgy"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.17026/dans-x3z-fsq5"}],"collectedfrom":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"hostedby":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"instancetype":{"classid":"0021","classname":"Dataset","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"alternateIdentifier":[],"refereed":{"classid":"0000","classname":"Unknown","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["","http://dx.doi.org/10.17026/dans-x3z-fsq5"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1635434508886,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"http%3A%2F%2Fservices.nod.dans.knaw.nl%2Foa-cerif","datestamp":"2021-08-16T14:01:37Z","harvestDate":"2021-09-25T10:43:13.768Z","identifier":"oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:129566","metadataNamespace":""}},"originalId":["oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:129566","50|DansKnawCris::203a27996ddc0fd1948258e5b7dec61c"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"0021","classname":"0021","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"early medieval enamelled disc brooch variant A9"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Broader Match: disc brooches"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Broader Match: schijffibula - geemailleerd"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"metal"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"copper alloy"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Temporal coverage: Early Middle Ages C"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Temporal coverage: Early Middle Ages D"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Temporal coverage: 800 until 1000"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Archaeology"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"PAN-00054604 - early medieval enamelled disc brooch variant A9"}]} \ No newline at end of file +{"author":[{"affiliation":[],"fullname":"Greenough, B","name":"B","pid":[],"rank":1,"surname":"Greenough"}],"bestaccessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|opendoar____::358aee4cc897452c00244351e4d91f69","value":"Zenodo"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":true,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofcollection":"2021-09-25T10:55:00.639Z","dateoftransformation":"2021-09-25T11:00:04.201Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Heritage Education"}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"geolocation":[],"id":"50|doi_________::09821844208a5cd6300b2bfb13bca1b9","instance":[{"instanceTypeMapping":[{"originalType":"journal-article","vocabularyName":"openaire::coar_resource_types_3_1"}],"accessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"urn","classname":"urn","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"urn:nbn:nl:ui:13-59-cjhf"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.17632/96bpgw5j9d.1"}],"collectedfrom":{"key":"10|opendoar____::358aee4cc897452c00244351e4d91f69","value":"Zenodo"},"hostedby":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"instancetype":{"classid":"0021","classname":"Dataset","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"alternateIdentifier":[],"refereed":{"classid":"0000","classname":"Unknown","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["","http://dx.doi.org/10.17632/96bpgw5j9d.1"]}],"language":{"classid":"und","classname":"Undetermined","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1635434801681,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"http%3A%2F%2Fservices.nod.dans.knaw.nl%2Foa-cerif","datestamp":"2021-08-16T15:29:45Z","harvestDate":"2021-09-25T10:55:00.639Z","identifier":"oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:211323","metadataNamespace":""}},"originalId":["50|DansKnawCris::09821844208a5cd6300b2bfb13bca1b9","oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:211323"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"0021","classname":"0021","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Interdisciplinary sciences"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Interdisciplinary sciences"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Heritage Education"}]} +{"author":[{"affiliation":[],"fullname":"Keijers, D.M.G.","name":"D.M.G.","pid":[],"rank":1,"surname":"Keijers"}],"bestaccessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":true,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofcollection":"2021-09-25T10:41:59.767Z","dateoftransformation":"2021-09-25T11:00:19.238Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"onderzoeksrapport"}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"geolocation":[],"id":"50|DansKnawCris::0dd644304b7116e8e58da3a5e3adc37a","instance":[{"instanceTypeMapping":[{"originalType":"journal-article","vocabularyName":"openaire::coar_resource_types_3_1"}],"accessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"urn","classname":"urn","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"urn:nbn:nl:ui:13-das-fkq"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.17026/dans-xsw-qtnx"}],"collectedfrom":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"hostedby":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"instancetype":{"classid":"0021","classname":"Dataset","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"alternateIdentifier":[],"refereed":{"classid":"0000","classname":"Unknown","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["","http://dx.doi.org/10.17026/dans-xsw-qtnx"]}],"language":{"classid":"dut/nld","classname":"Dutch; Flemish","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1635434847381,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"http%3A%2F%2Fservices.nod.dans.knaw.nl%2Foa-cerif","datestamp":"2021-08-16T13:53:29Z","harvestDate":"2021-09-25T10:41:59.767Z","identifier":"oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:20759","metadataNamespace":""}},"originalId":["oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:20759","50|DansKnawCris::0dd644304b7116e8e58da3a5e3adc37a"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"0021","classname":"0021","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"PROSPECTIE"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Archaeology"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Plangebied Lange Ekker te Vessem, gemeente Eersel"}]} +{"author":[],"bestaccessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofcollection":"2021-09-25T10:43:13.768Z","dateoftransformation":"2021-09-25T11:01:22.863Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"This find is registered at Portable Antiquities of the Netherlands with number PAN-00054604"}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"geolocation":[],"id":"50|DansKnawCris::203a27996ddc0fd1948258e5b7dec61c","instance":[{"instanceTypeMapping":[{"originalType":"journal-article","vocabularyName":"openaire::coar_resource_types_3_1"}],"accessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"urn","classname":"urn","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"urn:nbn:nl:ui:13-a7-hwgy"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.17026/dans-x3z-fsq5"}],"collectedfrom":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"hostedby":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"instancetype":{"classid":"0021","classname":"Dataset","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"alternateIdentifier":[],"refereed":{"classid":"0000","classname":"Unknown","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["","http://dx.doi.org/10.17026/dans-x3z-fsq5"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1635434508886,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"http%3A%2F%2Fservices.nod.dans.knaw.nl%2Foa-cerif","datestamp":"2021-08-16T14:01:37Z","harvestDate":"2021-09-25T10:43:13.768Z","identifier":"oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:129566","metadataNamespace":""}},"originalId":["oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:129566","50|DansKnawCris::203a27996ddc0fd1948258e5b7dec61c"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"0021","classname":"0021","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"dataset","classname":"dataset","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"early medieval enamelled disc brooch variant A9"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Broader Match: disc brooches"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Broader Match: schijffibula - geemailleerd"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"metal"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"copper alloy"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Temporal coverage: Early Middle Ages C"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Temporal coverage: Early Middle Ages D"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Temporal coverage: 800 until 1000"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Archaeology"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"PAN-00054604 - early medieval enamelled disc brooch variant A9"}]} \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/group/publication/publication.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/group/publication/publication.json index 29ce76df3..90cf4936b 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/group/publication/publication.json +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/group/publication/publication.json @@ -1,3 +1,3 @@ -{"author":[{"affiliation":[],"fullname":"Greenough, B","name":"B","pid":[],"rank":1,"surname":"Greenough"}],"bestaccessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::9e3be59865b2c1c335d32dae2fe7b254","value":"Datacite"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":true,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofcollection":"2021-09-25T10:55:00.639Z","dateoftransformation":"2021-09-25T11:00:04.201Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Heritage Education"}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"geolocation":[],"id":"50|doi_________::09821844208a5cd6300b2bfb13bca1b9","instance":[{"accessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"urn","classname":"urn","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"urn:nbn:nl:ui:13-59-cjhf"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.17632/96bpgw5j9d.1"}],"collectedfrom":{"key":"10|openaire____::9e3be59865b2c1c335d32dae2fe7b254","value":"Datacite"},"hostedby":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"instancetype":{"classid":"0021","classname":"Dataset","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"alternateIdentifier":[],"refereed":{"classid":"0000","classname":"Unknown","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["","http://dx.doi.org/10.17632/96bpgw5j9d.1"]}],"language":{"classid":"und","classname":"Undetermined","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1635434801681,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"http%3A%2F%2Fservices.nod.dans.knaw.nl%2Foa-cerif","datestamp":"2021-08-16T15:29:45Z","harvestDate":"2021-09-25T10:55:00.639Z","identifier":"oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:211323","metadataNamespace":""}},"originalId":["50|DansKnawCris::09821844208a5cd6300b2bfb13bca1b9","oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:211323"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"0021","classname":"0021","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Interdisciplinary sciences"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Interdisciplinary sciences"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Heritage Education"}]} -{"author":[{"affiliation":[],"fullname":"Keijers, D.M.G.","name":"D.M.G.","pid":[],"rank":1,"surname":"Keijers"}],"bestaccessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":true,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofcollection":"2021-09-25T10:41:59.767Z","dateoftransformation":"2021-09-25T11:00:19.238Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"onderzoeksrapport"}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"geolocation":[],"id":"50|DansKnawCris::0dd644304b7116e8e58da3a5e3adc37a","instance":[{"accessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"urn","classname":"urn","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"urn:nbn:nl:ui:13-das-fkq"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.17026/dans-xsw-qtnx"}],"collectedfrom":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"hostedby":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"instancetype":{"classid":"0021","classname":"Dataset","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"alternateIdentifier":[],"refereed":{"classid":"0000","classname":"Unknown","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["","http://dx.doi.org/10.17026/dans-xsw-qtnx"]}],"language":{"classid":"dut/nld","classname":"Dutch; Flemish","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1635434847381,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"http%3A%2F%2Fservices.nod.dans.knaw.nl%2Foa-cerif","datestamp":"2021-08-16T13:53:29Z","harvestDate":"2021-09-25T10:41:59.767Z","identifier":"oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:20759","metadataNamespace":""}},"originalId":["oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:20759","50|DansKnawCris::0dd644304b7116e8e58da3a5e3adc37a"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"0021","classname":"0021","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"PROSPECTIE"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Archaeology"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Plangebied Lange Ekker te Vessem, gemeente Eersel"}]} -{"author":[],"bestaccessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofcollection":"2021-09-25T10:43:13.768Z","dateoftransformation":"2021-09-25T11:01:22.863Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"This find is registered at Portable Antiquities of the Netherlands with number PAN-00054604"}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"geolocation":[],"id":"50|DansKnawCris::203a27996ddc0fd1948258e5b7dec61c","instance":[{"accessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"urn","classname":"urn","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"urn:nbn:nl:ui:13-a7-hwgy"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.17026/dans-x3z-fsq5"}],"collectedfrom":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"hostedby":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"instancetype":{"classid":"0021","classname":"Dataset","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"alternateIdentifier":[],"refereed":{"classid":"0000","classname":"Unknown","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["","http://dx.doi.org/10.17026/dans-x3z-fsq5"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1635434508886,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"http%3A%2F%2Fservices.nod.dans.knaw.nl%2Foa-cerif","datestamp":"2021-08-16T14:01:37Z","harvestDate":"2021-09-25T10:43:13.768Z","identifier":"oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:129566","metadataNamespace":""}},"originalId":["oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:129566","50|DansKnawCris::203a27996ddc0fd1948258e5b7dec61c"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"0021","classname":"0021","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"early medieval enamelled disc brooch variant A9"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Broader Match: disc brooches"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Broader Match: schijffibula - geemailleerd"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"metal"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"copper alloy"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Temporal coverage: Early Middle Ages C"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Temporal coverage: Early Middle Ages D"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Temporal coverage: 800 until 1000"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Archaeology"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"PAN-00054604 - early medieval enamelled disc brooch variant A9"}]} \ No newline at end of file +{"author":[{"affiliation":[],"fullname":"Greenough, B","name":"B","pid":[],"rank":1,"surname":"Greenough"}],"bestaccessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::9e3be59865b2c1c335d32dae2fe7b254","value":"Datacite"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":true,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofcollection":"2021-09-25T10:55:00.639Z","dateoftransformation":"2021-09-25T11:00:04.201Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Heritage Education"}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"geolocation":[],"id":"50|doi_________::09821844208a5cd6300b2bfb13bca1b9","instance":[{"instanceTypeMapping":[{"originalType":"journal-article","vocabularyName":"openaire::coar_resource_types_3_1"}],"accessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"urn","classname":"urn","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"urn:nbn:nl:ui:13-59-cjhf"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.17632/96bpgw5j9d.1"}],"collectedfrom":{"key":"10|openaire____::9e3be59865b2c1c335d32dae2fe7b254","value":"Datacite"},"hostedby":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"instancetype":{"classid":"0021","classname":"Dataset","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"alternateIdentifier":[],"refereed":{"classid":"0000","classname":"Unknown","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["","http://dx.doi.org/10.17632/96bpgw5j9d.1"]}],"language":{"classid":"und","classname":"Undetermined","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1635434801681,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"http%3A%2F%2Fservices.nod.dans.knaw.nl%2Foa-cerif","datestamp":"2021-08-16T15:29:45Z","harvestDate":"2021-09-25T10:55:00.639Z","identifier":"oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:211323","metadataNamespace":""}},"originalId":["50|DansKnawCris::09821844208a5cd6300b2bfb13bca1b9","oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:211323"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"0021","classname":"0021","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Interdisciplinary sciences"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Interdisciplinary sciences"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Heritage Education"}]} +{"author":[{"affiliation":[],"fullname":"Keijers, D.M.G.","name":"D.M.G.","pid":[],"rank":1,"surname":"Keijers"}],"bestaccessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":true,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofcollection":"2021-09-25T10:41:59.767Z","dateoftransformation":"2021-09-25T11:00:19.238Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"onderzoeksrapport"}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"geolocation":[],"id":"50|DansKnawCris::0dd644304b7116e8e58da3a5e3adc37a","instance":[{"instanceTypeMapping":[{"originalType":"journal-article","vocabularyName":"openaire::coar_resource_types_3_1"}],"accessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"urn","classname":"urn","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"urn:nbn:nl:ui:13-das-fkq"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.17026/dans-xsw-qtnx"}],"collectedfrom":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"hostedby":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"instancetype":{"classid":"0021","classname":"Dataset","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"alternateIdentifier":[],"refereed":{"classid":"0000","classname":"Unknown","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["","http://dx.doi.org/10.17026/dans-xsw-qtnx"]}],"language":{"classid":"dut/nld","classname":"Dutch; Flemish","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1635434847381,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"http%3A%2F%2Fservices.nod.dans.knaw.nl%2Foa-cerif","datestamp":"2021-08-16T13:53:29Z","harvestDate":"2021-09-25T10:41:59.767Z","identifier":"oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:20759","metadataNamespace":""}},"originalId":["oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:20759","50|DansKnawCris::0dd644304b7116e8e58da3a5e3adc37a"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"0021","classname":"0021","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"PROSPECTIE"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Archaeology"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Plangebied Lange Ekker te Vessem, gemeente Eersel"}]} +{"author":[],"bestaccessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofcollection":"2021-09-25T10:43:13.768Z","dateoftransformation":"2021-09-25T11:01:22.863Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"This find is registered at Portable Antiquities of the Netherlands with number PAN-00054604"}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"geolocation":[],"id":"50|DansKnawCris::203a27996ddc0fd1948258e5b7dec61c","instance":[{"accessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"pid":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"urn","classname":"urn","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"urn:nbn:nl:ui:13-a7-hwgy"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.17026/dans-x3z-fsq5"}],"collectedfrom":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"hostedby":{"key":"10|openaire____::c6df70599aa984f16ee52b4b86d2e89f","value":"DANS (Data Archiving and Networked Services)"},"instancetype":{"classid":"0019","classname":"Patent","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"alternateIdentifier":[],"refereed":{"classid":"0000","classname":"Unknown","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["","http://dx.doi.org/10.17026/dans-x3z-fsq5"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1635434508886,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"http%3A%2F%2Fservices.nod.dans.knaw.nl%2Foa-cerif","datestamp":"2021-08-16T14:01:37Z","harvestDate":"2021-09-25T10:43:13.768Z","identifier":"oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:129566","metadataNamespace":""}},"originalId":["oai:services.nod.dans.knaw.nl:Products/dans:oai:easy.dans.knaw.nl:easy-dataset:129566","50|DansKnawCris::203a27996ddc0fd1948258e5b7dec61c"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"0021","classname":"0021","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"early medieval enamelled disc brooch variant A9"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Broader Match: disc brooches"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Broader Match: schijffibula - geemailleerd"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"metal"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"copper alloy"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Temporal coverage: Early Middle Ages C"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Temporal coverage: Early Middle Ages D"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Temporal coverage: 800 until 1000"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Archaeology"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:datasetarchive","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"PAN-00054604 - early medieval enamelled disc brooch variant A9"}]} \ No newline at end of file diff --git a/pom.xml b/pom.xml index 0f1ec78af..3fd351c1d 100644 --- a/pom.xml +++ b/pom.xml @@ -888,7 +888,7 @@ 3.3.3 3.4.2 [2.12,3.0) - [4.17.2-SNAPSHOT] + [4.17.2] [4.0.3] [6.0.5] [3.1.6] From 359e81b7a60d987db509b7c53121863380c64aff Mon Sep 17 00:00:00 2001 From: dimitrispie Date: Thu, 23 Nov 2023 10:48:55 +0200 Subject: [PATCH 24/60] Update StatsAtomicActionsJob.java Bug fix for duplicate bronze checks --- .../actionmanager/stats_actionsets/StatsAtomicActionsJob.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsAtomicActionsJob.java b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsAtomicActionsJob.java index e984d68fc..a64a63c24 100644 --- a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsAtomicActionsJob.java +++ b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsAtomicActionsJob.java @@ -138,10 +138,10 @@ public class StatsAtomicActionsJob implements Serializable { r.setPubliclyFunded(usm.isPublicly_funded()); if (usm.isIs_bronze_oa()) r.setOpenAccessColor(OpenAccessColor.bronze); - else if (usm.isIs_gold()) - r.setOpenAccessColor(OpenAccessColor.bronze); else if (usm.isIs_gold()) r.setOpenAccessColor(OpenAccessColor.gold); + else if (usm.isIs_hybrid()) + r.setOpenAccessColor(OpenAccessColor.hybrid); return r; }, Encoders.bean(Result.class)); } From 34a4b3cbdfe3bc4d07b565b95089c89ddbd25f9c Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Fri, 24 Nov 2023 12:39:58 +0100 Subject: [PATCH 25/60] Implemented ORCID Enrichment --- .../eu/dnetlib/dhp/oa/merge/AuthorMerger.java | 292 +++++++++++++++++- .../eu/dnetlib/oa/merge/AuthorMergerTest.java | 125 ++++++++ .../collection/orcid/DownloadORCIDTest.java | 39 --- .../orcid/enrich_graph_orcid_parameters.json | 26 ++ .../enrich/orcid/oozie_app/config-default.xml | 34 ++ .../dhp/enrich/orcid/oozie_app/workflow.xml | 52 ++++ .../dhp/enrich/orcid/AuthorEnricher.scala | 37 +++ .../SparkEnrichGraphWithOrcidAuthors.scala | 119 +++++++ .../dhp/enrich/orcid/EnrichOrcidTest.scala | 12 + 9 files changed, 696 insertions(+), 40 deletions(-) create mode 100644 dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/enrich_graph_orcid_parameters.json create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/AuthorEnricher.scala create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala create mode 100644 dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/EnrichOrcidTest.scala diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java index aea046203..6c3058303 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java @@ -1,11 +1,18 @@ package eu.dnetlib.dhp.oa.merge; +import java.io.FileWriter; +import java.io.IOException; import java.text.Normalizer; import java.util.*; +import java.util.function.Function; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.MutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.jetbrains.annotations.NotNull; import com.wcohen.ss.JaroWinkler; @@ -14,6 +21,28 @@ import eu.dnetlib.dhp.schema.oaf.StructuredProperty; import eu.dnetlib.pace.model.Person; import scala.Tuple2; +class SimilarityCellInfo implements Comparable { + + public int authorPosition = 0; + public int orcidPosition = 0; + + public double maxColumnSimilarity = 0.0; + + public SimilarityCellInfo() { + } + + public void setValues(final int authPos, final int orcidPos, final double similarity) { + this.authorPosition = authPos; + this.orcidPosition = orcidPos; + this.maxColumnSimilarity = similarity; + } + + @Override + public int compareTo(@NotNull SimilarityCellInfo o) { + return Double.compare(maxColumnSimilarity, o.maxColumnSimilarity); + } +} + public class AuthorMerger { private static final Double THRESHOLD = 0.95; @@ -119,6 +148,267 @@ public class AuthorMerger { }); } + public static String normalizeFullName(final String fullname) { + return nfd(fullname) + .toLowerCase() + // do not compact the regexes in a single expression, would cause StackOverflowError + // in case + // of large input strings + .replaceAll("(\\W)+", " ") + .replaceAll("(\\p{InCombiningDiacriticalMarks})+", " ") + .replaceAll("(\\p{Punct})+", " ") + .replaceAll("(\\d)+", " ") + .replaceAll("(\\n)+", " ") + .trim(); +// return Arrays.stream(fullname.split("[\\s | , | ;]+")).map(String::toLowerCase).sorted().collect(Collectors.joining()); + } + + private static String generateAuthorkey(final Author a) { + if (a.getSurname() == null) + return "NOSURNAME"; + + return normalize(a.getSurname()); + } + +// +// public static List enrichOrcid2(List baseAuthor, List orcidAuthor) { +// if (baseAuthor == null || baseAuthor.isEmpty()) +// return orcidAuthor; +// +// if (orcidAuthor == null || orcidAuthor.isEmpty()) +// return baseAuthor; +// +// if (baseAuthor.size() == 1 && orcidAuthor.size() > 10) +// return baseAuthor; +// +// +// Map> pubClusters = baseAuthor.stream().collect(Collectors.toMap(AuthorMerger::generateAuthorkey, Arrays::asList, (a, b) -> { +// a.addAll(b); +// return a; +// })); +// +// Map> orcidClusters = baseAuthor.stream().collect(Collectors.toMap(AuthorMerger::generateAuthorkey, Arrays::asList, (a, b) -> { +// a.addAll(b); +// return a; +// })); +// +// System.out.println(pubClusters.keySet().size()); +// System.out.println(orcidClusters.keySet().size()); +// +// +// +// +// return null; +// +// +// } + + static int hammingDist(String str1, String str2) { + if (str1.length() != str2.length()) + return Math.max(str1.length(), str2.length()); + int i = 0, count = 0; + while (i < str1.length()) { + if (str1.charAt(i) != str2.charAt(i)) + count++; + i++; + } + return count; + } + + private static String authorFieldToBeCompared(Author author) { + if (StringUtils.isNotBlank(author.getSurname())) { + return author.getSurname(); + + } + if (StringUtils.isNotBlank(author.getFullname())) { + return author.getFullname(); + } + return null; + } + + public static boolean checkSimilarity3(final Author left, final Author right) { + + if (StringUtils.isNotBlank(left.getSurname()) && StringUtils.isNotBlank(left.getName()) + && + StringUtils.isNotBlank(right.getSurname()) && StringUtils.isNotBlank(right.getName()) + + ) + return left.getSurname().equalsIgnoreCase(right.getSurname()) + && left.getName().substring(0, 1).equalsIgnoreCase(right.getName().substring(0, 1)); + + final Person pl = parse(left); + final Person pr = parse(right); + + // If one of them didn't have a surname the match is false + if (!(pl.getSurname() != null && pl.getSurname().stream().anyMatch(StringUtils::isNotBlank) && + pr.getSurname() != null && pr.getSurname().stream().anyMatch(StringUtils::isNotBlank))) + return false; + + // The Authors have one surname in common + if (pl.getSurname().stream().anyMatch(sl -> pr.getSurname().stream().anyMatch(sr -> sr.equalsIgnoreCase(sl)))) { + + // If one of them has only a surname and is the same we can say that they are the same author + if ((pl.getName() == null || pl.getName().stream().allMatch(StringUtils::isBlank)) || + (pr.getName() == null || pr.getName().stream().allMatch(StringUtils::isBlank))) + return true; + // The authors have the same initials of Name in common + if (pl + .getName() + .stream() + .anyMatch( + nl -> pr + .getName() + .stream() + .anyMatch(nr -> nr.substring(0, 1).equalsIgnoreCase(nl.substring(0, 1))))) + return true; + } + return false; + } + + public static boolean checkSimilarity2(final Author left, final Author right) { + final Person pl = parse(left); + final Person pr = parse(right); + + // If one of them didn't have a surname the match is false + if (!(pl.getSurname() != null && pl.getSurname().stream().anyMatch(StringUtils::isNotBlank) && + pr.getSurname() != null && pr.getSurname().stream().anyMatch(StringUtils::isNotBlank))) + return false; + + // The Authors have one surname in common + if (pl.getSurname().stream().anyMatch(sl -> pr.getSurname().stream().anyMatch(sr -> sr.equalsIgnoreCase(sl)))) { + + // If one of them has only a surname and is the same we can say that they are the same author + if ((pl.getName() == null || pl.getName().stream().allMatch(StringUtils::isBlank)) || + (pr.getName() == null || pr.getName().stream().allMatch(StringUtils::isBlank))) + return true; + // The authors have the same initials of Name in common + if (pl + .getName() + .stream() + .anyMatch( + nl -> pr + .getName() + .stream() + .anyMatch(nr -> nr.substring(0, 1).equalsIgnoreCase(nl.substring(0, 1))))) + return true; + } + return false; + } + + public static boolean checkSimilarity(final Author left, final Author right) { + + if (left.getSurname() == null && left.getFullname() == null) + return false; + if (right.getSurname() == null && right.getFullname() == null) + return false; + + // The Authors have the same surname, or we are tolerant from 1 different char(lets say 1 Typo) + if (StringUtils.isNotBlank(left.getSurname()) && StringUtils.isNotBlank(right.getSurname())) { + if (left.getSurname().equalsIgnoreCase(right.getSurname()) + || hammingDist(left.getSurname().toLowerCase(), right.getSurname().toLowerCase()) < 2) { + // IN case on of the two Authors has no given Name the match is true + if (StringUtils.isBlank(left.getName()) || StringUtils.isBlank(right.getName())) + return true; + // If the surname is correct, and they have the same name or the name starts with the same Letter we can + // say is the same author + if (left.getName().equalsIgnoreCase(right.getName()) + || left.getName().substring(0, 1).equalsIgnoreCase(right.getName().substring(0, 1))) + return true; + } + // Different SURNAME + else { + return false; + } + } else { + // This is the case where the two authors have or the surname or the fullname + // get the first not null of the surname or fullname of both + final String l = authorFieldToBeCompared(left); + final String r = authorFieldToBeCompared(right); + if (l == null || r == null) + return false; + // The same length means they are the same field + if (l.length() == r.length()) { + return normalize(l).equals(normalize(r)); + } + // In this case probably l contains the surname and r contains the fullname + if (l.length() < r.length()) + return normalize(r).contains(normalize(l)); + // In this case probably l contains the fullname and r contains the surname + return normalize(l).contains(normalize(r)); + } + return false; + } + + public static List enrichOrcid2(List baseAuthor, List orcidAuthor) { + + final Integer match_itm = 0; + if (baseAuthor == null || baseAuthor.isEmpty()) + return orcidAuthor; + + if (orcidAuthor == null || orcidAuthor.isEmpty()) + return baseAuthor; + + if (baseAuthor.size() == 1 && orcidAuthor.size() > 10) + return baseAuthor; + + final List oAuthor = new ArrayList<>(); + oAuthor.addAll(orcidAuthor); + + baseAuthor.forEach(ba -> { + Optional aMatch = oAuthor.stream().filter(oa -> checkSimilarity2(ba, oa)).findFirst(); + if (aMatch.isPresent()) { + final Author sameAuthor = aMatch.get(); + addPid(ba, sameAuthor.getPid()); + oAuthor.remove(sameAuthor); + } + }); + return baseAuthor; + } + + public static List enrichOrcid(List baseAuthor, List orcidAuthor) { + + if (baseAuthor == null || baseAuthor.isEmpty()) + return orcidAuthor; + + if (orcidAuthor == null || orcidAuthor.isEmpty()) + return baseAuthor; + + if (baseAuthor.size() == 1 && orcidAuthor.size() > 10) + return baseAuthor; + + final Double similarityMatrix[][] = new Double[baseAuthor.size()][orcidAuthor.size()]; + + final List maxColums = new ArrayList<>(); + + for (int i = 0; i < orcidAuthor.size(); i++) + maxColums.add(new SimilarityCellInfo()); + + for (int i = 0; i < baseAuthor.size(); i++) { + for (int j = 0; j < orcidAuthor.size(); j++) { + similarityMatrix[i][j] = sim(baseAuthor.get(i), orcidAuthor.get(j)); + if (maxColums.get(j).maxColumnSimilarity < similarityMatrix[i][j]) + maxColums.get(j).setValues(i, j, similarityMatrix[i][j]); + } + } + maxColums + .stream() + .sorted() + .filter(si -> si.maxColumnSimilarity > 0.85) + .forEach(si -> addPid(baseAuthor.get(si.authorPosition), orcidAuthor.get(si.orcidPosition).getPid())); + return baseAuthor; + + } + + private static void addPid(final Author a, final List pids) { + + if (a.getPid() == null) { + a.setPid(new ArrayList<>()); + } + + a.getPid().addAll(pids); + + } + public static String pidToComparableString(StructuredProperty pid) { final String classid = pid.getQualifier().getClassid() != null ? pid.getQualifier().getClassid().toLowerCase() : ""; @@ -171,7 +461,7 @@ public class AuthorMerger { } } - private static String normalize(final String s) { + public static String normalize(final String s) { String[] normalized = nfd(s) .toLowerCase() // do not compact the regexes in a single expression, would cause StackOverflowError diff --git a/dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java b/dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java new file mode 100644 index 000000000..7f8d673d1 --- /dev/null +++ b/dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java @@ -0,0 +1,125 @@ + +package eu.dnetlib.oa.merge; + +import static org.junit.jupiter.api.Assertions.*; + +import java.io.BufferedReader; +import java.io.FileReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import org.junit.jupiter.api.Test; +import org.junit.platform.commons.util.StringUtils; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.oa.merge.AuthorMerger; +import eu.dnetlib.dhp.schema.oaf.Author; + +public class AuthorMergerTest { + + @Test + public void testNormalization() { + + assertEquals("bruzzolasandro", AuthorMerger.normalizeFullName("Sandro, La Bruzzo")); + assertEquals("baglionimiriam", AuthorMerger.normalizeFullName("Miriam Baglioni")); + assertEquals("baglionimiriam", AuthorMerger.normalizeFullName("Miriam ;Baglioni,")); + + } + + public void testEnrcichAuthor() throws Exception { + final ObjectMapper mapper = new ObjectMapper(); + + BufferedReader pr = new BufferedReader(new InputStreamReader( + AuthorMergerTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/merge/authors_publication.json"))); + BufferedReader or = new BufferedReader(new InputStreamReader( + AuthorMergerTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/merge/authors_orcid.json"))); + + TypeReference> aclass = new TypeReference>() { + }; + String pubLine; + + int i = 0; + while ((pubLine = pr.readLine()) != null) { + final String pubId = pubLine; + final String MatchPidOrcid = or.readLine(); + final String pubOrcid = or.readLine(); + + final String data = pr.readLine(); + + if (StringUtils.isNotBlank(data)) { + List publicationAuthors = mapper.readValue(data, aclass); + List orcidAuthors = mapper.readValue(or.readLine(), aclass); + System.out.printf("OAF ID = %s \n", pubId); + System.out.printf("ORCID Intersected ID = %s \n", pubOrcid); + System.out.printf("OAF Author Size = %d \n", publicationAuthors.size()); + System.out.printf("Oricd Author Size = %d \n", orcidAuthors.size()); + System.out.printf("Oricd Matched PID = %s \n", MatchPidOrcid); + + long originalAuthorWithPiD = publicationAuthors + .stream() + .filter( + a -> a.getPid() != null && a + .getPid() + .stream() + .anyMatch( + p -> p.getQualifier() != null + && p.getQualifier().getClassid().toLowerCase().contains("orcid"))) + .count(); + long start = System.currentTimeMillis(); + +// final List enrichedList = AuthorMerger.enrichOrcid(publicationAuthors, orcidAuthors); + final List enrichedList = AuthorMerger.enrichOrcid2(publicationAuthors, orcidAuthors); + + long enrichedAuthorWithPid = enrichedList + .stream() + .filter( + a -> a.getPid() != null && a + .getPid() + .stream() + .anyMatch( + p -> p.getQualifier() != null + && p.getQualifier().getClassid().toLowerCase().contains("orcid"))) + .count(); + + long totalTime = (System.currentTimeMillis() - start) / 1000; + System.out + .printf( + "Enriched authors in %d seconds from %d pid to %d pid \n", totalTime, originalAuthorWithPiD, + enrichedAuthorWithPid); + + System.out.println("================="); + + if (++i > 30) + break; + } + + } + + } + + @Test + public void checkSimilarityTest() { + final Author left = new Author(); + left.setSurname("Wu"); + left.setName("M."); + left.setFullname("Wu, M."); + + System.out.println(AuthorMerger.normalizeFullName(left.getFullname())); + + final Author right = new Author(); + right.setName("Xin"); + right.setSurname("Wu"); + right.setFullname("Xin Wu"); +// System.out.println(AuthorMerger.normalize(right.getFullname())); + boolean same = AuthorMerger.checkSimilarity2(left, right); + + assertFalse(same); + + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/orcid/DownloadORCIDTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/orcid/DownloadORCIDTest.java index be5555fc0..868f4e92d 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/orcid/DownloadORCIDTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/orcid/DownloadORCIDTest.java @@ -32,45 +32,6 @@ import eu.dnetlib.dhp.parser.utility.VtdException; public class DownloadORCIDTest { private final Logger log = LoggerFactory.getLogger(DownloadORCIDTest.class); -// public void test() throws Exception { -// -// Configuration conf = new Configuration(); -// // Set FileSystem URI -//// conf.set("fs.defaultFS", "file://"); -// // Because of Maven -// conf.set("fs.hdfs.impl", org.apache.hadoop.hdfs.DistributedFileSystem.class.getName()); -// conf.set("fs.file.impl", org.apache.hadoop.fs.LocalFileSystem.class.getName()); -// -// System.setProperty("hadoop.home.dir", "file:///Users/sandro/orcid/"); -// -// final FileSystem fileSystem = FileSystem.get(conf); -// -// new ExtractORCIDDump(fileSystem).run("/Users/sandro/orcid/", "/Users/sandro/orcid/extracted"); -// -//// final GZIPInputStream gzip = new GZIPInputStream(Files.newInputStream(Paths.get("/Users/sandro/orcid/ORCID_2023_10_activities_1.tar.gz"))); -//// try(final TarArchiveInputStream tais = new TarArchiveInputStream(gzip)) { -//// -//// TarArchiveEntry entry; -//// while ((entry = tais.getNextTarEntry()) != null) { -//// -//// if (entry.isFile() && entry.getName().contains("employments")) { -//// -//// System.out.println(entry.getName()); -//// final String [] items = entry.getName().split("/"); -//// -//// final String res = IOUtils.toString(new BufferedReader(new InputStreamReader(tais))); -//// System.out.println("res = " + res); -//// -//// System.out.println(items[items.length-2]); -//// break; -//// } -//// -//// -//// } -//// } -// -// } - @Test public void testSummary() throws Exception { final String xml = IOUtils diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/enrich_graph_orcid_parameters.json b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/enrich_graph_orcid_parameters.json new file mode 100644 index 000000000..765c0e8ff --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/enrich_graph_orcid_parameters.json @@ -0,0 +1,26 @@ +[ + { + "paramName": "mt", + "paramLongName": "master", + "paramDescription": "should be local or yarn", + "paramRequired": true + }, + { + "paramName": "op", + "paramLongName": "orcidPath", + "paramDescription": "the path of the orcid Table generated by the dump", + "paramRequired": true + }, + { + "paramName": "gp", + "paramLongName": "graphPath", + "paramDescription": "the path of the graph we want to apply enrichment", + "paramRequired": true + }, + { + "paramName": "tp", + "paramLongName": "targetPath", + "paramDescription": "the output path of the graph enriched", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/config-default.xml new file mode 100644 index 000000000..8a7bc8942 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/config-default.xml @@ -0,0 +1,34 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + + hiveMetastoreUris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + + hiveJdbcUrl + jdbc:hive2://iis-cdh5-test-m3.ocean.icm.edu.pl:10000 + + + hiveDbName + openaire + + + oozie.launcher.mapreduce.user.classpath.first + true + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml new file mode 100644 index 000000000..1284cceda --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml @@ -0,0 +1,52 @@ + + + + orcidPath + the path of the orcid Table generated by the dump + + + graphPath + the path of the graph we want to apply enrichment + + + targetPath + the output path of the graph enriched + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + yarn + cluster + Enrich Graph with ORCID + eu.dnetlib.dhp.enrich.orcid.SparkEnrichGraphWithOrcidAuthors + dhp-graph-mapper-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.executor.memoryOverhead=2g + --conf spark.sql.shuffle.partitions=3000 + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --orcidPath${orcidPath} + --targetPath${targetPath} + --graphPath${graphPath}/publication + --masteryarn + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/AuthorEnricher.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/AuthorEnricher.scala new file mode 100644 index 000000000..a67de4b95 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/AuthorEnricher.scala @@ -0,0 +1,37 @@ +package eu.dnetlib.dhp.enrich.orcid + +import eu.dnetlib.dhp.schema.oaf.{Author, Publication} +import eu.dnetlib.dhp.schema.sx.OafUtils +import org.apache.spark.sql.Row + +import scala.collection.JavaConverters._ + +object AuthorEnricher extends Serializable { + + def createAuthor(givenName: String, familyName: String, orcid: String): Author = { + val a = new Author + a.setName(givenName) + a.setSurname(familyName) + a.setFullname(s"$givenName $familyName") + a.setPid(List(OafUtils.createSP(orcid, "ORCID", "ORCID")).asJava) + a + + } + + def toOAFAuthor(r: Row): java.util.List[Author] = { + r.getList[Row](1) + .asScala + .map(s => createAuthor(s.getAs[String]("givenName"), s.getAs[String]("familyName"), s.getAs[String]("orcid"))) + .toList + .asJava + } + +// def enrichAuthor(p:Publication,r:Row): Unit = { +// val k:Map[String, OAuthor] =r.getList[Row](1).asScala.map(s => (s.getAs[String]("orcid"), OAuthor(s.getAs[String]("givenName") ,s.getAs[String]("familyName") ))).groupBy(_._1).mapValues(_.map(_._2).head) +// println(k) +// +// +// +// } + +} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala new file mode 100644 index 000000000..e190b2b33 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala @@ -0,0 +1,119 @@ +package eu.dnetlib.dhp.enrich.orcid + +import com.fasterxml.jackson.databind.ObjectMapper +import eu.dnetlib.dhp.application.AbstractScalaApplication +import eu.dnetlib.dhp.oa.merge.AuthorMerger +import eu.dnetlib.dhp.schema.oaf.{Author, DataInfo, Instance, Publication, StructuredProperty} +import org.apache.spark.sql.{Dataset, Encoder, Encoders, Row, SaveMode, SparkSession} +import org.apache.spark.sql.functions.{col, collect_set, concat, explode, expr, first, flatten, lower, size, struct} +import org.slf4j.{Logger, LoggerFactory} +import org.apache.spark.sql.types._ + +class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String], log: Logger) + extends AbstractScalaApplication(propertyPath, args, log: Logger) { + + /** Here all the spark applications runs this method + * where the whole logic of the spark node is defined + */ + override def run(): Unit = { + val graphPath = parser.get("graphPath") + log.info(s"graphPath is '$graphPath'") + val orcidPath = parser.get("orcidPath") + log.info(s"orcidPath is '$orcidPath'") + val targetPath = parser.get("targetPath") + log.info(s"targetPath is '$targetPath'") + enrichResult(spark, graphPath, orcidPath, targetPath) + } + + def enrichResult(spark: SparkSession, graphPath: String, orcidPath: String, outputPath: String): Unit = { + val orcidPublication = generateOrcidTable(spark, orcidPath) + implicit val publicationEncoder = Encoders.bean(classOf[Publication]) + + val aschema = new StructType() + .add("id", StringType) + .add("dataInfo", Encoders.bean(classOf[DataInfo]).schema) + .add( + "author",Encoders.bean(classOf[Author]).schema + + ) + + val schema = new StructType() + .add("id", StringType) + .add("dataInfo", Encoders.bean(classOf[DataInfo]).schema) + .add( + "instance", + ArrayType(new StructType().add("pid", ArrayType(Encoders.bean(classOf[StructuredProperty]).schema))) + ) + val entities = spark.read + .schema(schema) + .json(graphPath) + .where("datainfo.deletedbyinference = false") + .drop("datainfo") + .withColumn("instances", explode(col("instance"))) + .withColumn("pids", explode(col("instances.pid"))) + .select( + col("pids.qualifier.classid").alias("pid_schema"), + col("pids.value").alias("pid_value"), + col("id").alias("dnet_id") + ) + val orcidDnet = orcidPublication + .join( + entities, + lower(col("schema")).equalTo(lower(col("pid_schema"))) && + lower(col("value")).equalTo(lower(col("pid_value"))), + "inner" + ) + .groupBy(col("dnet_id")) + .agg(collect_set(orcidPublication("author")).alias("orcid_authors")) + .select("dnet_id", "orcid_authors") + .cache() + + val publication = spark.read.schema(publicationEncoder.schema).json(graphPath).as[Publication] + + publication + .joinWith(orcidDnet, publication("id").equalTo(orcidDnet("dnet_id")), "left") + .map { + case (p: Publication, null) => { + p + } + case (p: Publication, r: Row) => + p.setAuthor(AuthorMerger.enrichOrcid2(p.getAuthor, AuthorEnricher.toOAFAuthor(r))) + p + } + .write + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath) + } + + def generateOrcidTable(spark: SparkSession, inputPath: String): Dataset[Row] = { + val orcidAuthors = + spark.read.load(s"$inputPath/Authors").select("orcid", "familyName", "givenName", "creditName", "otherNames") + val orcidWorks = spark.read + .load(s"$inputPath/Works") + .select(col("orcid"), explode(col("pids")).alias("identifier")) + .where( + "identifier.schema = 'doi' or identifier.schema ='pmid' or identifier.schema ='pmc' or identifier.schema ='arxiv' or identifier.schema ='handle'" + ) + orcidAuthors + .join(orcidWorks, orcidAuthors("orcid").equalTo(orcidWorks("orcid"))) + .select( + col("identifier.schema").alias("schema"), + col("identifier.value").alias("value"), + struct(orcidAuthors("orcid").alias("orcid"), col("givenName"), col("familyName")).alias("author") + ) + } +} + +object SparkEnrichGraphWithOrcidAuthors { + + val log: Logger = LoggerFactory.getLogger(SparkEnrichGraphWithOrcidAuthors.getClass) + + def main(args: Array[String]): Unit = { + + new SparkEnrichGraphWithOrcidAuthors("/eu/dnetlib/dhp/enrich/orcid/enrich_graph_orcid_parameters.json", args, log) + .initialize() + .run() + + } +} diff --git a/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/EnrichOrcidTest.scala b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/EnrichOrcidTest.scala new file mode 100644 index 000000000..0ddb7c0aa --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/EnrichOrcidTest.scala @@ -0,0 +1,12 @@ +package eu.dnetlib.dhp.enrich.orcid + +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession +import org.junit.jupiter.api.Test +import org.slf4j.{Logger, LoggerFactory} + +class EnrichOrcidTest { + + val log: Logger = LoggerFactory.getLogger(getClass) + +} From 48e0427a234b1bdee63f21d6c8879f10ceeba3da Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Nov 2023 15:10:27 +0100 Subject: [PATCH 26/60] changed the parameter from production to baseURL. Fixed issue in tagging configuration --- .../eu/dnetlib/dhp/api/QueryCommunityAPI.java | 10 +- .../dnetlib/dhp/bulktag/SparkBulkTagJob.java | 1 + .../dhp/bulktag/input_bulkTag_parameters.json | 8 +- ...t_preparecommunitytoresult_parameters.json | 8 +- ...t_preparecommunitytoresult_parameters.json | 6 +- .../dnetlib/dhp/bulktag/BulkTagJobTest.java | 8 +- .../tagging_conf_remove.xml | 182 +++++++++--------- 7 files changed, 112 insertions(+), 111 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java index dc10fce15..b7c0164c1 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java @@ -34,32 +34,32 @@ public class QueryCommunityAPI { public static String communities(String baseURL) throws IOException { - return get(baseURL + "community/communities"); + return get(baseURL + "communities"); } public static String community(String id, String baseURL ) throws IOException { - return get(baseURL + "community/" + id); + return get(baseURL + id); } public static String communityDatasource(String id, String baseURL ) throws IOException { - return get(baseURL + "community/" + id + "/contentproviders"); + return get(baseURL + id + "/contentproviders"); } public static String communityPropagationOrganization(String id, String baseURL ) throws IOException { - return get(baseURL + "community/" + id + "/propagationOrganizations"); + return get(baseURL + id + "/propagationOrganizations"); } public static String communityProjects(String id, String page, String size, String baseURL ) throws IOException { - return get(baseURL + "community/" + id + "/projects/" + page + "/" + size); + return get(baseURL + id + "/projects/" + page + "/" + size); } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java index d972cecac..f01063955 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java @@ -80,6 +80,7 @@ public class SparkBulkTagJob { cc = CommunityConfigurationFactory.newInstance(taggingConf); } else { cc = Utils.getCommunityConfiguration(baseURL); + log.info(OBJECT_MAPPER.writeValueAsString(cc)); } runWithSparkSession( diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json index dbe2d088f..d88904cd9 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json @@ -34,10 +34,10 @@ }, { - "paramName": "p", - "paramLongName": "production", - "paramDescription": "this parameter is intended for testing purposes only. It is a possible tagging configuration obtained via the XQUERY. Intended to be removed", - "paramRequired": true + "paramName": "bu", + "paramLongName": "baseURL", + "paramDescription": "this parameter is to specify the api to be queried (beta or production)", + "paramRequired": false } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json index 8b6291e5d..3601db7ac 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/input_preparecommunitytoresult_parameters.json @@ -24,10 +24,10 @@ "paramRequired": true }, { - "paramName": "p", - "paramLongName": "production", - "paramDescription": "the path used to store temporary output files", - "paramRequired": true + "paramName": "bu", + "paramLongName": "baseURL", + "paramDescription": "the base URL to the community API to use", + "paramRequired": false } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/input_preparecommunitytoresult_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/input_preparecommunitytoresult_parameters.json index 9a50c79fa..cbc01c2d5 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/input_preparecommunitytoresult_parameters.json +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/input_preparecommunitytoresult_parameters.json @@ -19,10 +19,10 @@ "paramRequired": true }, { - "paramName": "p", - "paramLongName": "production", + "paramName": "bu", + "paramLongName": "baseURL", "paramDescription": "the path used to store temporary output files", - "paramRequired": true + "paramRequired": false } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java index ba0c6b252..d193716a8 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java @@ -533,6 +533,7 @@ public class BulkTagJobTest { + "where MyD.inferenceprovenance = 'bulktagging'"; org.apache.spark.sql.Dataset idExplodeCommunity = spark.sql(query); + Assertions.assertEquals(7, idExplodeCommunity.count()); Assertions @@ -1572,11 +1573,10 @@ public class BulkTagJobTest { "-isSparkSessionManaged", Boolean.FALSE.toString(), "-sourcePath", getClass().getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/no_updates/").getPath(), - "-taggingConf", taggingConf, - "-outputPath", workingDir.toString() + "/", - "-production", Boolean.TRUE.toString(), - "-pathMap", pathMap +// "-baseURL", "https://services.openaire.eu/openaire/community/", + "-pathMap", pathMap, + "-taggingConf", taggingConf }); final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/tagging_conf_remove.xml b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/tagging_conf_remove.xml index edd6c7e0a..21c7d452f 100644 --- a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/tagging_conf_remove.xml +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/bulktag/communityconfiguration/tagging_conf_remove.xml @@ -26,7 +26,7 @@ - re3data_____::a507cdacc5bbcc08761c92185dee5cab + 10|re3data_____::a507cdacc5bbcc08761c92185dee5cab @@ -140,39 +140,39 @@ - re3data_____::9ebe127e5f3a0bf401875690f3bb6b81 + 10|re3data_____::9ebe127e5f3a0bf401875690f3bb6b81 - doajarticles::c6cd4b532e12868c1d760a8d7cda6815 + 10|doajarticles::c6cd4b532e12868c1d760a8d7cda6815 - doajarticles::a6de4499bb87bf3c01add0a9e2c9ed0b + 10|doajarticles::a6de4499bb87bf3c01add0a9e2c9ed0b - doajarticles::6eb31d13b12bc06bbac06aef63cf33c9 + 10|doajarticles::6eb31d13b12bc06bbac06aef63cf33c9 - doajarticles::0da84e9dfdc8419576169e027baa8028 + 10|doajarticles::0da84e9dfdc8419576169e027baa8028 - re3data_____::84e123776089ce3c7a33db98d9cd15a8 + 10|re3data_____::84e123776089ce3c7a33db98d9cd15a8 - openaire____::c5502a43e76feab55dd00cf50f519125 + 10|openaire____::c5502a43e76feab55dd00cf50f519125 - re3data_____::a48f09c562b247a9919acfe195549b47 + 10|re3data_____::a48f09c562b247a9919acfe195549b47 - opendoar____::97275a23ca44226c9964043c8462be96 + 10|opendoar____::97275a23ca44226c9964043c8462be96 @@ -287,55 +287,55 @@ - doajarticles::8cec81178926caaca531afbd8eb5d64c + 10|doajarticles::8cec81178926caaca531afbd8eb5d64c - doajarticles::0f7a7f30b5400615cae1829f3e743982 + 10|doajarticles::0f7a7f30b5400615cae1829f3e743982 - doajarticles::9740f7f5af3e506d2ad2c215cdccd51a + 10|doajarticles::9740f7f5af3e506d2ad2c215cdccd51a - doajarticles::9f3fbaae044fa33cb7069b72935a3254 + 10|doajarticles::9f3fbaae044fa33cb7069b72935a3254 - doajarticles::cb67f33eb9819f5c624ce0313957f6b3 + 10|doajarticles::cb67f33eb9819f5c624ce0313957f6b3 - doajarticles::e21c97cbb7a209afc75703681c462906 + 10|doajarticles::e21c97cbb7a209afc75703681c462906 - doajarticles::554cde3be9e5c4588b4c4f9f503120cb + 10|doajarticles::554cde3be9e5c4588b4c4f9f503120cb - tubitakulakb::11e22f49e65b9fd11d5b144b93861a1b + 10|tubitakulakb::11e22f49e65b9fd11d5b144b93861a1b - doajarticles::57c5d3837da943e93b28ec4db82ec7a5 + 10|doajarticles::57c5d3837da943e93b28ec4db82ec7a5 - doajarticles::a186f5ddb8e8c7ecc992ef51cf3315b1 + 10|doajarticles::a186f5ddb8e8c7ecc992ef51cf3315b1 - doajarticles::e21c97cbb7a209afc75703681c462906 + 10|doajarticles::e21c97cbb7a209afc75703681c462906 - doajarticles::dca64612dfe0963fffc119098a319957 + 10|doajarticles::dca64612dfe0963fffc119098a319957 - doajarticles::dd70e44479f0ade25aa106aef3e87a0a + 10|doajarticles::dd70e44479f0ade25aa106aef3e87a0a @@ -406,27 +406,27 @@ - re3data_____::5b9bf9171d92df854cf3c520692e9122 + 10|re3data_____::5b9bf9171d92df854cf3c520692e9122 - doajarticles::c7d3de67dc77af72f6747157441252ec + 10|doajarticles::c7d3de67dc77af72f6747157441252ec - re3data_____::8515794670370f49c1d176c399c714f5 + 10|re3data_____::8515794670370f49c1d176c399c714f5 - doajarticles::d640648c84b10d425f96f11c3de468f3 + 10|doajarticles::d640648c84b10d425f96f11c3de468f3 - doajarticles::0c0e74daa5d95504eade9c81ebbd5b8a + 10|doajarticles::0c0e74daa5d95504eade9c81ebbd5b8a - rest________::fb1a3d4523c95e63496e3bc7ba36244b + 10|rest________::fb1a3d4523c95e63496e3bc7ba36244b @@ -743,27 +743,27 @@ - opendoar____::1a551829d50f1400b0dab21fdd969c04 + 10|opendoar____::1a551829d50f1400b0dab21fdd969c04 - opendoar____::49af6c4e558a7569d80eee2e035e2bd7 + 10|opendoar____::49af6c4e558a7569d80eee2e035e2bd7 - opendoar____::0266e33d3f546cb5436a10798e657d97 + 10|opendoar____::0266e33d3f546cb5436a10798e657d97 - opendoar____::fd4c2dc64ccb8496e6f1f94c85f30d06 + 10|opendoar____::fd4c2dc64ccb8496e6f1f94c85f30d06 - opendoar____::41bfd20a38bb1b0bec75acf0845530a7 + 10|opendoar____::41bfd20a38bb1b0bec75acf0845530a7 - opendoar____::87ae6fb631f7c8a627e8e28785d9992d + 10|opendoar____::87ae6fb631f7c8a627e8e28785d9992d @@ -983,11 +983,11 @@ - opendoar____::7e7757b1e12abcb736ab9a754ffb617a + 10|opendoar____::7e7757b1e12abcb736ab9a754ffb617a {"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]} - opendoar____::96da2f590cd7246bbde0051047b0d6f7 + 10|opendoar____::96da2f590cd7246bbde0051047b0d6f7 {"criteria":[{"constraint":[{"verb":"contains","field":"contributor","value":"DARIAH"}]}]} @@ -1166,87 +1166,87 @@ - doajarticles::1c5bdf8fca58937894ad1441cca99b76 + 10|doajarticles::1c5bdf8fca58937894ad1441cca99b76 - doajarticles::b37a634324a45c821687e6e80e6f53b4 + 10|doajarticles::b37a634324a45c821687e6e80e6f53b4 - doajarticles::4bf64f2a104040e4e055cd9594b2d77c + 10|doajarticles::4bf64f2a104040e4e055cd9594b2d77c - doajarticles::479ca537c12755d1868bbf02938a900c + 10|doajarticles::479ca537c12755d1868bbf02938a900c - doajarticles::55f31df96a60e2309f45b7c265fcf7a2 + 10|doajarticles::55f31df96a60e2309f45b7c265fcf7a2 - doajarticles::c52a09891a5301f9986ebbfe3761810c + 10|doajarticles::c52a09891a5301f9986ebbfe3761810c - doajarticles::379807bc7f6c71a227ef1651462c414c + 10|doajarticles::379807bc7f6c71a227ef1651462c414c - doajarticles::36069db531a00b85a2e8fb301f4bdc19 + 10|doajarticles::36069db531a00b85a2e8fb301f4bdc19 - doajarticles::b6a898da311ded96fabf49c520b80d5d + 10|doajarticles::b6a898da311ded96fabf49c520b80d5d - doajarticles::d0753d9180b35a271d8b4a31f449749f + 10|doajarticles::d0753d9180b35a271d8b4a31f449749f - doajarticles::172050a92511838393a3fe237ae47e31 + 10|doajarticles::172050a92511838393a3fe237ae47e31 - doajarticles::301ed96c62abb160a3e29796efe5c95c + 10|doajarticles::301ed96c62abb160a3e29796efe5c95c - doajarticles::0f4f805b3d842f2c7f1b077c3426fa59 + 10|doajarticles::0f4f805b3d842f2c7f1b077c3426fa59 - doajarticles::ba73728b84437b8d48ae287b867c7215 + 10|doajarticles::ba73728b84437b8d48ae287b867c7215 - doajarticles::86faef424d804309ccf45f692523aa48 + 10|doajarticles::86faef424d804309ccf45f692523aa48 - doajarticles::73bd758fa41671de70964c3ecba013af + 10|doajarticles::73bd758fa41671de70964c3ecba013af - doajarticles::e661fc0bdb24af42b740a08f0ddc6cf4 + 10|doajarticles::e661fc0bdb24af42b740a08f0ddc6cf4 - doajarticles::a6d3052047d5dbfbd43d95b4afb0f3d7 + 10|doajarticles::a6d3052047d5dbfbd43d95b4afb0f3d7 - doajarticles::ca61df07089acc53a1569bde6673d82a + 10|doajarticles::ca61df07089acc53a1569bde6673d82a - doajarticles::237dd6f1606600459d0297abd8ed9976 + 10|doajarticles::237dd6f1606600459d0297abd8ed9976 - doajarticles::fba6191177ede7c51ea1cdf58eae7f8b + 10|doajarticles::fba6191177ede7c51ea1cdf58eae7f8b @@ -1345,87 +1345,87 @@ - doajarticles::c6f0ed5fa41e98863e7c73501fe4bd6d + 10|doajarticles::c6f0ed5fa41e98863e7c73501fe4bd6d - doajarticles::ae4c7286c79590f19fdca670156ce816 + 10|doajarticles::ae4c7286c79590f19fdca670156ce816 - doajarticles::0f664bce92ce953e0c7a92068c46bfb3 + 10|doajarticles::0f664bce92ce953e0c7a92068c46bfb3 - doajarticles::00017183dc4c858fb77541985323a4ef + 10|doajarticles::00017183dc4c858fb77541985323a4ef - doajarticles::93b306f458cce3d7aaaf58c0a725f4f9 + 10|doajarticles::93b306f458cce3d7aaaf58c0a725f4f9 - doajarticles::9dbf8fbf3e9fe0fe1fc01e55fbd90bfc + 10|doajarticles::9dbf8fbf3e9fe0fe1fc01e55fbd90bfc - doajarticles::a2bda8785c863279bba4b8f34827b4c9 + 10|doajarticles::a2bda8785c863279bba4b8f34827b4c9 - doajarticles::019a1fcb42c3fea1c1b689df76330b58 + 10|doajarticles::019a1fcb42c3fea1c1b689df76330b58 - doajarticles::0daa8281938831e9c82bfed8b55a2975 + 10|doajarticles::0daa8281938831e9c82bfed8b55a2975 - doajarticles::f67ad6d268162079b3abd51a24468744 + 10|doajarticles::f67ad6d268162079b3abd51a24468744 - doajarticles::c6f0ed5fa41e98863e7c73501fe4bd6d + 10|doajarticles::c6f0ed5fa41e98863e7c73501fe4bd6d - doajarticles::ad114356e196a4a3d84dda59c720dacd + 10|doajarticles::ad114356e196a4a3d84dda59c720dacd - doajarticles::01e8a54fdecaaf354c67a2dd74ae7d4f + 10|doajarticles::01e8a54fdecaaf354c67a2dd74ae7d4f - doajarticles::449305f096b10a9464449ff2d0e10e06 + 10|doajarticles::449305f096b10a9464449ff2d0e10e06 - doajarticles::982c0c0ac378256254cce2fa6572bb6c + 10|doajarticles::982c0c0ac378256254cce2fa6572bb6c - doajarticles::49d6ed47138884566ce93cf0ccb12c02 + 10|doajarticles::49d6ed47138884566ce93cf0ccb12c02 - doajarticles::a98e820dbc2e8ee0fc84ab66f263267c + 10|doajarticles::a98e820dbc2e8ee0fc84ab66f263267c - doajarticles::50b1ce37427b36368f8f0f1317e47f83 + 10|doajarticles::50b1ce37427b36368f8f0f1317e47f83 - doajarticles::f0ec29b7450b2ac5d0ad45327eeb531a + 10|doajarticles::f0ec29b7450b2ac5d0ad45327eeb531a - doajarticles::d8d421d3b0349a7aaa93758b27a54e84 + 10|doajarticles::d8d421d3b0349a7aaa93758b27a54e84 - doajarticles::7ffc35ac5133da01d421ccf8af5b70bc + 10|doajarticles::7ffc35ac5133da01d421ccf8af5b70bc @@ -1454,81 +1454,81 @@ - opendoar____::358aee4cc897452c00244351e4d91f69 + 10|opendoar____::358aee4cc897452c00244351e4d91f69 {"criteria":[{"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"COVID-19"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"SARS-CoV-2"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"2019-nCoV"}}]} - re3data_____::7b0ad08687b2c960d5aeef06f811d5e6 + 10|re3data_____::7b0ad08687b2c960d5aeef06f811d5e6 {"criteria":[{"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"COVID-19"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"SARS-CoV-2"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"2019-nCoV"}]}]} - driver______::bee53aa31dc2cbb538c10c2b65fa5824 + 10|driver______::bee53aa31dc2cbb538c10c2b65fa5824 {"criteria":[{"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"COVID-19"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"SARS-CoV-2"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"2019-nCoV"}]}]} - openaire____::437f4b072b1aa198adcbc35910ff3b98 + 10|openaire____::437f4b072b1aa198adcbc35910ff3b98 {"criteria":[{"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"COVID-19"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"SARS-CoV-2"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"2019-nCoV"}]}]} - openaire____::081b82f96300b6a6e3d282bad31cb6e2 + 10|openaire____::081b82f96300b6a6e3d282bad31cb6e2 {"criteria":[{"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"COVID-19"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"SARS-CoV-2"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"2019-nCoV"}]}]} - openaire____::9e3be59865b2c1c335d32dae2fe7b254 + 10|openaire____::9e3be59865b2c1c335d32dae2fe7b254 {"criteria":[{"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"COVID-19"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"SARS-CoV-2"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"2019-nCoV"}]}]} - opendoar____::8b6dd7db9af49e67306feb59a8bdc52c + 10|opendoar____::8b6dd7db9af49e67306feb59a8bdc52c {"criteria":[{"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"COVID-19"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"SARS-CoV-2"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"2019-nCoV"}]}]} - share_______::4719356ec8d7d55d3feb384ce879ad6c + 10|share_______::4719356ec8d7d55d3feb384ce879ad6c {"criteria":[{"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"COVID-19"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"SARS-CoV-2"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"2019-nCoV"}]}]} - share_______::bbd802baad85d1fd440f32a7a3a2c2b1 + 10|share_______::bbd802baad85d1fd440f32a7a3a2c2b1 {"criteria":[{"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"COVID-19"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"SARS-CoV-2"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"2019-nCoV"}]}]} - opendoar____::6f4922f45568161a8cdf4ad2299f6d23 + 10|opendoar____::6f4922f45568161a8cdf4ad2299f6d23 {"criteria":[{"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"COVID-19"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"SARS-CoV-2"}]}, {"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"2019-nCoV"}]}]} - re3data_____::7980778c78fb4cf0fab13ce2159030dc + 10|re3data_____::7980778c78fb4cf0fab13ce2159030dc {"criteria":[{"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"2019-nCov"}]}]} - re3data_____::978378def740bbf2bfb420de868c460b + 10|re3data_____::978378def740bbf2bfb420de868c460b {"criteria":[{"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"SARS-CoV-2"}]},{"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"COVID-19"}]},{"constraint":[{"verb":"contains_caseinsensitive","field":"title","value":"2019-nCov"}]}]} From 8eb70e6657870e0d661a5c8619076bc013eb23ee Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Mon, 27 Nov 2023 15:13:15 +0100 Subject: [PATCH 27/60] refactoring --- .../PrepareAffiliationRelationsTest.java | 4 ++-- .../eu/dnetlib/dhp/api/QueryCommunityAPI.java | 21 ++++++++----------- .../PrepareResultCommunitySet.java | 2 +- .../dnetlib/dhp/bulktag/BulkTagJobTest.java | 2 +- 4 files changed, 13 insertions(+), 16 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java index e2639996c..b87738879 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java @@ -79,8 +79,8 @@ public class PrepareAffiliationRelationsTest { .getPath(); String pubmedAffiliationRelationsPath = getClass() - .getResource("/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror.json") - .getPath(); + .getResource("/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror.json") + .getPath(); String outputPath = workingDir.toString() + "/actionSet"; diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java index b7c0164c1..cf33c6509 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/api/QueryCommunityAPI.java @@ -15,8 +15,6 @@ import org.jetbrains.annotations.NotNull; */ public class QueryCommunityAPI { - - private static String get(String geturl) throws IOException { URL url = new URL(geturl); HttpURLConnection conn = (HttpURLConnection) url.openConnection(); @@ -34,32 +32,31 @@ public class QueryCommunityAPI { public static String communities(String baseURL) throws IOException { - return get(baseURL + "communities"); + return get(baseURL + "communities"); } - public static String community(String id, String baseURL ) throws IOException { + public static String community(String id, String baseURL) throws IOException { - return get(baseURL + id); + return get(baseURL + id); } - public static String communityDatasource(String id, String baseURL ) throws IOException { - - return get(baseURL + id + "/contentproviders"); + public static String communityDatasource(String id, String baseURL) throws IOException { + return get(baseURL + id + "/contentproviders"); } - public static String communityPropagationOrganization(String id, String baseURL ) throws IOException { + public static String communityPropagationOrganization(String id, String baseURL) throws IOException { - return get(baseURL + id + "/propagationOrganizations"); + return get(baseURL + id + "/propagationOrganizations"); } - public static String communityProjects(String id, String page, String size, String baseURL ) throws IOException { + public static String communityProjects(String id, String page, String size, String baseURL) throws IOException { - return get(baseURL + id + "/projects/" + page + "/" + size); + return get(baseURL + id + "/projects/" + page + "/" + size); } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java index 2c2eb69dd..7fed2606b 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java @@ -52,7 +52,7 @@ public class PrepareResultCommunitySet { final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); - final String baseURL = parser.get("baseURL"); + final String baseURL = parser.get("baseURL"); log.info("baseUEL: {}", baseURL); final CommunityEntityMap projectsMap = Utils.getCommunityProjects(baseURL); diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java index d193716a8..743e62b5f 100644 --- a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/bulktag/BulkTagJobTest.java @@ -1576,7 +1576,7 @@ public class BulkTagJobTest { "-outputPath", workingDir.toString() + "/", // "-baseURL", "https://services.openaire.eu/openaire/community/", "-pathMap", pathMap, - "-taggingConf", taggingConf + "-taggingConf", taggingConf }); final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); From 6f4d0c05eae2965d5b2afde1d78627cbf2ebbd63 Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Tue, 28 Nov 2023 08:43:56 +0100 Subject: [PATCH 28/60] Implemented Author MErger for ORCID that takes in account the case when name and surname are swapped --- .../eu/dnetlib/dhp/oa/merge/AuthorMerger.java | 114 ++++-------------- .../eu/dnetlib/oa/merge/AuthorMergerTest.java | 33 ++--- .../dhp/oa/merge/authors_orcid_sample.json | 3 + .../oa/merge/authors_publication_sample.json | 2 + .../SparkEnrichGraphWithOrcidAuthors.scala | 15 ++- 5 files changed, 51 insertions(+), 116 deletions(-) create mode 100644 dhp-common/src/test/resources/eu/dnetlib/dhp/oa/merge/authors_orcid_sample.json create mode 100644 dhp-common/src/test/resources/eu/dnetlib/dhp/oa/merge/authors_publication_sample.json diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java index 6c3058303..852ee163d 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java @@ -159,17 +159,11 @@ public class AuthorMerger { .replaceAll("(\\p{Punct})+", " ") .replaceAll("(\\d)+", " ") .replaceAll("(\\n)+", " ") + .trim(); // return Arrays.stream(fullname.split("[\\s | , | ;]+")).map(String::toLowerCase).sorted().collect(Collectors.joining()); } - private static String generateAuthorkey(final Author a) { - if (a.getSurname() == null) - return "NOSURNAME"; - - return normalize(a.getSurname()); - } - // // public static List enrichOrcid2(List baseAuthor, List orcidAuthor) { // if (baseAuthor == null || baseAuthor.isEmpty()) @@ -226,54 +220,26 @@ public class AuthorMerger { return null; } - public static boolean checkSimilarity3(final Author left, final Author right) { - - if (StringUtils.isNotBlank(left.getSurname()) && StringUtils.isNotBlank(left.getName()) - && - StringUtils.isNotBlank(right.getSurname()) && StringUtils.isNotBlank(right.getName()) - - ) - return left.getSurname().equalsIgnoreCase(right.getSurname()) - && left.getName().substring(0, 1).equalsIgnoreCase(right.getName().substring(0, 1)); - - final Person pl = parse(left); - final Person pr = parse(right); - - // If one of them didn't have a surname the match is false - if (!(pl.getSurname() != null && pl.getSurname().stream().anyMatch(StringUtils::isNotBlank) && - pr.getSurname() != null && pr.getSurname().stream().anyMatch(StringUtils::isNotBlank))) - return false; - - // The Authors have one surname in common - if (pl.getSurname().stream().anyMatch(sl -> pr.getSurname().stream().anyMatch(sr -> sr.equalsIgnoreCase(sl)))) { - - // If one of them has only a surname and is the same we can say that they are the same author - if ((pl.getName() == null || pl.getName().stream().allMatch(StringUtils::isBlank)) || - (pr.getName() == null || pr.getName().stream().allMatch(StringUtils::isBlank))) - return true; - // The authors have the same initials of Name in common - if (pl - .getName() - .stream() - .anyMatch( - nl -> pr - .getName() - .stream() - .anyMatch(nr -> nr.substring(0, 1).equalsIgnoreCase(nl.substring(0, 1))))) - return true; - } - return false; - } - public static boolean checkSimilarity2(final Author left, final Author right) { final Person pl = parse(left); final Person pr = parse(right); - // If one of them didn't have a surname the match is false + // If one of them didn't have a surname we verify if they have the fullName not empty + // and verify if the normalized version is equal if (!(pl.getSurname() != null && pl.getSurname().stream().anyMatch(StringUtils::isNotBlank) && - pr.getSurname() != null && pr.getSurname().stream().anyMatch(StringUtils::isNotBlank))) - return false; + pr.getSurname() != null && pr.getSurname().stream().anyMatch(StringUtils::isNotBlank))) { + if (pl.getFullname() != null && !pl.getFullname().isEmpty() && pr.getFullname() != null + && !pr.getFullname().isEmpty()) { + return pl + .getFullname() + .stream() + .anyMatch( + fl -> pr.getFullname().stream().anyMatch(fr -> normalize(fl).equalsIgnoreCase(normalize(fr)))); + } else { + return false; + } + } // The Authors have one surname in common if (pl.getSurname().stream().anyMatch(sl -> pr.getSurname().stream().anyMatch(sr -> sr.equalsIgnoreCase(sl)))) { @@ -292,56 +258,18 @@ public class AuthorMerger { .anyMatch(nr -> nr.substring(0, 1).equalsIgnoreCase(nl.substring(0, 1))))) return true; } - return false; - } - public static boolean checkSimilarity(final Author left, final Author right) { - - if (left.getSurname() == null && left.getFullname() == null) + // Sometimes we noticed that publication have author wrote in inverse order Surname, Name + // We verify if we have an exact match between name and surname + if (pl.getSurname().stream().anyMatch(sl -> pr.getName().stream().anyMatch(nr -> nr.equalsIgnoreCase(sl))) && + pl.getName().stream().anyMatch(nl -> pr.getSurname().stream().anyMatch(sr -> sr.equalsIgnoreCase(nl)))) + return true; + else return false; - if (right.getSurname() == null && right.getFullname() == null) - return false; - - // The Authors have the same surname, or we are tolerant from 1 different char(lets say 1 Typo) - if (StringUtils.isNotBlank(left.getSurname()) && StringUtils.isNotBlank(right.getSurname())) { - if (left.getSurname().equalsIgnoreCase(right.getSurname()) - || hammingDist(left.getSurname().toLowerCase(), right.getSurname().toLowerCase()) < 2) { - // IN case on of the two Authors has no given Name the match is true - if (StringUtils.isBlank(left.getName()) || StringUtils.isBlank(right.getName())) - return true; - // If the surname is correct, and they have the same name or the name starts with the same Letter we can - // say is the same author - if (left.getName().equalsIgnoreCase(right.getName()) - || left.getName().substring(0, 1).equalsIgnoreCase(right.getName().substring(0, 1))) - return true; - } - // Different SURNAME - else { - return false; - } - } else { - // This is the case where the two authors have or the surname or the fullname - // get the first not null of the surname or fullname of both - final String l = authorFieldToBeCompared(left); - final String r = authorFieldToBeCompared(right); - if (l == null || r == null) - return false; - // The same length means they are the same field - if (l.length() == r.length()) { - return normalize(l).equals(normalize(r)); - } - // In this case probably l contains the surname and r contains the fullname - if (l.length() < r.length()) - return normalize(r).contains(normalize(l)); - // In this case probably l contains the fullname and r contains the surname - return normalize(l).contains(normalize(r)); - } - return false; } public static List enrichOrcid2(List baseAuthor, List orcidAuthor) { - final Integer match_itm = 0; if (baseAuthor == null || baseAuthor.isEmpty()) return orcidAuthor; diff --git a/dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java b/dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java index 7f8d673d1..9eccab5f1 100644 --- a/dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java +++ b/dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java @@ -9,6 +9,7 @@ import java.io.IOException; import java.io.InputStreamReader; import java.util.Arrays; import java.util.List; +import java.util.Objects; import java.util.stream.Collectors; import org.junit.jupiter.api.Test; @@ -22,22 +23,15 @@ import eu.dnetlib.dhp.schema.oaf.Author; public class AuthorMergerTest { + @Test - public void testNormalization() { - - assertEquals("bruzzolasandro", AuthorMerger.normalizeFullName("Sandro, La Bruzzo")); - assertEquals("baglionimiriam", AuthorMerger.normalizeFullName("Miriam Baglioni")); - assertEquals("baglionimiriam", AuthorMerger.normalizeFullName("Miriam ;Baglioni,")); - - } - public void testEnrcichAuthor() throws Exception { final ObjectMapper mapper = new ObjectMapper(); BufferedReader pr = new BufferedReader(new InputStreamReader( - AuthorMergerTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/merge/authors_publication.json"))); + Objects.requireNonNull(AuthorMergerTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/merge/authors_publication_sample.json")))); BufferedReader or = new BufferedReader(new InputStreamReader( - AuthorMergerTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/merge/authors_orcid.json"))); + Objects.requireNonNull(AuthorMergerTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/merge/authors_orcid_sample.json")))); TypeReference> aclass = new TypeReference>() { }; @@ -93,32 +87,27 @@ public class AuthorMergerTest { enrichedAuthorWithPid); System.out.println("================="); - - if (++i > 30) - break; } - } - } @Test public void checkSimilarityTest() { final Author left = new Author(); - left.setSurname("Wu"); - left.setName("M."); - left.setFullname("Wu, M."); + left.setName("Anand"); + left.setSurname("Rachna"); + left.setFullname("Anand, Rachna"); System.out.println(AuthorMerger.normalizeFullName(left.getFullname())); final Author right = new Author(); - right.setName("Xin"); - right.setSurname("Wu"); - right.setFullname("Xin Wu"); + right.setName("Rachna"); + right.setSurname("Anand"); + right.setFullname("Rachna, Anand"); // System.out.println(AuthorMerger.normalize(right.getFullname())); boolean same = AuthorMerger.checkSimilarity2(left, right); - assertFalse(same); + assertTrue(same); } diff --git a/dhp-common/src/test/resources/eu/dnetlib/dhp/oa/merge/authors_orcid_sample.json b/dhp-common/src/test/resources/eu/dnetlib/dhp/oa/merge/authors_orcid_sample.json new file mode 100644 index 000000000..ec521b3b7 --- /dev/null +++ b/dhp-common/src/test/resources/eu/dnetlib/dhp/oa/merge/authors_orcid_sample.json @@ -0,0 +1,3 @@ +WrappedArray(arXiv1507.08202) +50|arXiv_dedup_::34e03f2336b8b28286550425e65634ea +[{"fullname":"Liron Barak","name":"Liron","surname":"Barak","rank":null,"pid":[{"value":"0000-0002-3436-2726","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Giuseppe Avolio","name":"Giuseppe","surname":"Avolio","rank":null,"pid":[{"value":"0000-0003-2664-3437","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Tobias Golling","name":"Tobias","surname":"Golling","rank":null,"pid":[{"value":"0000-0001-8535-6687","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Ivan Sykora","name":"Ivan","surname":"Sykora","rank":null,"pid":[{"value":"0000-0003-3447-5621","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Joao Carvalho","name":"Joao","surname":"Carvalho","rank":null,"pid":[{"value":"0000-0002-3015-7821","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jonathan David Long","name":"Jonathan David","surname":"Long","rank":null,"pid":[{"value":"0000-0002-2115-9382","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Heather Russell","name":"Heather","surname":"Russell","rank":null,"pid":[{"value":"0000-0003-4181-0678","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Alison Lister","name":"Alison","surname":"Lister","rank":null,"pid":[{"value":"0000-0002-1552-3651","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Malte Backhaus","name":"Malte","surname":"Backhaus","rank":null,"pid":[{"value":"0000-0002-5888-2304","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Xin Wu","name":"Xin","surname":"Wu","rank":null,"pid":[{"value":"0000-0001-7655-389X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Luke Lambourne","name":"Luke","surname":"Lambourne","rank":null,"pid":[{"value":"0000-0002-7001-7575","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Kazunori Hanagaki","name":"Kazunori","surname":"Hanagaki","rank":null,"pid":[{"value":"0000-0003-0676-0441","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Krzysztof Sliwa","name":"Krzysztof","surname":"Sliwa","rank":null,"pid":[{"value":"0000-0002-1201-4771","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jiri Chudoba","name":"Jiri","surname":"Chudoba","rank":null,"pid":[{"value":"0000-0002-6425-2579","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jörn Große-Knetter","name":"Jörn","surname":"Große-Knetter","rank":null,"pid":[{"value":"0000-0003-3085-7067","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Katharine Leney","name":"Katharine","surname":"Leney","rank":null,"pid":[{"value":"0000-0002-1525-2695","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Michel Lefebvre","name":"Michel","surname":"Lefebvre","rank":null,"pid":[{"value":"0000-0002-5560-0586","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Ahmed Bassalat","name":"Ahmed","surname":"Bassalat","rank":null,"pid":[{"value":"0000-0002-0129-1423","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Borut Kersevan","name":"Borut","surname":"Kersevan","rank":null,"pid":[{"value":"0000-0002-4529-452X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Calum Macdonald","name":"Calum","surname":"Macdonald","rank":null,"pid":[{"value":"0000-0001-7857-9188","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Davide Costanzo","name":"Davide","surname":"Costanzo","rank":null,"pid":[{"value":"0000-0003-4920-6264","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Wladyslaw Dabrowski","name":"Wladyslaw","surname":"Dabrowski","rank":null,"pid":[{"value":"0000-0001-9061-9568","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jean-François Grivaz","name":"Jean-François","surname":"Grivaz","rank":null,"pid":[{"value":"0000-0003-4793-7995","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Elisabetta Pianori","name":"Elisabetta","surname":"Pianori","rank":null,"pid":[{"value":"0000-0001-9233-5892","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Christoph Falk Anders","name":"Christoph Falk","surname":"Anders","rank":null,"pid":[{"value":"0000-0001-6632-6327","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Vasiliki Kouskoura","name":"Vasiliki","surname":"Kouskoura","rank":null,"pid":[{"value":"0000-0002-8987-3208","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Ruth Pöttgen","name":"Ruth","surname":"Pöttgen","rank":null,"pid":[{"value":"0000-0002-3304-0987","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sergey Burdin","name":"Sergey","surname":"Burdin","rank":null,"pid":[{"value":"0000-0003-4831-4132","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Farid Ould-Saada","name":"Farid","surname":"Ould-Saada","rank":null,"pid":[{"value":"0000-0002-9404-835X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Peter Onyisi","name":"Peter","surname":"Onyisi","rank":null,"pid":[{"value":"0000-0003-4201-7997","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Javier Llorente Merino","name":"Javier","surname":"Llorente Merino","rank":null,"pid":[{"value":"0000-0003-0027-7969","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Michel Vetterli","name":"Michel","surname":"Vetterli","rank":null,"pid":[{"value":"0000-0002-7223-2965","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Nicolas Morange","name":"Nicolas","surname":"Morange","rank":null,"pid":[{"value":"0000-0003-0047-7215","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Gianluca Introzzi","name":"Gianluca","surname":"Introzzi","rank":null,"pid":[{"value":"0000-0002-1314-2580","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Roger Moore","name":"Roger","surname":"Moore","rank":null,"pid":[{"value":"0000-0003-4160-4700","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Pierre-Antoine Delsart","name":"Pierre-Antoine","surname":"Delsart","rank":null,"pid":[{"value":"0000-0002-9556-2924","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Victor Solovyev","name":"Victor","surname":"Solovyev","rank":null,"pid":[{"value":"0000-0002-9402-6329","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jiangyong jia","name":"Jiangyong","surname":"jia","rank":null,"pid":[{"value":"0000-0002-5725-3397","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Kyle Cranmer","name":"Kyle","surname":"Cranmer","rank":null,"pid":[{"value":"0000-0002-5769-7094","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Frederik Ruehr","name":"Frederik","surname":"Ruehr","rank":null,"pid":[{"value":"0000-0003-4452-620X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Peter van Gemmeren","name":"Peter","surname":"van Gemmeren","rank":null,"pid":[{"value":"0000-0002-7227-4006","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jacob Kempster","name":"Jacob","surname":"Kempster","rank":null,"pid":[{"value":"0000-0003-4168-3373","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Oleg Brandt","name":"Oleg","surname":"Brandt","rank":null,"pid":[{"value":"0000-0001-5219-1417","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Antonio Policicchio","name":"Antonio","surname":"Policicchio","rank":null,"pid":[{"value":"0000-0002-1290-220X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Fabrice Hubaut","name":"Fabrice","surname":"Hubaut","rank":null,"pid":[{"value":"0000-0002-0113-2465","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Christopher Gorham Lester","name":"Christopher Gorham","surname":"Lester","rank":null,"pid":[{"value":"0000-0001-5770-4883","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Royer Edson Ticse Torres","name":"Royer Edson","surname":"Ticse Torres","rank":null,"pid":[{"value":"0000-0001-8178-5257","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Maria Josefina Alconada Verzini","name":"Maria Josefina","surname":"Alconada Verzini","rank":null,"pid":[{"value":"0000-0003-2212-7830","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"PHILLIP URQUIJO","name":"PHILLIP","surname":"URQUIJO","rank":null,"pid":[{"value":"0000-0002-0887-7953","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Annick Lleres","name":"Annick","surname":"Lleres","rank":null,"pid":[{"value":"0000-0003-1769-8524","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Andrei Snesarev","name":"Andrei","surname":"Snesarev","rank":null,"pid":[{"value":"0000-0002-9067-8362","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Marcin Wolter","name":"Marcin","surname":"Wolter","rank":null,"pid":[{"value":"0000-0001-9184-2921","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Garabed Halladjian","name":"Garabed","surname":"Halladjian","rank":null,"pid":[{"value":"0000-0001-7162-0301","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Anthony Morley","name":"Anthony","surname":"Morley","rank":null,"pid":[{"value":"0000-0003-0373-1346","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Francesco Nuti","name":"Francesco","surname":"Nuti","rank":null,"pid":[{"value":"0000-0003-3491-7637","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Mar Capeans","name":"Mar","surname":"Capeans","rank":null,"pid":[{"value":"0000-0001-7727-9175","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jose Enrique Garcia Navarro","name":"Jose Enrique","surname":"Garcia Navarro","rank":null,"pid":[{"value":"0000-0002-0279-0523","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Aidan Robson","name":"Aidan","surname":"Robson","rank":null,"pid":[{"value":"0000-0002-1659-8284","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Eduardo Ros","name":"Eduardo","surname":"Ros","rank":null,"pid":[{"value":"0000-0003-2812-9554","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Kevin Varvell","name":"Kevin","surname":"Varvell","rank":null,"pid":[{"value":"0000-0003-1017-1295","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Ina Carli","name":"Ina","surname":"Carli","rank":null,"pid":[{"value":"0000-0002-0411-1141","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Ivo van Vulpen","name":"Ivo","surname":"van Vulpen","rank":null,"pid":[{"value":"0000-0001-7074-5655","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Liaoshan Shi","name":"Liaoshan","surname":"Shi","rank":null,"pid":[{"value":"0000-0001-9532-5075","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Pavel Starovoitov","name":"Pavel","surname":"Starovoitov","rank":null,"pid":[{"value":"0000-0003-1990-0992","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Lorenzo Feligioni","name":"Lorenzo","surname":"Feligioni","rank":null,"pid":[{"value":"0000-0002-1403-0951","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Christoph Wasicki","name":"Christoph","surname":"Wasicki","rank":null,"pid":[{"value":"0000-0001-8041-741X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Nuno Castro","name":"Nuno","surname":"Castro","rank":null,"pid":[{"value":"0000-0001-8491-4376","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Mikhail Levchenko","name":"Mikhail","surname":"Levchenko","rank":null,"pid":[{"value":"0000-0002-5495-0656","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Marek Palka","name":"Marek","surname":"Palka","rank":null,"pid":[{"value":"0000-0002-7185-3540","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Philip Allport","name":"Philip","surname":"Allport","rank":null,"pid":[{"value":"0000-0001-7303-2570","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Tomas Jakoubek","name":"Tomas","surname":"Jakoubek","rank":null,"pid":[{"value":"0000-0001-7038-0369","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Elizabeth Brost","name":"Elizabeth","surname":"Brost","rank":null,"pid":[{"value":"0000-0002-6800-9808","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Maciej Trzebinski","name":"Maciej","surname":"Trzebinski","rank":null,"pid":[{"value":"0000-0002-5151-7101","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Nikola Makovec","name":"Nikola","surname":"Makovec","rank":null,"pid":[{"value":"0000-0001-5124-904X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Bernhard Meirose","name":"Bernhard","surname":"Meirose","rank":null,"pid":[{"value":"0000-0003-0032-7022","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Andrea Dell’Acqua","name":"Andrea","surname":"Dell’Acqua","rank":null,"pid":[{"value":"0000-0003-2453-7745","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Carlos Escobar Ibáñez","name":"Carlos","surname":"Escobar Ibáñez","rank":null,"pid":[{"value":"0000-0003-4442-4537","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Hideyuki Oide","name":"Hideyuki","surname":"Oide","rank":null,"pid":[{"value":"0000-0002-2173-3233","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Valentina Maria Martina Cairo","name":"Valentina Maria Martina","surname":"Cairo","rank":null,"pid":[{"value":"0000-0002-0758-7575","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Johannes Erdmann","name":"Johannes","surname":"Erdmann","rank":null,"pid":[{"value":"0000-0002-8073-2740","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Frederic Deliot","name":"Frederic","surname":"Deliot","rank":null,"pid":[{"value":"0000-0003-0777-6031","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Fabian Hügging","name":"Fabian","surname":"Hügging","rank":null,"pid":[{"value":"0000-0002-7472-3151","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Antoine Marzin","name":"Antoine","surname":"Marzin","rank":null,"pid":[{"value":"0000-0003-4364-4351","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sebastien Prince","name":"Sebastien","surname":"Prince","rank":null,"pid":[{"value":"0000-0001-9947-3892","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Melissa Ridel","name":"Melissa","surname":"Ridel","rank":null,"pid":[{"value":"0000-0002-2601-7420","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Christian Weiser","name":"Christian","surname":"Weiser","rank":null,"pid":[{"value":"0000-0002-6456-6834","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Farida Fassi","name":"Farida","surname":"Fassi","rank":null,"pid":[{"value":"0000-0002-6423-7213","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Manuella Vincter","name":"Manuella","surname":"Vincter","rank":null,"pid":[{"value":"0000-0002-5338-8972","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Julien Caudron","name":"Julien","surname":"Caudron","rank":null,"pid":[{"value":"0000-0002-3530-6531","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Giada Mancini","name":"Giada","surname":"Mancini","rank":null,"pid":[{"value":"0000-0001-6158-2751","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Vasiliki A. Mitsou","name":"Vasiliki A.","surname":"Mitsou","rank":null,"pid":[{"value":"0000-0002-1533-8886","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"David Wardrope","name":"David","surname":"Wardrope","rank":null,"pid":[{"value":"0000-0002-8208-2964","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Loïc Valéry","name":"Loïc","surname":"Valéry","rank":null,"pid":[{"value":"0000-0002-5510-1111","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Krisztian Peters","name":"Krisztian","surname":"Peters","rank":null,"pid":[{"value":"0000-0002-7654-1677","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Andrea Knue","name":"Andrea","surname":"Knue","rank":null,"pid":[{"value":"0000-0002-1559-9285","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Stathes Paganis","name":"Stathes","surname":"Paganis","rank":null,"pid":[{"value":"0000-0002-1950-8993","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Steffen Stärz","name":"Steffen","surname":"Stärz","rank":null,"pid":[{"value":"0000-0002-2908-3909","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Emma Torro Pastor","name":"Emma","surname":"Torro Pastor","rank":null,"pid":[{"value":"0000-0002-5507-7924","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Andrey Kiryunin","name":"Andrey","surname":"Kiryunin","rank":null,"pid":[{"value":"0000-0001-7490-6890","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Daniela Marcella Rebuzzi","name":"Daniela Marcella","surname":"Rebuzzi","rank":null,"pid":[{"value":"0000-0003-4461-3880","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Frederick Luehring","name":"Frederick","surname":"Luehring","rank":null,"pid":[{"value":"0000-0001-8721-6901","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"alessandro cerri","name":"alessandro","surname":"cerri","rank":null,"pid":[{"value":"0000-0002-1904-6661","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Remi Lafaye","name":"Remi","surname":"Lafaye","rank":null,"pid":[{"value":"0000-0001-7848-6088","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Torsten Akesson","name":"Torsten","surname":"Akesson","rank":null,"pid":[{"value":"0000-0003-4141-5408","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Hongbo Zhu","name":"Hongbo","surname":"Zhu","rank":null,"pid":[{"value":"0000-0001-8066-7048","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Paul Laycock","name":"Paul","surname":"Laycock","rank":null,"pid":[{"value":"0000-0002-8572-5339","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Paolo Giromini","name":"Paolo","surname":"Giromini","rank":null,"pid":[{"value":"0000-0003-0276-287X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Leszek Adamczyk","name":"Leszek","surname":"Adamczyk","rank":null,"pid":[{"value":"0000-0002-5859-2075","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Stephen Lloyd","name":"Stephen","surname":"Lloyd","rank":null,"pid":[{"value":"0000-0002-5073-2264","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Aaron Angerami","name":"Aaron","surname":"Angerami","rank":null,"pid":[{"value":"0000-0001-7834-8750","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Susumu Oda","name":"Susumu","surname":"Oda","rank":null,"pid":[{"value":"0000-0001-5836-768X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Nathalie Besson","name":"Nathalie","surname":"Besson","rank":null,"pid":[{"value":"0000-0001-9248-6252","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"MARCELLO FANTI","name":"MARCELLO","surname":"FANTI","rank":null,"pid":[{"value":"0000-0002-8773-145X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Peter Kodyš","name":"Peter","surname":"Kodyš","rank":null,"pid":[{"value":"0000-0002-8644-2349","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Teng Jian Khoo","name":"Teng Jian","surname":"Khoo","rank":null,"pid":[{"value":"0000-0002-5954-3101","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Javier Montejo Berlingen","name":"Javier","surname":"Montejo Berlingen","rank":null,"pid":[{"value":"0000-0001-9213-904X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sandro Palestini","name":"Sandro","surname":"Palestini","rank":null,"pid":[{"value":"0000-0002-4110-096X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Mark Oreglia","name":"Mark","surname":"Oreglia","rank":null,"pid":[{"value":"0000-0001-6203-2209","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Krzysztof Korcyl","name":"Krzysztof","surname":"Korcyl","rank":null,"pid":[{"value":"0000-0001-8085-4505","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sebastian Schmitt","name":"Sebastian","surname":"Schmitt","rank":null,"pid":[{"value":"0000-0002-7935-0470","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Irinel Caprini","name":"Irinel","surname":"Caprini","rank":null,"pid":[{"value":"0000-0003-3343-3200","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Anthony Doyle","name":"Anthony","surname":"Doyle","rank":null,"pid":[{"value":"0000-0001-6322-6195","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Markus Elsing","name":"Markus","surname":"Elsing","rank":null,"pid":[{"value":"0000-0002-1213-0545","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Christian Schmitt","name":"Christian","surname":"Schmitt","rank":null,"pid":[{"value":"0000-0003-1471-690X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Konstantinos Nikolopoulos","name":"Konstantinos","surname":"Nikolopoulos","rank":null,"pid":[{"value":"0000-0002-3048-489X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Janet Dietrich","name":"Janet","surname":"Dietrich","rank":null,"pid":[{"value":"0000-0001-7061-1585","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Yuri Kulchitsky","name":"Yuri","surname":"Kulchitsky","rank":null,"pid":[{"value":"0000-0002-3036-5575","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sinead Farrington","name":"Sinead","surname":"Farrington","rank":null,"pid":[{"value":"0000-0001-5350-9271","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Stefano Terzo","name":"Stefano","surname":"Terzo","rank":null,"pid":[{"value":"0000-0003-3388-3906","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"stefania xella","name":"stefania","surname":"xella","rank":null,"pid":[{"value":"0000-0002-0988-1655","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jonathan Butterworth","name":"Jonathan","surname":"Butterworth","rank":null,"pid":[{"value":"0000-0002-5905-5394","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Gideon Bella","name":"Gideon","surname":"Bella","rank":null,"pid":[{"value":"0000-0002-4009-0990","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Marcello Bindi","name":"Marcello","surname":"Bindi","rank":null,"pid":[{"value":"0000-0001-6172-545X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Heather Gray","name":"Heather","surname":"Gray","rank":null,"pid":[{"value":"0000-0002-5293-4716","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Marcel Vos","name":"Marcel","surname":"Vos","rank":null,"pid":[{"value":"0000-0001-8474-5357","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Ljiljana Morvaj","name":"Ljiljana","surname":"Morvaj","rank":null,"pid":[{"value":"0000-0003-2061-2904","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Didier Ferrere","name":"Didier","surname":"Ferrere","rank":null,"pid":[{"value":"0000-0002-5687-9240","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Mauro Villa","name":"Mauro","surname":"Villa","rank":null,"pid":[{"value":"0000-0002-9181-8048","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Deepak Kar","name":"Deepak","surname":"Kar","rank":null,"pid":[{"value":"0000-0002-4238-9822","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Frank Berghaus","name":"Frank","surname":"Berghaus","rank":null,"pid":[{"value":"0000-0003-1887-3910","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Bruce Yabsley","name":"Bruce","surname":"Yabsley","rank":null,"pid":[{"value":"0000-0002-2680-0474","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Frank Winklmeier","name":"Frank","surname":"Winklmeier","rank":null,"pid":[{"value":"0000-0001-8290-3200","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Zoya Karpova","name":"Zoya","surname":"Karpova","rank":null,"pid":[{"value":"0000-0003-0254-4629","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jason Nielsen","name":"Jason","surname":"Nielsen","rank":null,"pid":[{"value":"0000-0002-9175-4419","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Claire Gwenlan","name":"Claire","surname":"Gwenlan","rank":null,"pid":[{"value":"0000-0002-3518-0617","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Stephanie Majewski","name":"Stephanie","surname":"Majewski","rank":null,"pid":[{"value":"0000-0002-6871-3395","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"James Mueller","name":"James","surname":"Mueller","rank":null,"pid":[{"value":"0000-0001-5099-4718","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Mario Giordani","name":"Mario","surname":"Giordani","rank":null,"pid":[{"value":"0000-0002-0792-6039","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Laura Fabbri","name":"Laura","surname":"Fabbri","rank":null,"pid":[{"value":"0000-0002-4002-8353","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Michael Begel","name":"Michael","surname":"Begel","rank":null,"pid":[{"value":"0000-0002-1634-4399","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Laurent Schoeffel","name":"Laurent","surname":"Schoeffel","rank":null,"pid":[{"value":"0000-0002-8081-2353","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Bobby Acharya","name":"Bobby","surname":"Acharya","rank":null,"pid":[{"value":"0000-0002-8588-9157","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jan Kretzschmar","name":"Jan","surname":"Kretzschmar","rank":null,"pid":[{"value":"0000-0002-8515-1355","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Thi Ngoc Loan Truong","name":"Thi Ngoc Loan","surname":"Truong","rank":null,"pid":[{"value":"0000-0001-8249-7150","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Yury Smirnov","name":"Yury","surname":"Smirnov","rank":null,"pid":[{"value":"0000-0002-2891-0781","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Andreas Warburton","name":"Andreas","surname":"Warburton","rank":null,"pid":[{"value":"0000-0002-2298-7315","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Osamu Jinnouchi","name":"Osamu","surname":"Jinnouchi","rank":null,"pid":[{"value":"0000-0001-5073-0974","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Nicola Orlando","name":"Nicola","surname":"Orlando","rank":null,"pid":[{"value":"0000-0003-0616-245X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Magda Anna Chelstowska","name":"Magda Anna","surname":"Chelstowska","rank":null,"pid":[{"value":"0000-0003-1030-2099","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Armin Nairz","name":"Armin","surname":"Nairz","rank":null,"pid":[{"value":"0000-0003-3561-0880","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Chris Malena Delitzsch","name":"Chris Malena","surname":"Delitzsch","rank":null,"pid":[{"value":"0000-0001-7021-3333","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Kendall Reeves","name":"Kendall","surname":"Reeves","rank":null,"pid":[{"value":"0000-0003-3504-4882","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Minghui Liu","name":"Minghui","surname":"Liu","rank":null,"pid":[{"value":"0000-0003-0056-7296","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Clara Troncon","name":"Clara","surname":"Troncon","rank":null,"pid":[{"value":"0000-0002-7997-8524","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Chris Hays","name":"Chris","surname":"Hays","rank":null,"pid":[{"value":"0000-0003-2371-9723","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Driss Benchekroun","name":"Driss","surname":"Benchekroun","rank":null,"pid":[{"value":"0000-0001-5196-8327","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Tamar Djobava","name":"Tamar","surname":"Djobava","rank":null,"pid":[{"value":"0000-0002-9414-8350","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Joany Manjarrés Ramos","name":"Joany","surname":"Manjarrés Ramos","rank":null,"pid":[{"value":"0000-0003-3896-5222","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Craig Wiglesworth","name":"Craig","surname":"Wiglesworth","rank":null,"pid":[{"value":"0000-0001-6219-8946","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Lawrence Lee","name":"Lawrence","surname":"Lee","rank":null,"pid":[{"value":"0000-0002-5590-335X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Elisabetta Barberio","name":"Elisabetta","surname":"Barberio","rank":null,"pid":[{"value":"0000-0002-3111-0910","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Robert McPherson","name":"Robert","surname":"McPherson","rank":null,"pid":[{"value":"0000-0001-9211-7019","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Paul Jackson","name":"Paul","surname":"Jackson","rank":null,"pid":[{"value":"0000-0002-0847-402X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Peter Henrik Hansen","name":"Peter Henrik","surname":"Hansen","rank":null,"pid":[{"value":"0000-0002-6764-4789","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Cristobal Padilla","name":"Cristobal","surname":"Padilla","rank":null,"pid":[{"value":"0000-0001-7951-0166","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Nora Pettersson","name":"Nora","surname":"Pettersson","rank":null,"pid":[{"value":"0000-0001-7451-3544","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Paul Glaysher","name":"Paul","surname":"Glaysher","rank":null,"pid":[{"value":"0000-0002-5437-971X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Patrick Czodrowski","name":"Patrick","surname":"Czodrowski","rank":null,"pid":[{"value":"0000-0003-0723-1437","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"María Moreno Llácer","name":"María","surname":"Moreno Llácer","rank":null,"pid":[{"value":"0000-0003-1113-3645","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Soumya Mohapatra","name":"Soumya","surname":"Mohapatra","rank":null,"pid":[{"value":"0000-0003-3006-6337","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Andrea Gaudiello","name":"Andrea","surname":"Gaudiello","rank":null,"pid":[{"value":"0000-0001-7721-8217","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Maximiliano Sioli","name":"Maximiliano","surname":"Sioli","rank":null,"pid":[{"value":"0000-0002-0912-9121","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Christian Schroeder","name":"Christian","surname":"Schroeder","rank":null,"pid":[{"value":"0000-0001-6449-0668","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Guillaume Unal","name":"Guillaume","surname":"Unal","rank":null,"pid":[{"value":"0000-0001-8130-7423","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Christian Gutschow","name":"Christian","surname":"Gutschow","rank":null,"pid":[{"value":"0000-0003-0857-794X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Spyridon Argyropoulos","name":"Spyridon","surname":"Argyropoulos","rank":null,"pid":[{"value":"0000-0001-7748-1429","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Vadim Bednyakov","name":"Vadim","surname":"Bednyakov","rank":null,"pid":[{"value":"0000-0003-4864-8909","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Caterina Doglioni","name":"Caterina","surname":"Doglioni","rank":null,"pid":[{"value":"0000-0002-1509-0390","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Andrew Haas","name":"Andrew","surname":"Haas","rank":null,"pid":[{"value":"0000-0002-4832-0455","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Masahiro Morii","name":"Masahiro","surname":"Morii","rank":null,"pid":[{"value":"0000-0001-9324-057X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Andrea Coccaro","name":"Andrea","surname":"Coccaro","rank":null,"pid":[{"value":"0000-0003-2368-4559","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Edward Moyse","name":"Edward","surname":"Moyse","rank":null,"pid":[{"value":"0000-0003-4449-6178","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Takuya Nobe","name":"Takuya","surname":"Nobe","rank":null,"pid":[{"value":"0000-0002-5809-325X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jahred Adelman","name":"Jahred","surname":"Adelman","rank":null,"pid":[{"value":"0000-0002-1041-3496","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Antonio Ereditato","name":"Antonio","surname":"Ereditato","rank":null,"pid":[{"value":"0000-0002-5423-8079","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Stefan Schmitt","name":"Stefan","surname":"Schmitt","rank":null,"pid":[{"value":"0000-0001-8387-1853","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"David DeMarco","name":"David","surname":"DeMarco","rank":null,"pid":[{"value":"0000-0002-8921-8828","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Alessandro La Rosa","name":"Alessandro","surname":"La Rosa","rank":null,"pid":[{"value":"0000-0001-6291-2142","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Liza Mijovic","name":"Liza","surname":"Mijovic","rank":null,"pid":[{"value":"0000-0003-0162-2891","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Stephane Willocq","name":"Stephane","surname":"Willocq","rank":null,"pid":[{"value":"0000-0002-4120-1453","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"George Iakovidis","name":"George","surname":"Iakovidis","rank":null,"pid":[{"value":"0000-0002-0330-5921","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Madar Romain","name":"Madar","surname":"Romain","rank":null,"pid":[{"value":"0000-0002-6875-6408","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"giuseppe iacobucci","name":"giuseppe","surname":"iacobucci","rank":null,"pid":[{"value":"0000-0001-9965-5442","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Allen Mincer","name":"Allen","surname":"Mincer","rank":null,"pid":[{"value":"0000-0002-6307-1418","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"TROCME Benjamin","name":"TROCME","surname":"Benjamin","rank":null,"pid":[{"value":"0000-0001-9500-2487","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Liang Li","name":"Liang","surname":"Li","rank":null,"pid":[{"value":"0000-0001-6411-6107","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"William Murray","name":"William","surname":"Murray","rank":null,"pid":[{"value":"0000-0003-1710-6306","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Chaowaroj Wanotayaroj","name":"Chaowaroj","surname":"Wanotayaroj","rank":null,"pid":[{"value":"0000-0002-8178-5705","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Luc Goossens","name":"Luc","surname":"Goossens","rank":null,"pid":[{"value":"0000-0002-2536-4498","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Kerstin Jon-And","name":"Kerstin","surname":"Jon-And","rank":null,"pid":[{"value":"0000-0001-8201-7700","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Arwa Bannoura","name":"Arwa","surname":"Bannoura","rank":null,"pid":[{"value":"0000-0002-7166-8118","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Filipe Veloso","name":"Filipe","surname":"Veloso","rank":null,"pid":[{"value":"0000-0002-5956-4244","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Maximilian Swiatlowski","name":"Maximilian","surname":"Swiatlowski","rank":null,"pid":[{"value":"0000-0001-7287-0468","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Marija Vranjes Milosavljevic","name":"Marija","surname":"Vranjes Milosavljevic","rank":null,"pid":[{"value":"0000-0003-4477-9733","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Riccardo Maria Bianchi","name":"Riccardo Maria","surname":"Bianchi","rank":null,"pid":[{"value":"0000-0001-7345-7798","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Elias Coniavitis","name":"Elias","surname":"Coniavitis","rank":null,"pid":[{"value":"0000-0002-2148-8012","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Ondrej Penc","name":"Ondrej","surname":"Penc","rank":null,"pid":[{"value":"0000-0002-5433-3981","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Julie Kirk","name":"Julie","surname":"Kirk","rank":null,"pid":[{"value":"0000-0001-8096-7577","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Hongtao Yang","name":"Hongtao","surname":"Yang","rank":null,"pid":[{"value":"0000-0003-3554-7113","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Ewelina Maria Lobodzinska","name":"Ewelina Maria","surname":"Lobodzinska","rank":null,"pid":[{"value":"0000-0001-9012-3431","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Christos Leonidopoulos","name":"Christos","surname":"Leonidopoulos","rank":null,"pid":[{"value":"0000-0002-7241-2114","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Silvia Biondi","name":"Silvia","surname":"Biondi","rank":null,"pid":[{"value":"0000-0002-1492-6715","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Stephane JEZEQUEL","name":"Stephane","surname":"JEZEQUEL","rank":null,"pid":[{"value":"0000-0001-7369-6975","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Zhiqing Zhang","name":"Zhiqing","surname":"Zhang","rank":null,"pid":[{"value":"0000-0002-7853-9079","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Kathleen Whalen","name":"Kathleen","surname":"Whalen","rank":null,"pid":[{"value":"0000-0002-9383-8763","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sophie HENROT","name":"Sophie","surname":"HENROT","rank":null,"pid":[{"value":"0000-0003-1218-2991","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Louise Heelan","name":"Louise","surname":"Heelan","rank":null,"pid":[{"value":"0000-0002-4879-0131","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Dominik Derendarz","name":"Dominik","surname":"Derendarz","rank":null,"pid":[{"value":"0000-0001-5660-3095","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Ning Zhou","name":"Ning","surname":"Zhou","rank":null,"pid":[{"value":"0000-0002-1775-2511","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Mihai Caprini","name":"Mihai","surname":"Caprini","rank":null,"pid":[{"value":"0000-0002-6806-6730","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Lee Sawyer","name":"Lee","surname":"Sawyer","rank":null,"pid":[{"value":"0000-0001-8295-0605","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Lydia Roos","name":"Lydia","surname":"Roos","rank":null,"pid":[{"value":"0000-0001-7151-9983","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Stanislav Nemecek","name":"Stanislav","surname":"Nemecek","rank":null,"pid":[{"value":"0000-0001-8978-7150","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Matthias Danninger","name":"Matthias","surname":"Danninger","rank":null,"pid":[{"value":"0000-0002-7807-7484","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Lailin Xu","name":"Lailin","surname":"Xu","rank":null,"pid":[{"value":"0000-0001-8997-3199","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Angel Campoverde","name":"Angel","surname":"Campoverde","rank":null,"pid":[{"value":"0000-0003-1968-1216","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Carl Gwilliam","name":"Carl","surname":"Gwilliam","rank":null,"pid":[{"value":"0000-0002-9401-5304","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Fabrizio Salvatore","name":"Fabrizio","surname":"Salvatore","rank":null,"pid":[{"value":"0000-0002-3709-1554","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Werner Wiedenmann","name":"Werner","surname":"Wiedenmann","rank":null,"pid":[{"value":"0000-0003-3605-3633","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Adriaan König","name":"Adriaan","surname":"König","rank":null,"pid":[{"value":"0000-0001-6702-6473","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Romain Kukla","name":"Romain","surname":"Kukla","rank":null,"pid":[{"value":"0000-0002-1140-2465","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sarah Heim","name":"Sarah","surname":"Heim","rank":null,"pid":[{"value":"0000-0002-2639-6571","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Stamatios Gkaitatzis","name":"Stamatios","surname":"Gkaitatzis","rank":null,"pid":[{"value":"0000-0001-9420-7499","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Alison Elliot","name":"Alison","surname":"Elliot","rank":null,"pid":[{"value":"0000-0003-0921-0314","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Marc Escalier","name":"Marc","surname":"Escalier","rank":null,"pid":[{"value":"0000-0003-4270-2775","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Philipp Stolte","name":"Philipp","surname":"Stolte","rank":null,"pid":[{"value":"0000-0002-8828-3564","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Peter Loch","name":"Peter","surname":"Loch","rank":null,"pid":[{"value":"0000-0002-2005-671X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Tamara Vazquez Schroeder","name":"Tamara","surname":"Vazquez Schroeder","rank":null,"pid":[{"value":"0000-0002-9780-099X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"lily asquith","name":"lily","surname":"asquith","rank":null,"pid":[{"value":"0000-0001-8035-7162","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Javier Sanchez","name":"Javier","surname":"Sanchez","rank":null,"pid":[{"value":"0000-0001-9913-310X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Tomas Davidek","name":"Tomas","surname":"Davidek","rank":null,"pid":[{"value":"0000-0002-3770-8307","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Lidia Dell'Asta","name":"Lidia","surname":"Dell'Asta","rank":null,"pid":[{"value":"0000-0002-9601-4225","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Francisco Alonso","name":"Francisco","surname":"Alonso","rank":null,"pid":[{"value":"0000-0001-9431-8156","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"J. Katharina Behr","name":"J. Katharina","surname":"Behr","rank":null,"pid":[{"value":"0000-0002-5501-4640","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"MARIO MARTINEZ","name":"MARIO","surname":"MARTINEZ","rank":null,"pid":[{"value":"0000-0002-3135-945X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Konstantinos Bachas","name":"Konstantinos","surname":"Bachas","rank":null,"pid":[{"value":"0000-0002-9047-6517","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Arnaud Lucotte","name":"Arnaud","surname":"Lucotte","rank":null,"pid":[{"value":"0000-0002-5992-0640","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"scott snyder","name":"scott","surname":"snyder","rank":null,"pid":[{"value":"0000-0001-8610-8423","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Edson Carquin","name":"Edson","surname":"Carquin","rank":null,"pid":[{"value":"0000-0002-7863-1166","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Kristin Lohwasser","name":"Kristin","surname":"Lohwasser","rank":null,"pid":[{"value":"0000-0003-1833-9160","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jens Weingarten","name":"Jens","surname":"Weingarten","rank":null,"pid":[{"value":"0000-0003-2165-871X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Victor Maleev","name":"Victor","surname":"Maleev","rank":null,"pid":[{"value":"0000-0003-1028-8602","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Francesca Ungaro","name":"Francesca","surname":"Ungaro","rank":null,"pid":[{"value":"0000-0003-2005-595X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Koji Terashi","name":"Koji","surname":"Terashi","rank":null,"pid":[{"value":"0000-0001-6520-8070","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Kohei Yorita","name":"Kohei","surname":"Yorita","rank":null,"pid":[{"value":"0000-0003-1988-8401","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Roland Jansky","name":"Roland","surname":"Jansky","rank":null,"pid":[{"value":"0000-0003-0456-4658","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sandro De Cecco","name":"Sandro","surname":"De Cecco","rank":null,"pid":[{"value":"0000-0003-4907-8610","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Wade Fisher","name":"Wade","surname":"Fisher","rank":null,"pid":[{"value":"0000-0003-3043-3045","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jaroslav Guenther","name":"Jaroslav","surname":"Guenther","rank":null,"pid":[{"value":"0000-0003-3189-3959","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Tatsuya Masubuchi","name":"Tatsuya","surname":"Masubuchi","rank":null,"pid":[{"value":"0000-0001-9984-8009","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Janusz Chwastowski","name":"Janusz","surname":"Chwastowski","rank":null,"pid":[{"value":"0000-0002-6190-8376","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Ian Watson","name":"Ian","surname":"Watson","rank":null,"pid":[{"value":"0000-0003-2141-3413","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Patrick Rieck","name":"Patrick","surname":"Rieck","rank":null,"pid":[{"value":"0000-0003-0290-0566","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Mario Lassnig","name":"Mario","surname":"Lassnig","rank":null,"pid":[{"value":"0000-0002-9541-0592","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jana Schaarschmidt","name":"Jana","surname":"Schaarschmidt","rank":null,"pid":[{"value":"0000-0002-0433-6439","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Joseph Reichert","name":"Joseph","surname":"Reichert","rank":null,"pid":[{"value":"0000-0003-2110-8021","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Bjarne Stugu","name":"Bjarne","surname":"Stugu","rank":null,"pid":[{"value":"0000-0002-1728-9272","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Paolo Gauzzi","name":"Paolo","surname":"Gauzzi","rank":null,"pid":[{"value":"0000-0003-4841-5822","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sergei Smirnov","name":"Sergei","surname":"Smirnov","rank":null,"pid":[{"value":"0000-0002-6778-073X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Xiangyang Ju","name":"Xiangyang","surname":"Ju","rank":null,"pid":[{"value":"0000-0002-9745-1638","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Fernando Barreiro","name":"Fernando","surname":"Barreiro","rank":null,"pid":[{"value":"0000-0002-3021-0258","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Markus Cristinziani","name":"Markus","surname":"Cristinziani","rank":null,"pid":[{"value":"0000-0003-3893-9171","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Antonio Amorim","name":"Antonio","surname":"Amorim","rank":null,"pid":[{"value":"0000-0003-0638-2321","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sune Jakobsen","name":"Sune","surname":"Jakobsen","rank":null,"pid":[{"value":"0000-0002-6564-040X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Alejandro Alonso","name":"Alejandro","surname":"Alonso","rank":null,"pid":[{"value":"0000-0003-1259-0573","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Makoto Shimojima","name":"Makoto","surname":"Shimojima","rank":null,"pid":[{"value":"0000-0002-8738-1664","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Francesco Guescini","name":"Francesco","surname":"Guescini","rank":null,"pid":[{"value":"0000-0001-5351-2673","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Christina Potter","name":"Christina","surname":"Potter","rank":null,"pid":[{"value":"0000-0002-9815-5208","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Evelina Bouhova-Thacker","name":"Evelina","surname":"Bouhova-Thacker","rank":null,"pid":[{"value":"0000-0002-5103-1558","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Eduard Simioni","name":"Eduard","surname":"Simioni","rank":null,"pid":[{"value":"0000-0002-8929-6236","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Paolo Camarri","name":"Paolo","surname":"Camarri","rank":null,"pid":[{"value":"0000-0002-5732-5645","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"William Leight","name":"William","surname":"Leight","rank":null,"pid":[{"value":"0000-0002-2968-7841","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Luis Roberto Flores Castillo","name":"Luis Roberto","surname":"Flores Castillo","rank":null,"pid":[{"value":"0000-0003-1551-5974","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Alessandro null","name":"Alessandro","surname":null,"rank":null,"pid":[{"value":"0000-0002-8224-6105","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Roman Lysak","name":"Roman","surname":"Lysak","rank":null,"pid":[{"value":"0000-0003-2990-1673","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Matteo Franchini","name":"Matteo","surname":"Franchini","rank":null,"pid":[{"value":"0000-0002-4554-252X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Martin Nagel","name":"Martin","surname":"Nagel","rank":null,"pid":[{"value":"0000-0002-2588-6691","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Harald Fox","name":"Harald","surname":"Fox","rank":null,"pid":[{"value":"0000-0003-3089-6090","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Klaus Moenig","name":"Klaus","surname":"Moenig","rank":null,"pid":[{"value":"0000-0002-3169-7117","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sergey Karpov","name":"Sergey","surname":"Karpov","rank":null,"pid":[{"value":"0000-0002-2230-5353","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Remi Zaidan","name":"Remi","surname":"Zaidan","rank":null,"pid":[{"value":"0000-0002-3710-4554","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Kilian Rosbach","name":"Kilian","surname":"Rosbach","rank":null,"pid":[{"value":"0000-0002-4241-2949","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Pawel Klimek","name":"Pawel","surname":"Klimek","rank":null,"pid":[{"value":"0000-0003-1661-6873","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Nikolaos Konstantinidis","name":"Nikolaos","surname":"Konstantinidis","rank":null,"pid":[{"value":"0000-0002-4140-6360","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Alberto Mengarelli","name":"Alberto","surname":"Mengarelli","rank":null,"pid":[{"value":"0000-0002-1884-854X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Robert Kowalewski","name":"Robert","surname":"Kowalewski","rank":null,"pid":[{"value":"0000-0002-7314-0990","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Miguel Arratia","name":"Miguel","surname":"Arratia","rank":null,"pid":[{"value":"0000-0001-6877-3315","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Paola Giannetti","name":"Paola","surname":"Giannetti","rank":null,"pid":[{"value":"0000-0002-3721-9490","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Joern Lange","name":"Joern","surname":"Lange","rank":null,"pid":[{"value":"0000-0003-1307-1441","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Pavel Řezníček","name":"Pavel","surname":"Řezníček","rank":null,"pid":[{"value":"0000-0003-4017-9829","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Vojtech Pleskot","name":"Vojtech","surname":"Pleskot","rank":null,"pid":[{"value":"0000-0001-5435-497X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Federico Sforza","name":"Federico","surname":"Sforza","rank":null,"pid":[{"value":"0000-0002-4065-7352","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Danijela Bogavac","name":"Danijela","surname":"Bogavac","rank":null,"pid":[{"value":"0000-0003-2138-9062","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Nenad Vranjes","name":"Nenad","surname":"Vranjes","rank":null,"pid":[{"value":"0000-0001-5415-5225","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Peter Watkins","name":"Peter","surname":"Watkins","rank":null,"pid":[{"value":"0000-0002-1290-6833","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Yohei Yamaguchi","name":"Yohei","surname":"Yamaguchi","rank":null,"pid":[{"value":"0000-0002-3725-4800","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Matthias Saimpert","name":"Matthias","surname":"Saimpert","rank":null,"pid":[{"value":"0000-0002-3765-1320","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Alexander Grohsjean","name":"Alexander","surname":"Grohsjean","rank":null,"pid":[{"value":"0000-0003-0748-8494","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Tibor Zenis","name":"Tibor","surname":"Zenis","rank":null,"pid":[{"value":"0000-0001-8265-6916","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Stefano Camarda","name":"Stefano","surname":"Camarda","rank":null,"pid":[{"value":"0000-0003-0479-7689","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Carlos Lacasta","name":"Carlos","surname":"Lacasta","rank":null,"pid":[{"value":"0000-0002-2623-6252","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Oliver Ricken","name":"Oliver","surname":"Ricken","rank":null,"pid":[{"value":"0000-0001-5107-7276","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Claude Leroy","name":"Claude","surname":"Leroy","rank":null,"pid":[{"value":"0000-0003-3105-7045","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Murrough Landon","name":"Murrough","surname":"Landon","rank":null,"pid":[{"value":"0000-0001-6828-9769","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Martin White","name":"Martin","surname":"White","rank":null,"pid":[{"value":"0000-0001-5474-4580","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Dimitrii Krasnopevtsev","name":"Dimitrii","surname":"Krasnopevtsev","rank":null,"pid":[{"value":"0000-0002-6356-372X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Susana Cabrera Urbán","name":"Susana","surname":"Cabrera Urbán","rank":null,"pid":[{"value":"0000-0001-7640-7913","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Hans-Christian Schultz-Coulon","name":"Hans-Christian","surname":"Schultz-Coulon","rank":null,"pid":[{"value":"0000-0002-0860-7240","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Stephen Jiggins","name":"Stephen","surname":"Jiggins","rank":null,"pid":[{"value":"0000-0003-2906-1977","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Andre Sopczak","name":"Andre","surname":"Sopczak","rank":null,"pid":[{"value":"0000-0001-6981-0544","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Cristinel Diaconu","name":"Cristinel","surname":"Diaconu","rank":null,"pid":[{"value":"0000-0002-6193-5091","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Hector de la Torre Perez","name":"Hector","surname":"de la Torre Perez","rank":null,"pid":[{"value":"0000-0002-4516-5269","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Costa Mezquita","name":"Costa","surname":"Mezquita","rank":null,"pid":[{"value":"0000-0002-2064-2954","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sahal Yacoob","name":"Sahal","surname":"Yacoob","rank":null,"pid":[{"value":"0000-0001-6977-3456","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jian Liu","name":"Jian","surname":"Liu","rank":null,"pid":[{"value":"0000-0002-8397-7620","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Oxana Smirnova","name":"Oxana","surname":"Smirnova","rank":null,"pid":[{"value":"0000-0003-2517-531X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Thomas Koffas","name":"Thomas","surname":"Koffas","rank":null,"pid":[{"value":"0000-0001-9612-4988","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Stephen Burke","name":"Stephen","surname":"Burke","rank":null,"pid":[{"value":"0000-0002-1962-8493","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Elizaveta Shabalina","name":"Elizaveta","surname":"Shabalina","rank":null,"pid":[{"value":"0000-0003-4849-556X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Nicolas Ellis","name":"Nicolas","surname":"Ellis","rank":null,"pid":[{"value":"0000-0002-1920-4930","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Robert Astalos","name":"Robert","surname":"Astalos","rank":null,"pid":[{"value":"0000-0001-5095-605X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Valerio Dao","name":"Valerio","surname":"Dao","rank":null,"pid":[{"value":"0000-0003-1645-8393","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Javier Alberto Murillo Quijada","name":"Javier Alberto","surname":"Murillo Quijada","rank":null,"pid":[{"value":"0000-0003-4933-2092","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Alan Litke","name":"Alan","surname":"Litke","rank":null,"pid":[{"value":"0000-0003-3973-3642","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Hans Peter Beck","name":"Hans Peter","surname":"Beck","rank":null,"pid":[{"value":"0000-0001-7212-1096","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Flavia De Almeida Dias","name":"Flavia","surname":"De Almeida Dias","rank":null,"pid":[{"value":"0000-0001-6882-5402","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Mykhailo Lisovyi","name":"Mykhailo","surname":"Lisovyi","rank":null,"pid":[{"value":"0000-0002-3014-5855","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Frank Ellinghaus","name":"Frank","surname":"Ellinghaus","rank":null,"pid":[{"value":"0000-0003-3596-5331","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Alexey Ezhilov","name":"Alexey","surname":"Ezhilov","rank":null,"pid":[{"value":"0000-0002-7520-293X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Anna Sfyrla","name":"Anna","surname":"Sfyrla","rank":null,"pid":[{"value":"0000-0002-3003-9905","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Marc-André Pleier","name":"Marc-André","surname":"Pleier","rank":null,"pid":[{"value":"0000-0002-9461-3494","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Aliaksei Hrynevich","name":"Aliaksei","surname":"Hrynevich","rank":null,"pid":[{"value":"0000-0002-5411-114X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Lucia Masetti","name":"Lucia","surname":"Masetti","rank":null,"pid":[{"value":"0000-0002-0038-5372","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Petr Balek","name":"Petr","surname":"Balek","rank":null,"pid":[{"value":"0000-0002-0942-1966","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Marco Sessa","name":"Marco","surname":"Sessa","rank":null,"pid":[{"value":"0000-0002-1402-7525","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Ilija Vukotic","name":"Ilija","surname":"Vukotic","rank":null,"pid":[{"value":"0000-0003-0472-3516","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Tova Holmes","name":"Tova","surname":"Holmes","rank":null,"pid":[{"value":"0000-0002-3959-5174","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Hermann Kolanoski","name":"Hermann","surname":"Kolanoski","rank":null,"pid":[{"value":"0000-0003-0435-2524","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sigve Haug","name":"Sigve","surname":"Haug","rank":null,"pid":[{"value":"0000-0003-0442-3361","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Bruno Galhardo","name":"Bruno","surname":"Galhardo","rank":null,"pid":[{"value":"0000-0003-0641-301X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Nello Bruscino","name":"Nello","surname":"Bruscino","rank":null,"pid":[{"value":"0000-0002-6168-689X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jonas Strandberg","name":"Jonas","surname":"Strandberg","rank":null,"pid":[{"value":"0000-0002-8913-0981","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Riccardo Vari","name":"Riccardo","surname":"Vari","rank":null,"pid":[{"value":"0000-0002-2814-1337","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Emmanuel Sauvan","name":"Emmanuel","surname":"Sauvan","rank":null,"pid":[{"value":"0000-0003-1921-2647","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Hartmut Sadrozinski","name":"Hartmut","surname":"Sadrozinski","rank":null,"pid":[{"value":"0000-0003-0019-5410","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Juerg Beringer","name":"Juerg","surname":"Beringer","rank":null,"pid":[{"value":"0000-0002-9975-1781","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"laurent chevalier","name":"laurent","surname":"chevalier","rank":null,"pid":[{"value":"0000-0003-3762-7264","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Dominik Duda","name":"Dominik","surname":"Duda","rank":null,"pid":[{"value":"0000-0002-5916-3467","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Takanori Kono","name":"Takanori","surname":"Kono","rank":null,"pid":[{"value":"0000-0003-1553-2950","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Igor Gorelov","name":"Igor","surname":"Gorelov","rank":null,"pid":[{"value":"0000-0001-5570-0133","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Noam Tal Hod","name":"Noam","surname":"Tal Hod","rank":null,"pid":[{"value":"0000-0001-5241-0544","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Umberto De Sanctis","name":"Umberto","surname":"De Sanctis","rank":null,"pid":[{"value":"0000-0003-4704-525X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Andrii Tykhonov","name":"Andrii","surname":"Tykhonov","rank":null,"pid":[{"value":"0000-0003-2908-7915","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Francois Corriveau","name":"Francois","surname":"Corriveau","rank":null,"pid":[{"value":"0000-0002-4970-7600","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Michaela Queitsch-Maitland","name":"Michaela","surname":"Queitsch-Maitland","rank":null,"pid":[{"value":"0000-0003-4643-515X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Iacopo Vivarelli","name":"Iacopo","surname":"Vivarelli","rank":null,"pid":[{"value":"0000-0003-0097-123X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"sotirios vlachos","name":"sotirios","surname":"vlachos","rank":null,"pid":[{"value":"0000-0002-1879-3745","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Archil Durglishvili","name":"Archil","surname":"Durglishvili","rank":null,"pid":[{"value":"0000-0003-4157-592X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Vladimir Cindro","name":"Vladimir","surname":"Cindro","rank":null,"pid":[{"value":"0000-0002-2037-7185","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Tim Adye","name":"Tim","surname":"Adye","rank":null,"pid":[{"value":"0000-0003-0627-5059","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Ruggero Turra","name":"Ruggero","surname":"Turra","rank":null,"pid":[{"value":"0000-0001-8740-796X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Susana Amor Santos","name":"Susana","surname":"Amor Santos","rank":null,"pid":[{"value":"0000-0001-7566-6067","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Thorsten Wengler","name":"Thorsten","surname":"Wengler","rank":null,"pid":[{"value":"0000-0002-4375-5265","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sergio Grancagnolo","name":"Sergio","surname":"Grancagnolo","rank":null,"pid":[{"value":"0000-0001-8490-8304","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Koji Sato","name":"Koji","surname":"Sato","rank":null,"pid":[{"value":"0000-0001-8988-4065","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Marjorie Shapiro","name":"Marjorie","surname":"Shapiro","rank":null,"pid":[{"value":"0000-0001-8540-9654","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Paul Thompson","name":"Paul","surname":"Thompson","rank":null,"pid":[{"value":"0000-0002-6239-7715","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Daniele Zanzi","name":"Daniele","surname":"Zanzi","rank":null,"pid":[{"value":"0000-0002-1222-7937","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Adrian Irles","name":"Adrian","surname":"Irles","rank":null,"pid":[{"value":"0000-0001-5668-151X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Walter Hopkins","name":"Walter","surname":"Hopkins","rank":null,"pid":[{"value":"0000-0001-7814-8740","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Aurelio Juste","name":"Aurelio","surname":"Juste","rank":null,"pid":[{"value":"0000-0002-1558-3291","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Yoram Rozen","name":"Yoram","surname":"Rozen","rank":null,"pid":[{"value":"0000-0001-6969-0634","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Mateusz Dyndal","name":"Mateusz","surname":"Dyndal","rank":null,"pid":[{"value":"0000-0001-9632-6352","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Monika Wielers","name":"Monika","surname":"Wielers","rank":null,"pid":[{"value":"0000-0001-9232-4827","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Anna Kaczmarska","name":"Anna","surname":"Kaczmarska","rank":null,"pid":[{"value":"0000-0002-8880-4120","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Evgeny Khramov","name":"Evgeny","surname":"Khramov","rank":null,"pid":[{"value":"0000-0001-7400-6454","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Vincent Alexander Croft","name":"Vincent Alexander","surname":"Croft","rank":null,"pid":[{"value":"0000-0002-8731-4525","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Yuji Yamazaki","name":"Yuji","surname":"Yamazaki","rank":null,"pid":[{"value":"0000-0003-3710-6995","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sau Lan Wu","name":"Sau Lan","surname":"Wu","rank":null,"pid":[{"value":"0000-0001-5866-1504","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Kevin Kröninger","name":"Kevin","surname":"Kröninger","rank":null,"pid":[{"value":"0000-0001-9873-0228","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Calin Alexa","name":"Calin","surname":"Alexa","rank":null,"pid":[{"value":"0000-0003-0922-7669","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Frank Filthaut","name":"Frank","surname":"Filthaut","rank":null,"pid":[{"value":"0000-0003-3338-2247","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Alaettin Serhan Mete","name":"Alaettin Serhan","surname":"Mete","rank":null,"pid":[{"value":"0000-0002-5508-530X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Michael Duehrssen-Debling","name":"Michael","surname":"Duehrssen-Debling","rank":null,"pid":[{"value":"0000-0002-5833-7058","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Torre Wenaus","name":"Torre","surname":"Wenaus","rank":null,"pid":[{"value":"0000-0002-8678-893X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Michele LIVAN","name":"Michele","surname":"LIVAN","rank":null,"pid":[{"value":"0000-0002-5877-0062","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jose Guillermo Panduro Vazquez","name":"Jose Guillermo","surname":"Panduro Vazquez","rank":null,"pid":[{"value":"0000-0003-2605-8940","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Pascal Pralavorio","name":"Pascal","surname":"Pralavorio","rank":null,"pid":[{"value":"0000-0002-2452-6715","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Yuto Minami","name":"Yuto","surname":"Minami","rank":null,"pid":[{"value":"0000-0003-2176-8089","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Stephen Hillier","name":"Stephen","surname":"Hillier","rank":null,"pid":[{"value":"0000-0002-7599-6469","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Simon Viel","name":"Simon","surname":"Viel","rank":null,"pid":[{"value":"0000-0001-9554-4059","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Andreas Kugel","name":"Andreas","surname":"Kugel","rank":null,"pid":[{"value":"0000-0002-8493-6660","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Lene Kristian Bryngemark","name":"Lene Kristian","surname":"Bryngemark","rank":null,"pid":[{"value":"0000-0002-8420-3408","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Guenter Duckeck","name":"Guenter","surname":"Duckeck","rank":null,"pid":[{"value":"0000-0002-7756-7801","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Alexey Anisenkov","name":"Alexey","surname":"Anisenkov","rank":null,"pid":[{"value":"0000-0002-7201-5936","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Javier Jiménez Peña","name":"Javier","surname":"Jiménez Peña","rank":null,"pid":[{"value":"0000-0002-8705-628X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sascha null","name":"Sascha","surname":null,"rank":null,"pid":[{"value":"0000-0003-2941-2829","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Evelyn Thomson","name":"Evelyn","surname":"Thomson","rank":null,"pid":[{"value":"0000-0001-6031-2768","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Andrea Formica","name":"Andrea","surname":"Formica","rank":null,"pid":[{"value":"0000-0001-8308-2643","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Mattias Ellert","name":"Mattias","surname":"Ellert","rank":null,"pid":[{"value":"0000-0001-5265-3175","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Richard Keeler","name":"Richard","surname":"Keeler","rank":null,"pid":[{"value":"0000-0002-0510-4189","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Karolos Potamianos","name":"Karolos","surname":"Potamianos","rank":null,"pid":[{"value":"0000-0001-7839-9785","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Muhammad Alhroob","name":"Muhammad","surname":"Alhroob","rank":null,"pid":[{"value":"0000-0001-7569-7111","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Marie-Helene Genest","name":"Marie-Helene","surname":"Genest","rank":null,"pid":[{"value":"0000-0002-4098-2024","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"melissa franklin","name":"melissa","surname":"franklin","rank":null,"pid":[{"value":"0000-0002-6595-883X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"John Baines","name":"John","surname":"Baines","rank":null,"pid":[{"value":"0000-0003-0770-2702","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jean-Baptiste de Vivie","name":"Jean-Baptiste","surname":"de Vivie","rank":null,"pid":[{"value":"0000-0001-9163-2211","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Kristian Gregersen","name":"Kristian","surname":"Gregersen","rank":null,"pid":[{"value":"0000-0003-0295-1670","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Elisabeth Schopf","name":"Elisabeth","surname":"Schopf","rank":null,"pid":[{"value":"0000-0002-9340-2214","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Tristan Beau","name":"Tristan","surname":"Beau","rank":null,"pid":[{"value":"0000-0002-2022-2140","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Ask Emil Løvschall-Jensen","name":"Ask Emil","surname":"Løvschall-Jensen","rank":null,"pid":[{"value":"0000-0003-1834-4904","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Karel Smolek","name":"Karel","surname":"Smolek","rank":null,"pid":[{"value":"0000-0002-5996-7000","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Wendy Taylor","name":"Wendy","surname":"Taylor","rank":null,"pid":[{"value":"0000-0002-6596-9125","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"James Robinson","name":"James","surname":"Robinson","rank":null,"pid":[{"value":"0000-0002-2856-9413","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Arno Straessner","name":"Arno","surname":"Straessner","rank":null,"pid":[{"value":"0000-0003-2460-6659","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Bruno Lenzi","name":"Bruno","surname":"Lenzi","rank":null,"pid":[{"value":"0000-0002-1024-4004","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Thomas G McCarthy","name":"Thomas G","surname":"McCarthy","rank":null,"pid":[{"value":"0000-0002-1182-3526","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Maria Curatolo","name":"Maria","surname":"Curatolo","rank":null,"pid":[{"value":"0000-0003-0978-4879","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Zuzana Rurikova","name":"Zuzana","surname":"Rurikova","rank":null,"pid":[{"value":"0000-0003-3051-9607","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Norman Gee","name":"Norman","surname":"Gee","rank":null,"pid":[{"value":"0000-0002-8833-3154","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Craig Buttar","name":"Craig","surname":"Buttar","rank":null,"pid":[{"value":"0000-0003-0188-6491","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sophie Trincaz-Duvoid","name":"Sophie","surname":"Trincaz-Duvoid","rank":null,"pid":[{"value":"0000-0001-5913-0828","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Danilo Enoque Ferreira de Lima","name":"Danilo Enoque","surname":"Ferreira de Lima","rank":null,"pid":[{"value":"0000-0002-6606-3595","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Nektarios Benekos","name":"Nektarios","surname":"Benekos","rank":null,"pid":[{"value":"0000-0001-7831-8762","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Evgenii Baldin","name":"Evgenii","surname":"Baldin","rank":null,"pid":[{"value":"0000-0002-9854-975X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Craig Sawyer","name":"Craig","surname":"Sawyer","rank":null,"pid":[{"value":"0000-0002-2027-1428","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"George Redlinger","name":"George","surname":"Redlinger","rank":null,"pid":[{"value":"0000-0002-6437-9991","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Philip Sommer","name":"Philip","surname":"Sommer","rank":null,"pid":[{"value":"0000-0003-1703-7304","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Simone Monzani","name":"Simone","surname":"Monzani","rank":null,"pid":[{"value":"0000-0002-0479-2207","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Eirik Gramstad","name":"Eirik","surname":"Gramstad","rank":null,"pid":[{"value":"0000-0001-5792-5352","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Anna Lipniacka","name":"Anna","surname":"Lipniacka","rank":null,"pid":[{"value":"0000-0002-8759-8564","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Oana Vickey Boeriu","name":"Oana","surname":"Vickey Boeriu","rank":null,"pid":[{"value":"0000-0002-6497-6809","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Fridolin Dittus","name":"Fridolin","surname":"Dittus","rank":null,"pid":[{"value":"0000-0002-1760-8237","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Antonio Salvucci","name":"Antonio","surname":"Salvucci","rank":null,"pid":[{"value":"0000-0003-4876-2613","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Carlo Dallapiccola","name":"Carlo","surname":"Dallapiccola","rank":null,"pid":[{"value":"0000-0002-1391-2477","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Konstantinos Ntekas","name":"Konstantinos","surname":"Ntekas","rank":null,"pid":[{"value":"0000-0001-9252-6509","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Juan Antonio Aguilar Saavedra","name":"Juan Antonio","surname":"Aguilar Saavedra","rank":null,"pid":[{"value":"0000-0002-5475-8920","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Martine Bosman","name":"Martine","surname":"Bosman","rank":null,"pid":[{"value":"0000-0002-7290-643X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Trevor Vickey","name":"Trevor","surname":"Vickey","rank":null,"pid":[{"value":"0000-0002-1596-2611","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Christian Ohm","name":"Christian","surname":"Ohm","rank":null,"pid":[{"value":"0000-0002-8015-7512","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"William Davey","name":"William","surname":"Davey","rank":null,"pid":[{"value":"0000-0002-8140-8619","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Wolfgang Walkowiak","name":"Wolfgang","surname":"Walkowiak","rank":null,"pid":[{"value":"0000-0002-0385-3784","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Austin Basye","name":"Austin","surname":"Basye","rank":null,"pid":[{"value":"0000-0002-7519-1310","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Kuhan Wang","name":"Kuhan","surname":"Wang","rank":null,"pid":[{"value":"0000-0002-6151-0034","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Cristiano Alpigiani","name":"Cristiano","surname":"Alpigiani","rank":null,"pid":[{"value":"0000-0002-7641-5814","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"James Monk","name":"James","surname":"Monk","rank":null,"pid":[{"value":"0000-0001-8471-9247","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Christopher Pollard","name":"Christopher","surname":"Pollard","rank":null,"pid":[{"value":"0000-0002-3690-3960","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jozsef Toth","name":"Jozsef","surname":"Toth","rank":null,"pid":[{"value":"0000-0001-9128-6080","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Vitaliano Chiarella","name":"Vitaliano","surname":"Chiarella","rank":null,"pid":[{"value":"0000-0002-4210-2924","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jos Vermeulen","name":"Jos","surname":"Vermeulen","rank":null,"pid":[{"value":"0000-0003-4378-5736","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Yann Coadou","name":"Yann","surname":"Coadou","rank":null,"pid":[{"value":"0000-0001-8195-7004","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Christine Kourkoumelis","name":"Christine","surname":"Kourkoumelis","rank":null,"pid":[{"value":"0000-0003-0083-274X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Karishma Sekhon","name":"Karishma","surname":"Sekhon","rank":null,"pid":[{"value":"0000-0001-7677-8394","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Kun Liu","name":"Kun","surname":"Liu","rank":null,"pid":[{"value":"0000-0001-5807-0501","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Matt LeBlanc","name":"Matt","surname":"LeBlanc","rank":null,"pid":[{"value":"0000-0001-5977-6418","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Ewan Hill","name":"Ewan","surname":"Hill","rank":null,"pid":[{"value":"0000-0002-1725-7414","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Diane Cinca","name":"Diane","surname":"Cinca","rank":null,"pid":[{"value":"0000-0003-0944-8998","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Benedict Tobias Winter","name":"Benedict Tobias","surname":"Winter","rank":null,"pid":[{"value":"0000-0001-9606-7688","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Steven Schramm","name":"Steven","surname":"Schramm","rank":null,"pid":[{"value":"0000-0001-9031-6751","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Yusheng Wu","name":"Yusheng","surname":"Wu","rank":null,"pid":[{"value":"0000-0002-1528-4865","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Orel Gueta","name":"Orel","surname":"Gueta","rank":null,"pid":[{"value":"0000-0002-9440-2398","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Federico Meloni","name":"Federico","surname":"Meloni","rank":null,"pid":[{"value":"0000-0001-7075-2214","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sylvain Tisserant","name":"Sylvain","surname":"Tisserant","rank":null,"pid":[{"value":"0000-0002-0294-6727","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Dimitrios Iliadis","name":"Dimitrios","surname":"Iliadis","rank":null,"pid":[{"value":"0000-0001-6303-2761","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Sebastien Binet","name":"Sebastien","surname":"Binet","rank":null,"pid":[{"value":"0000-0003-4913-6104","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Marcella Bona","name":"Marcella","surname":"Bona","rank":null,"pid":[{"value":"0000-0002-9660-580X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Ki Lie","name":"Ki","surname":"Lie","rank":null,"pid":[{"value":"0000-0002-5779-5989","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"David Sankey","name":"David","surname":"Sankey","rank":null,"pid":[{"value":"0000-0003-0955-4213","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Michele Faucci Giannelli","name":"Michele","surname":"Faucci Giannelli","rank":null,"pid":[{"value":"0000-0003-3731-820X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Georges Azuelos","name":"Georges","surname":"Azuelos","rank":null,"pid":[{"value":"0000-0003-4241-022X","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"David Britton","name":"David","surname":"Britton","rank":null,"pid":[{"value":"0000-0001-9998-4342","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Henri Bachacou","name":"Henri","surname":"Bachacou","rank":null,"pid":[{"value":"0000-0002-2256-4515","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Stefania Stucci","name":"Stefania","surname":"Stucci","rank":null,"pid":[{"value":"0000-0002-1639-4484","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Peter Berta","name":"Peter","surname":"Berta","rank":null,"pid":[{"value":"0000-0003-0780-0345","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"SERKANT ALİ ÇETİN","name":"SERKANT ALİ","surname":"ÇETİN","rank":null,"pid":[{"value":"0000-0001-5050-8441","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Alexey Zhemchugov","name":"Alexey","surname":"Zhemchugov","rank":null,"pid":[{"value":"0000-0002-3360-4965","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Yasushi Nagasaka","name":"Yasushi","surname":"Nagasaka","rank":null,"pid":[{"value":"0000-0002-3669-9525","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"James Walder","name":"James","surname":"Walder","rank":null,"pid":[{"value":"0000-0002-9039-8758","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Marina Rotaru","name":"Marina","surname":"Rotaru","rank":null,"pid":[{"value":"0000-0003-4088-6275","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Chunhui Chen","name":"Chunhui","surname":"Chen","rank":null,"pid":[{"value":"0000-0003-1589-9955","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Valentina Tudorache","name":"Valentina","surname":"Tudorache","rank":null,"pid":[{"value":"0000-0001-5384-3843","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Amanda Cooper-Sarkar","name":"Amanda","surname":"Cooper-Sarkar","rank":null,"pid":[{"value":"0000-0002-7107-5902","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Paul Dervan","name":"Paul","surname":"Dervan","rank":null,"pid":[{"value":"0000-0003-3929-8046","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Laura Gonella","name":"Laura","surname":"Gonella","rank":null,"pid":[{"value":"0000-0002-4919-0808","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Georges Aad","name":"Georges","surname":"Aad","rank":null,"pid":[{"value":"0000-0002-6665-4934","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Hok-Chuen Cheng","name":"Hok-Chuen","surname":"Cheng","rank":null,"pid":[{"value":"0000-0002-8912-4389","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Michal Svatos","name":"Michal","surname":"Svatos","rank":null,"pid":[{"value":"0000-0002-7199-3383","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jens Janssen","name":"Jens","surname":"Janssen","rank":null,"pid":[{"value":"0000-0002-2391-3078","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Jana Faltova","name":"Jana","surname":"Faltova","rank":null,"pid":[{"value":"0000-0003-4278-7182","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Trisha Farooque","name":"Trisha","surname":"Farooque","rank":null,"pid":[{"value":"0000-0003-1363-9324","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"andrea gabrielli","name":"andrea","surname":"gabrielli","rank":null,"pid":[{"value":"0000-0003-0768-9325","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Santiago Gonzalez de la Hoz","name":"Santiago","surname":"Gonzalez de la Hoz","rank":null,"pid":[{"value":"0000-0001-5304-5390","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"José Ocariz","name":"José","surname":"Ocariz","rank":null,"pid":[{"value":"0000-0003-2262-0780","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null},{"fullname":"Michele Weber","name":"Michele","surname":"Weber","rank":null,"pid":[{"value":"0000-0002-2770-9031","qualifier":{"classid":"ORCID","classname":"ORCID","schemeid":"ORCID","schemename":"ORCID"},"dataInfo":null}],"affiliation":null}] diff --git a/dhp-common/src/test/resources/eu/dnetlib/dhp/oa/merge/authors_publication_sample.json b/dhp-common/src/test/resources/eu/dnetlib/dhp/oa/merge/authors_publication_sample.json new file mode 100644 index 000000000..eac239b93 --- /dev/null +++ b/dhp-common/src/test/resources/eu/dnetlib/dhp/oa/merge/authors_publication_sample.json @@ -0,0 +1,2 @@ +50|arXiv_dedup_::34e03f2336b8b28286550425e65634ea +[{"fullname":"Aad, G.","name":"G.","surname":"Aad","rank":1,"pid":[],"affiliation":null},{"fullname":"Ayoub, M.K.","name":"M. K.","surname":"Ayoub","rank":2,"pid":[],"affiliation":null},{"fullname":"Bassalat, A.","name":"A.","surname":"Bassalat","rank":3,"pid":[],"affiliation":null},{"fullname":"Becot, C.","name":"C.","surname":"Becot","rank":4,"pid":[],"affiliation":null},{"fullname":"Binet, S.","name":"S.","surname":"Binet","rank":5,"pid":[],"affiliation":null},{"fullname":"Bourdarios, C.","name":"C.","surname":"Bourdarios","rank":6,"pid":[],"affiliation":null},{"fullname":"Regie, J.B. de Vivie de","name":"J. B. Vivie","surname":"Regie","rank":7,"pid":[],"affiliation":null},{"fullname":"Delgove, D.","name":"D.","surname":"Delgove","rank":8,"pid":[],"affiliation":null},{"fullname":"Duflot, L.","name":"L.","surname":"Duflot","rank":9,"pid":[],"affiliation":null},{"fullname":"Escalier, M.","name":"M.","surname":"Escalier","rank":10,"pid":[],"affiliation":null},{"fullname":"Fayard, L.","name":"L.","surname":"Fayard","rank":11,"pid":[],"affiliation":null},{"fullname":"Fournier, D.","name":"D.","surname":"Fournier","rank":12,"pid":[],"affiliation":null},{"fullname":"Gkougkousis, E.L.","name":"E. L.","surname":"Gkougkousis","rank":13,"pid":[],"affiliation":null},{"fullname":"Grivaz, J.-F.","name":"J. -F","surname":"Grivaz","rank":14,"pid":[],"affiliation":null},{"fullname":"Guillemin, T.","name":"T.","surname":"Guillemin","rank":15,"pid":[],"affiliation":null},{"fullname":"Hariri, F.","name":"F.","surname":"Hariri","rank":16,"pid":[],"affiliation":null},{"fullname":"Henrot-Versillé, S.","name":"S.","surname":"Henrot-Versillé","rank":17,"pid":[],"affiliation":null},{"fullname":"Hrivnac, J.","name":"J.","surname":"Hrivnac","rank":18,"pid":[],"affiliation":null},{"fullname":"Iconomidou-Fayard, L.","name":"L.","surname":"Iconomidou-Fayard","rank":19,"pid":[],"affiliation":null},{"fullname":"Kado, M.","name":"M.","surname":"Kado","rank":20,"pid":[],"affiliation":null},{"fullname":"Lounis, A.","name":"A.","surname":"Lounis","rank":21,"pid":[],"affiliation":null},{"fullname":"Makovec, N.","name":"N.","surname":"Makovec","rank":22,"pid":[],"affiliation":null},{"fullname":"Morange, N.","name":"N.","surname":"Morange","rank":23,"pid":[],"affiliation":null},{"fullname":"Nellist, C.","name":"C.","surname":"Nellist","rank":24,"pid":[],"affiliation":null},{"fullname":"Poggioli, L.","name":"L.","surname":"Poggioli","rank":25,"pid":[],"affiliation":null},{"fullname":"Puzo, P.","name":"P.","surname":"Puzo","rank":26,"pid":[],"affiliation":null},{"fullname":"Renaud, A.","name":"A.","surname":"Renaud","rank":27,"pid":[],"affiliation":null},{"fullname":"Rousseau, D.","name":"D.","surname":"Rousseau","rank":28,"pid":[],"affiliation":null},{"fullname":"Rybkin, G.","name":"G.","surname":"Rybkin","rank":29,"pid":[],"affiliation":null},{"fullname":"Schaffer, A.C.","name":"A. C.","surname":"Schaffer","rank":30,"pid":[],"affiliation":null},{"fullname":"Scifo, E.","name":"E.","surname":"Scifo","rank":31,"pid":[],"affiliation":null},{"fullname":"Serin, L.","name":"L.","surname":"Serin","rank":32,"pid":[],"affiliation":null},{"fullname":"Simion, S.","name":"S.","surname":"Simion","rank":33,"pid":[],"affiliation":null},{"fullname":"Tanaka, R.","name":"R.","surname":"Tanaka","rank":34,"pid":[],"affiliation":null},{"fullname":"Tran, H.L.","name":"H. L.","surname":"Tran","rank":35,"pid":[],"affiliation":null},{"fullname":"Zerwas, D.","name":"D.","surname":"Zerwas","rank":36,"pid":[],"affiliation":null},{"fullname":"Zhang, Zhongkai","name":"Zhongkai","surname":"Zhang","rank":37,"pid":[],"affiliation":null},{"fullname":"Zhao, Y.","name":"Y.","surname":"Zhao","rank":38,"pid":[],"affiliation":null},{"fullname":"Rahal, G.","name":"G.","surname":"Rahal","rank":39,"pid":[],"affiliation":null},{"fullname":"Barnovska, Z.","name":"Z.","surname":"Barnovska","rank":40,"pid":[],"affiliation":null},{"fullname":"Berger, N.","name":"N.","surname":"Berger","rank":41,"pid":[],"affiliation":null},{"fullname":"Delmastro, M.","name":"M.","surname":"Delmastro","rank":42,"pid":[],"affiliation":null},{"fullname":"Ciaccio, L. Di","name":"L. Di","surname":"Ciaccio","rank":43,"pid":[],"affiliation":null},{"fullname":"Elles, S.","name":"S.","surname":"Elles","rank":44,"pid":[],"affiliation":null},{"fullname":"Hryn’ova, T.","name":"T.","surname":"Hryn’ova","rank":45,"pid":[],"affiliation":null},{"fullname":"Jézéquel, S.","name":"S.","surname":"Jézéquel","rank":46,"pid":[],"affiliation":null},{"fullname":"Koletsou, I.","name":"I.","surname":"Koletsou","rank":47,"pid":[],"affiliation":null},{"fullname":"Lafaye, R.","name":"R.","surname":"Lafaye","rank":48,"pid":[],"affiliation":null},{"fullname":"Leveque, J.","name":"J.","surname":"Leveque","rank":49,"pid":[],"affiliation":null},{"fullname":"Massol, N.","name":"N.","surname":"Massol","rank":50,"pid":[],"affiliation":null},{"fullname":"Sauvage, G.","name":"G.","surname":"Sauvage","rank":51,"pid":[],"affiliation":null},{"fullname":"Sauvan, E.","name":"E.","surname":"Sauvan","rank":52,"pid":[],"affiliation":null},{"fullname":"Simard, O.","name":"O.","surname":"Simard","rank":53,"pid":[],"affiliation":null},{"fullname":"Todorov, T.","name":"T.","surname":"Todorov","rank":54,"pid":[],"affiliation":null},{"fullname":"Wingerter-Seez, I.","name":"I.","surname":"Wingerter-Seez","rank":55,"pid":[],"affiliation":null},{"fullname":"Albrand, S.","name":"S.","surname":"Albrand","rank":56,"pid":[],"affiliation":null},{"fullname":"Brown, J.","name":"J.","surname":"Brown","rank":57,"pid":[],"affiliation":null},{"fullname":"Collot, J.","name":"J.","surname":"Collot","rank":58,"pid":[],"affiliation":null},{"fullname":"Crépé-Renaudin, S.","name":"S.","surname":"Crépé-Renaudin","rank":59,"pid":[],"affiliation":null},{"fullname":"Delsart, P.A.","name":"P. A.","surname":"Delsart","rank":60,"pid":[],"affiliation":null},{"fullname":"Gabaldon, C.","name":"C.","surname":"Gabaldon","rank":61,"pid":[],"affiliation":null},{"fullname":"Genest, M.H.","name":"M. H.","surname":"Genest","rank":62,"pid":[],"affiliation":null},{"fullname":"Hostachy, J.Y.","name":"J. Y.","surname":"Hostachy","rank":63,"pid":[],"affiliation":null},{"fullname":"Ledroit-Guillon, F.","name":"F.","surname":"Ledroit-Guillon","rank":64,"pid":[],"affiliation":null},{"fullname":"Lleres, A.","name":"A.","surname":"Lleres","rank":65,"pid":[],"affiliation":null},{"fullname":"Lucotte, A.","name":"A.","surname":"Lucotte","rank":66,"pid":[],"affiliation":null},{"fullname":"Malek, F.","name":"F.","surname":"Malek","rank":67,"pid":[],"affiliation":null},{"fullname":"Monini, C.","name":"C.","surname":"Monini","rank":68,"pid":[],"affiliation":null},{"fullname":"Stark, J.","name":"J.","surname":"Stark","rank":69,"pid":[],"affiliation":null},{"fullname":"Trocmé, B.","name":"B.","surname":"Trocmé","rank":70,"pid":[],"affiliation":null},{"fullname":"Wu, M.","name":"M.","surname":"Wu","rank":71,"pid":[],"affiliation":null},{"fullname":"Alio, L.","name":"L.","surname":"Alio","rank":72,"pid":[],"affiliation":null},{"fullname":"Barbero, M.","name":"M.","surname":"Barbero","rank":73,"pid":[],"affiliation":null},{"fullname":"Coadou, Y.","name":"Y.","surname":"Coadou","rank":74,"pid":[],"affiliation":null},{"fullname":"Diaconu, C.","name":"C.","surname":"Diaconu","rank":75,"pid":[],"affiliation":null},{"fullname":"Diglio, Sara","name":"Sara","surname":"Diglio","rank":76,"pid":[],"affiliation":null},{"fullname":"Djama, F.","name":"F.","surname":"Djama","rank":77,"pid":[],"affiliation":null},{"fullname":"Duccu, O.","name":"O.","surname":"Duccu","rank":78,"pid":[],"affiliation":null},{"fullname":"Feligioni, L.","name":"L.","surname":"Feligioni","rank":79,"pid":[],"affiliation":null},{"fullname":"Gao, J.","name":"J.","surname":"Gao","rank":80,"pid":[],"affiliation":null},{"fullname":"Hallewell, G.D.","name":"G. D.","surname":"Hallewell","rank":81,"pid":[],"affiliation":null},{"fullname":"Hubaut, F.","name":"F.","surname":"Hubaut","rank":82,"pid":[],"affiliation":null},{"fullname":"Kahn, S.J.","name":"S. J.","surname":"Kahn","rank":83,"pid":[],"affiliation":null},{"fullname":"Knoops, E. B. F. G.","name":"E. B. F. G.","surname":"Knoops","rank":84,"pid":[],"affiliation":null},{"fullname":"Guirriec, E. Le","name":"E. Le","surname":"Guirriec","rank":85,"pid":[],"affiliation":null},{"fullname":"Liu, J.","name":"J.","surname":"Liu","rank":86,"pid":[],"affiliation":null},{"fullname":"Liu, K.","name":"K.","surname":"Liu","rank":87,"pid":[],"affiliation":null},{"fullname":"Madaffari, D.","name":"D.","surname":"Madaffari","rank":88,"pid":[],"affiliation":null},{"fullname":"Mochizuki, K.","name":"K.","surname":"Mochizuki","rank":89,"pid":[],"affiliation":null},{"fullname":"Monnier, E.","name":"E.","surname":"Monnier","rank":90,"pid":[],"affiliation":null},{"fullname":"Muanza, S.","name":"S.","surname":"Muanza","rank":91,"pid":[],"affiliation":null},{"fullname":"Nagai, Y.","name":"Y.","surname":"Nagai","rank":92,"pid":[],"affiliation":null},{"fullname":"Nagy, E.","name":"E.","surname":"Nagy","rank":93,"pid":[],"affiliation":null},{"fullname":"Pralavorio, P.","name":"P.","surname":"Pralavorio","rank":94,"pid":[],"affiliation":null},{"fullname":"Rozanov, A.","name":"A.","surname":"Rozanov","rank":95,"pid":[],"affiliation":null},{"fullname":"Serre, T.","name":"T.","surname":"Serre","rank":96,"pid":[],"affiliation":null},{"fullname":"Talby, M.","name":"M.","surname":"Talby","rank":97,"pid":[],"affiliation":null},{"fullname":"Torres, R.E. Ticse","name":"R. E. Ticse","surname":"Torres","rank":98,"pid":[],"affiliation":null},{"fullname":"Tiouchichine, E.","name":"E.","surname":"Tiouchichine","rank":99,"pid":[],"affiliation":null},{"fullname":"Tisserant, S.","name":"S.","surname":"Tisserant","rank":100,"pid":[],"affiliation":null},{"fullname":"Toth, J.","name":"J.","surname":"Toth","rank":101,"pid":[],"affiliation":null},{"fullname":"Touchard, F.","name":"F.","surname":"Touchard","rank":102,"pid":[],"affiliation":null},{"fullname":"Vacavant, L.","name":"L.","surname":"Vacavant","rank":103,"pid":[],"affiliation":null},{"fullname":"Boumediene, D.","name":"D.","surname":"Boumediene","rank":104,"pid":[],"affiliation":null},{"fullname":"Busato, Emmanuel","name":"Emmanuel","surname":"Busato","rank":105,"pid":[],"affiliation":null},{"fullname":"Calvet, D.","name":"D.","surname":"Calvet","rank":106,"pid":[],"affiliation":null},{"fullname":"Calvet, S.","name":"S.","surname":"Calvet","rank":107,"pid":[],"affiliation":null},{"fullname":"Donini, J.","name":"J.","surname":"Donini","rank":108,"pid":[],"affiliation":null},{"fullname":"Dubreuil, E.","name":"E.","surname":"Dubreuil","rank":109,"pid":[],"affiliation":null},{"fullname":"Gilles, G.","name":"G.","surname":"Gilles","rank":110,"pid":[],"affiliation":null},{"fullname":"Gris, Ph.","name":"Ph","surname":"Gris","rank":111,"pid":[],"affiliation":null},{"fullname":"Liao, H.","name":"H.","surname":"Liao","rank":112,"pid":[],"affiliation":null},{"fullname":"Madar, R.","name":"R.","surname":"Madar","rank":113,"pid":[],"affiliation":null},{"fullname":"Pallin, D.","name":"D.","surname":"Pallin","rank":114,"pid":[],"affiliation":null},{"fullname":"Saez, S.M. Romano","name":"S. M. Romano","surname":"Saez","rank":115,"pid":[],"affiliation":null},{"fullname":"Santoni, C.","name":"C.","surname":"Santoni","rank":116,"pid":[],"affiliation":null},{"fullname":"Simon, D.","name":"D.","surname":"Simon","rank":117,"pid":[],"affiliation":null},{"fullname":"Theveneaux-Pelzer, Timothée","name":"Timothée","surname":"Theveneaux-Pelzer","rank":118,"pid":[],"affiliation":null},{"fullname":"Vazeille, F.","name":"F.","surname":"Vazeille","rank":119,"pid":[],"affiliation":null},{"fullname":"Yatsenko, E.","name":"E.","surname":"Yatsenko","rank":120,"pid":[],"affiliation":null}] \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala index e190b2b33..9b85ba4f2 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala @@ -27,6 +27,8 @@ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String] def enrichResult(spark: SparkSession, graphPath: String, orcidPath: String, outputPath: String): Unit = { val orcidPublication = generateOrcidTable(spark, orcidPath) + + implicit val publicationEncoder = Encoders.bean(classOf[Publication]) val aschema = new StructType() @@ -68,6 +70,16 @@ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String] .select("dnet_id", "orcid_authors") .cache() + + orcidPublication + .join( + entities, + lower(col("schema")).equalTo(lower(col("pid_schema"))) && + lower(col("value")).equalTo(lower(col("pid_value"))), + "inner" + ) + .groupBy(col("dnet_id")).agg(collect_set(struct(col("pid_schema"), col("pid_value")))).write.mode("Overwrite").save("/user/sandro.labruzzo/enrich_pub") + val publication = spark.read.schema(publicationEncoder.schema).json(graphPath).as[Publication] publication @@ -95,13 +107,14 @@ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String] .where( "identifier.schema = 'doi' or identifier.schema ='pmid' or identifier.schema ='pmc' or identifier.schema ='arxiv' or identifier.schema ='handle'" ) - orcidAuthors + val orcidPublication =orcidAuthors .join(orcidWorks, orcidAuthors("orcid").equalTo(orcidWorks("orcid"))) .select( col("identifier.schema").alias("schema"), col("identifier.value").alias("value"), struct(orcidAuthors("orcid").alias("orcid"), col("givenName"), col("familyName")).alias("author") ) + orcidPublication } } From 59111713fac09a4508899f8102946bb5f04c7910 Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Tue, 28 Nov 2023 09:00:48 +0100 Subject: [PATCH 29/60] added comment --- .../eu/dnetlib/dhp/oa/merge/AuthorMerger.java | 95 ++++--------------- .../eu/dnetlib/oa/merge/AuthorMergerTest.java | 8 +- .../SparkEnrichGraphWithOrcidAuthors.scala | 2 +- 3 files changed, 21 insertions(+), 84 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java index 852ee163d..a1c3c2cc0 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java @@ -1,17 +1,11 @@ package eu.dnetlib.dhp.oa.merge; -import java.io.FileWriter; -import java.io.IOException; import java.text.Normalizer; import java.util.*; -import java.util.function.Function; import java.util.stream.Collectors; -import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; -import org.apache.commons.lang3.tuple.MutablePair; -import org.apache.commons.lang3.tuple.Pair; import org.jetbrains.annotations.NotNull; import com.wcohen.ss.JaroWinkler; @@ -161,42 +155,8 @@ public class AuthorMerger { .replaceAll("(\\n)+", " ") .trim(); -// return Arrays.stream(fullname.split("[\\s | , | ;]+")).map(String::toLowerCase).sorted().collect(Collectors.joining()); } -// -// public static List enrichOrcid2(List baseAuthor, List orcidAuthor) { -// if (baseAuthor == null || baseAuthor.isEmpty()) -// return orcidAuthor; -// -// if (orcidAuthor == null || orcidAuthor.isEmpty()) -// return baseAuthor; -// -// if (baseAuthor.size() == 1 && orcidAuthor.size() > 10) -// return baseAuthor; -// -// -// Map> pubClusters = baseAuthor.stream().collect(Collectors.toMap(AuthorMerger::generateAuthorkey, Arrays::asList, (a, b) -> { -// a.addAll(b); -// return a; -// })); -// -// Map> orcidClusters = baseAuthor.stream().collect(Collectors.toMap(AuthorMerger::generateAuthorkey, Arrays::asList, (a, b) -> { -// a.addAll(b); -// return a; -// })); -// -// System.out.println(pubClusters.keySet().size()); -// System.out.println(orcidClusters.keySet().size()); -// -// -// -// -// return null; -// -// -// } - static int hammingDist(String str1, String str2) { if (str1.length() != str2.length()) return Math.max(str1.length(), str2.length()); @@ -220,7 +180,14 @@ public class AuthorMerger { return null; } - public static boolean checkSimilarity2(final Author left, final Author right) { + /** + * This method tries to figure out when two author are the same in the contest + * of ORCID enrichment + * @param left Author in the OAF entity + * @param right Author ORCID + * @return based on a heuristic on the names of the authors if they are the same. + */ + public static boolean checkORCIDSimilarity(final Author left, final Author right) { final Person pl = parse(left); final Person pr = parse(right); @@ -267,8 +234,16 @@ public class AuthorMerger { else return false; } + // - public static List enrichOrcid2(List baseAuthor, List orcidAuthor) { + + /** + * Method to enrich ORCID information in one list of authors based on another list + * @param baseAuthor the Author List in the OAF Entity + * @param orcidAuthor The list of ORCID Author intersected + * @return The Author List of the OAF Entity enriched with the orcid Author + */ + public static List enrichOrcid(List baseAuthor, List orcidAuthor) { if (baseAuthor == null || baseAuthor.isEmpty()) return orcidAuthor; @@ -283,7 +258,7 @@ public class AuthorMerger { oAuthor.addAll(orcidAuthor); baseAuthor.forEach(ba -> { - Optional aMatch = oAuthor.stream().filter(oa -> checkSimilarity2(ba, oa)).findFirst(); + Optional aMatch = oAuthor.stream().filter(oa -> checkORCIDSimilarity(ba, oa)).findFirst(); if (aMatch.isPresent()) { final Author sameAuthor = aMatch.get(); addPid(ba, sameAuthor.getPid()); @@ -293,40 +268,6 @@ public class AuthorMerger { return baseAuthor; } - public static List enrichOrcid(List baseAuthor, List orcidAuthor) { - - if (baseAuthor == null || baseAuthor.isEmpty()) - return orcidAuthor; - - if (orcidAuthor == null || orcidAuthor.isEmpty()) - return baseAuthor; - - if (baseAuthor.size() == 1 && orcidAuthor.size() > 10) - return baseAuthor; - - final Double similarityMatrix[][] = new Double[baseAuthor.size()][orcidAuthor.size()]; - - final List maxColums = new ArrayList<>(); - - for (int i = 0; i < orcidAuthor.size(); i++) - maxColums.add(new SimilarityCellInfo()); - - for (int i = 0; i < baseAuthor.size(); i++) { - for (int j = 0; j < orcidAuthor.size(); j++) { - similarityMatrix[i][j] = sim(baseAuthor.get(i), orcidAuthor.get(j)); - if (maxColums.get(j).maxColumnSimilarity < similarityMatrix[i][j]) - maxColums.get(j).setValues(i, j, similarityMatrix[i][j]); - } - } - maxColums - .stream() - .sorted() - .filter(si -> si.maxColumnSimilarity > 0.85) - .forEach(si -> addPid(baseAuthor.get(si.authorPosition), orcidAuthor.get(si.orcidPosition).getPid())); - return baseAuthor; - - } - private static void addPid(final Author a, final List pids) { if (a.getPid() == null) { diff --git a/dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java b/dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java index 9eccab5f1..a11d49b1e 100644 --- a/dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java +++ b/dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java @@ -4,13 +4,9 @@ package eu.dnetlib.oa.merge; import static org.junit.jupiter.api.Assertions.*; import java.io.BufferedReader; -import java.io.FileReader; -import java.io.IOException; import java.io.InputStreamReader; -import java.util.Arrays; import java.util.List; import java.util.Objects; -import java.util.stream.Collectors; import org.junit.jupiter.api.Test; import org.junit.platform.commons.util.StringUtils; @@ -67,7 +63,7 @@ public class AuthorMergerTest { long start = System.currentTimeMillis(); // final List enrichedList = AuthorMerger.enrichOrcid(publicationAuthors, orcidAuthors); - final List enrichedList = AuthorMerger.enrichOrcid2(publicationAuthors, orcidAuthors); + final List enrichedList = AuthorMerger.enrichOrcid(publicationAuthors, orcidAuthors); long enrichedAuthorWithPid = enrichedList .stream() @@ -105,7 +101,7 @@ public class AuthorMergerTest { right.setSurname("Anand"); right.setFullname("Rachna, Anand"); // System.out.println(AuthorMerger.normalize(right.getFullname())); - boolean same = AuthorMerger.checkSimilarity2(left, right); + boolean same = AuthorMerger.checkORCIDSimilarity(left, right); assertTrue(same); diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala index 9b85ba4f2..3c9e04a21 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala @@ -89,7 +89,7 @@ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String] p } case (p: Publication, r: Row) => - p.setAuthor(AuthorMerger.enrichOrcid2(p.getAuthor, AuthorEnricher.toOAFAuthor(r))) + p.setAuthor(AuthorMerger.enrichOrcid(p.getAuthor, AuthorEnricher.toOAFAuthor(r))) p } .write From aa239ec673a0891e0adc45617a2239724d9e9daa Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Wed, 29 Nov 2023 11:17:41 +0100 Subject: [PATCH 30/60] Changed implementation of check similarity to verify exact match of name instead of the first char --- .../java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java | 3 +-- .../java/eu/dnetlib/oa/merge/AuthorMergerTest.java | 10 +++++++--- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java index a1c3c2cc0..62e6764b7 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java @@ -222,7 +222,7 @@ public class AuthorMerger { nl -> pr .getName() .stream() - .anyMatch(nr -> nr.substring(0, 1).equalsIgnoreCase(nl.substring(0, 1))))) + .anyMatch(nr -> nr.equalsIgnoreCase(nl)))) return true; } @@ -236,7 +236,6 @@ public class AuthorMerger { } // - /** * Method to enrich ORCID information in one list of authors based on another list * @param baseAuthor the Author List in the OAF Entity diff --git a/dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java b/dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java index a11d49b1e..c0a8d6927 100644 --- a/dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java +++ b/dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java @@ -19,15 +19,19 @@ import eu.dnetlib.dhp.schema.oaf.Author; public class AuthorMergerTest { - @Test public void testEnrcichAuthor() throws Exception { final ObjectMapper mapper = new ObjectMapper(); BufferedReader pr = new BufferedReader(new InputStreamReader( - Objects.requireNonNull(AuthorMergerTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/merge/authors_publication_sample.json")))); + Objects + .requireNonNull( + AuthorMergerTest.class + .getResourceAsStream("/eu/dnetlib/dhp/oa/merge/authors_publication_sample.json")))); BufferedReader or = new BufferedReader(new InputStreamReader( - Objects.requireNonNull(AuthorMergerTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/merge/authors_orcid_sample.json")))); + Objects + .requireNonNull( + AuthorMergerTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/merge/authors_orcid_sample.json")))); TypeReference> aclass = new TypeReference>() { }; From 279100fa5220df87e17b535e14619a49fa726c70 Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Wed, 29 Nov 2023 11:17:58 +0100 Subject: [PATCH 31/60] added test --- .../dhp/enrich/orcid/EnrichOrcidTest.scala | 72 ++++++++++++++++++- 1 file changed, 69 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/EnrichOrcidTest.scala b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/EnrichOrcidTest.scala index 0ddb7c0aa..f58b06318 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/EnrichOrcidTest.scala +++ b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/EnrichOrcidTest.scala @@ -1,12 +1,78 @@ package eu.dnetlib.dhp.enrich.orcid - -import org.apache.spark.SparkConf -import org.apache.spark.sql.SparkSession +import eu.dnetlib.dhp.schema.oaf.Publication +import org.apache.spark.sql.{Column, Encoder, Encoders, Row, SparkSession} import org.junit.jupiter.api.Test import org.slf4j.{Logger, LoggerFactory} +import org.apache.spark.sql.functions._ + + +case class Pid(pidScheme: String, pidValue: String) {} + +case class AuthorPid(fullName: String, pids: List[Pid]) {} + +case class PubSummary(id: String, authorWithPids: List[AuthorPid]) class EnrichOrcidTest { val log: Logger = LoggerFactory.getLogger(getClass) + + def orcid_intersection_wrong(p: PubSummary): PubSummary = { + + if (p.authorWithPids.isEmpty) + null + else { + val incorrectAuthor = p.authorWithPids.filter(a => a.pids.filter(p => p.pidScheme != null && p.pidScheme.toLowerCase.contains("orcid")).map(p => p.pidValue.toLowerCase).distinct.size > 1) + if (incorrectAuthor.nonEmpty) { + PubSummary(p.id, incorrectAuthor) + } + else { + null + } + } + } + + + + def test() = { + val spark = SparkSession.builder().master("local[*]").getOrCreate() + spark.sparkContext.setLogLevel("ERROR") + + val schema = Encoders.bean(classOf[Publication]).schema + + + val simplifyAuthor = udf((r: Seq[Row]) => { + r + .map(k => + AuthorPid(k.getAs[String]("fullname"), + k.getAs[Seq[Row]]("pid") + .map( + p => Pid(p.getAs[Row]("qualifier").getAs[String]("classid"), p.getAs[String]("value")) + ).toList) + ).filter(l => l.pids.nonEmpty) + .toList + } + ) + + val wrong_orcid_intersection = udf((a: Seq[Row]) => { + a.map(author => { + val pids_with_orcid: Seq[Row] = author.getAs[Seq[Row]]("pids").filter(p => p.getAs[String]("pidScheme")!= null && p.getAs[String]("pidScheme").toLowerCase.contains("orcid")) + if (pids_with_orcid.exists(p => p.getAs[String]("pidScheme").equals("ORCID"))) { + if (pids_with_orcid.map(p => p.getAs[String]("pidValue").toLowerCase).distinct.size > 1) { + AuthorPid(author.getAs[String]("fullName"),pids_with_orcid.map(p => Pid(p.getAs[String]("pidScheme"),p.getAs[String]("pidValue"))).toList ) + + } + else + null + } else + + null + }).filter(author => author != null) + }) + val enriched = spark.read.schema(schema).json("/Users/sandro/orcid_test/publication_enriched").select(col("id"), simplifyAuthor(col("author")).alias("authors")) + .select(col("id"), wrong_orcid_intersection(col("authors")).alias("wi")).where("wi is not null") + enriched.show(20, 1000, true) + } + + } From af1c2634b3fc61c208dfd8e57fff1b11459dba5f Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Wed, 29 Nov 2023 12:45:30 +0100 Subject: [PATCH 32/60] added instanceTypeMapping original field in the mapping of - DOIBoost - Datacite - PubMed - Scholexplorer Datasource --- .../DataciteToOAFTransformation.scala | 15 ++++++++----- .../eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala | 21 +++++++++++++++++-- .../dhp/sx/bio/pubmed/PubMedToOaf.scala | 18 ++++++++++++---- .../doiboost/crossref/Crossref2Oaf.scala | 8 +++++-- 4 files changed, 49 insertions(+), 13 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala index 45f5f9729..17f1b90b0 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala @@ -166,7 +166,7 @@ object DataciteToOAFTransformation { resourceTypeGeneral: String, schemaOrg: String, vocabularies: VocabularyGroup - ): (Qualifier, Qualifier) = { + ): (Qualifier, Qualifier, String) = { if (resourceType != null && resourceType.nonEmpty) { val typeQualifier = vocabularies.getSynonymAsQualifier(ModelConstants.DNET_PUBLICATION_RESOURCE, resourceType) @@ -176,7 +176,7 @@ object DataciteToOAFTransformation { vocabularies.getSynonymAsQualifier( ModelConstants.DNET_RESULT_TYPOLOGIES, typeQualifier.getClassid - ) + ), resourceType ) } if (schemaOrg != null && schemaOrg.nonEmpty) { @@ -188,7 +188,7 @@ object DataciteToOAFTransformation { vocabularies.getSynonymAsQualifier( ModelConstants.DNET_RESULT_TYPOLOGIES, typeQualifier.getClassid - ) + ), schemaOrg ) } @@ -203,7 +203,7 @@ object DataciteToOAFTransformation { vocabularies.getSynonymAsQualifier( ModelConstants.DNET_RESULT_TYPOLOGIES, typeQualifier.getClassid - ) + ), resourceTypeGeneral ) } @@ -216,12 +216,17 @@ object DataciteToOAFTransformation { schemaOrg: String, vocabularies: VocabularyGroup ): Result = { - val typeQualifiers: (Qualifier, Qualifier) = + val typeQualifiers: (Qualifier, Qualifier, String) = getTypeQualifier(resourceType, resourceTypeGeneral, schemaOrg, vocabularies) if (typeQualifiers == null) return null val i = new Instance i.setInstancetype(typeQualifiers._1) + // ADD ORIGINAL TYPE + val itm = new InstanceTypeMapping + itm.setOriginalType(typeQualifiers._3) + i.setInstanceTypeMapping(List(itm).asJava) + typeQualifiers._2.getClassname match { case "dataset" => val r = new OafDataset diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala index c079f7537..3624c5369 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala @@ -176,7 +176,7 @@ object BioDBToOAF { i.setUrl(List(s"${resolvedURL(input.pidType)}${input.pid}").asJava) } - if (input.pidType.equalsIgnoreCase("clinicaltrials.gov")) + if (input.pidType.equalsIgnoreCase("clinicaltrials.gov")) { i.setInstancetype( OafMapperUtils.qualifier( "0037", @@ -185,7 +185,10 @@ object BioDBToOAF { ModelConstants.DNET_PUBLICATION_RESOURCE ) ) - else + val itm = new InstanceTypeMapping + itm.setOriginalType(input.pidType) + i.setInstanceTypeMapping(List(itm).asJava) + } else { i.setInstancetype( OafMapperUtils.qualifier( "0046", @@ -194,6 +197,10 @@ object BioDBToOAF { ModelConstants.DNET_PUBLICATION_RESOURCE ) ) + val itm = new InstanceTypeMapping + itm.setOriginalType("Bioentity") + i.setInstanceTypeMapping(List(itm).asJava) + } if (input.datasource == null || input.datasource.isEmpty) return null @@ -265,6 +272,9 @@ object BioDBToOAF { ModelConstants.DNET_PUBLICATION_RESOURCE ) ) + val itm = new InstanceTypeMapping + itm.setOriginalType("Bioentity") + i.setInstanceTypeMapping(List(itm).asJava) i.setCollectedfrom(collectedFromMap("uniprot")) d.setInstance(List(i).asJava) @@ -471,6 +481,9 @@ object BioDBToOAF { ModelConstants.DNET_PUBLICATION_RESOURCE ) ) + val itm = new InstanceTypeMapping + itm.setOriginalType("Bioentity") + i.setInstanceTypeMapping(List(itm).asJava) i.setCollectedfrom(collectedFromMap("pdb")) d.setInstance(List(i).asJava) @@ -571,6 +584,10 @@ object BioDBToOAF { ModelConstants.DNET_PUBLICATION_RESOURCE ) ) + val itm = new InstanceTypeMapping + itm.setOriginalType("Bioentity") + i.setInstanceTypeMapping(List(itm).asJava) + i.setCollectedfrom(collectedFromMap("ebi")) d.setInstance(List(i).asJava) diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala index 410686f97..40ac48a7a 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala @@ -188,12 +188,22 @@ object PubMedToOaf { val cojbCategory = getVocabularyTerm(ModelConstants.DNET_PUBLICATION_RESOURCE, vocabularies, ja.get.getValue) pubmedInstance.setInstancetype(cojbCategory) + // ADD ORIGINAL TYPE to the publication + val itm = new InstanceTypeMapping + itm.setOriginalType(ja.get.getValue) + pubmedInstance.setInstanceTypeMapping(List(itm).asJava) } else { val i_type = article.getPublicationTypes.asScala - .map(s => getVocabularyTerm(ModelConstants.DNET_PUBLICATION_RESOURCE, vocabularies, s.getValue)) - .find(q => q != null) - if (i_type.isDefined) - pubmedInstance.setInstancetype(i_type.get) + .map(s => (s.getValue,getVocabularyTerm(ModelConstants.DNET_PUBLICATION_RESOURCE, vocabularies, s.getValue))) + .find(q => q._2 != null) + + if (i_type.isDefined) { + pubmedInstance.setInstancetype(i_type.get._2) + // ADD ORIGINAL TYPE to the publication + val itm = new InstanceTypeMapping + itm.setOriginalType(i_type.get._1) + pubmedInstance.setInstanceTypeMapping(List(itm).asJava) + } else return null } diff --git a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala index 565d34e62..f45c303f0 100644 --- a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala @@ -107,7 +107,7 @@ case object Crossref2Oaf { .map(f => f.id) } - def mappingResult(result: Result, json: JValue, cobjCategory: String): Result = { + def mappingResult(result: Result, json: JValue, cobjCategory: String, originalType:String): Result = { implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats //MAPPING Crossref DOI into PID @@ -283,6 +283,10 @@ case object Crossref2Oaf { ModelConstants.DNET_PUBLICATION_RESOURCE ) ) + //ADD ORIGINAL TYPE to the mapping + val itm = new InstanceTypeMapping + itm.setOriginalType(originalType) + instance.setInstanceTypeMapping(List(itm).asJava) result.setResourcetype( OafMapperUtils.qualifier( cobjCategory.substring(0, 4), @@ -367,7 +371,7 @@ case object Crossref2Oaf { objectType, mappingCrossrefSubType.getOrElse(objectSubType, "0038 Other literature type") ) - mappingResult(result, json, cOBJCategory) + mappingResult(result, json, cOBJCategory, originalType) if (result == null || result.getId == null) return List() From 86b5775e0859233ececfbc7425afee5a5b2c2c6f Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Wed, 29 Nov 2023 13:15:43 +0100 Subject: [PATCH 33/60] added vocabulary in instanceTypeMapping for - DOIBoost - Datacite - PubMed - Scholexplorer Datasource --- .../dnetlib/dhp/datacite/DataciteToOAFTransformation.scala | 2 ++ .../src/main/scala/eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala | 5 +++++ .../scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala | 2 ++ .../scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala | 1 + 4 files changed, 10 insertions(+) diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala index 17f1b90b0..ee3660918 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala @@ -225,8 +225,10 @@ object DataciteToOAFTransformation { // ADD ORIGINAL TYPE val itm = new InstanceTypeMapping itm.setOriginalType(typeQualifiers._3) + itm.setVocabularyName(ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1) i.setInstanceTypeMapping(List(itm).asJava) + typeQualifiers._2.getClassname match { case "dataset" => val r = new OafDataset diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala index 3624c5369..89bc8d948 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala @@ -187,6 +187,7 @@ object BioDBToOAF { ) val itm = new InstanceTypeMapping itm.setOriginalType(input.pidType) + itm.setVocabularyName(ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1) i.setInstanceTypeMapping(List(itm).asJava) } else { i.setInstancetype( @@ -199,6 +200,7 @@ object BioDBToOAF { ) val itm = new InstanceTypeMapping itm.setOriginalType("Bioentity") + itm.setVocabularyName(ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1) i.setInstanceTypeMapping(List(itm).asJava) } @@ -274,6 +276,7 @@ object BioDBToOAF { ) val itm = new InstanceTypeMapping itm.setOriginalType("Bioentity") + itm.setVocabularyName(ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1) i.setInstanceTypeMapping(List(itm).asJava) i.setCollectedfrom(collectedFromMap("uniprot")) @@ -483,6 +486,7 @@ object BioDBToOAF { ) val itm = new InstanceTypeMapping itm.setOriginalType("Bioentity") + itm.setVocabularyName(ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1) i.setInstanceTypeMapping(List(itm).asJava) i.setCollectedfrom(collectedFromMap("pdb")) @@ -586,6 +590,7 @@ object BioDBToOAF { ) val itm = new InstanceTypeMapping itm.setOriginalType("Bioentity") + itm.setVocabularyName(ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1) i.setInstanceTypeMapping(List(itm).asJava) diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala index 40ac48a7a..f3f8b4f02 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala @@ -191,6 +191,7 @@ object PubMedToOaf { // ADD ORIGINAL TYPE to the publication val itm = new InstanceTypeMapping itm.setOriginalType(ja.get.getValue) + itm.setVocabularyName(ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1) pubmedInstance.setInstanceTypeMapping(List(itm).asJava) } else { val i_type = article.getPublicationTypes.asScala @@ -202,6 +203,7 @@ object PubMedToOaf { // ADD ORIGINAL TYPE to the publication val itm = new InstanceTypeMapping itm.setOriginalType(i_type.get._1) + itm.setVocabularyName(ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1) pubmedInstance.setInstanceTypeMapping(List(itm).asJava) } else diff --git a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala index f45c303f0..65e395357 100644 --- a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala @@ -286,6 +286,7 @@ case object Crossref2Oaf { //ADD ORIGINAL TYPE to the mapping val itm = new InstanceTypeMapping itm.setOriginalType(originalType) + itm.setVocabularyName(ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1) instance.setInstanceTypeMapping(List(itm).asJava) result.setResourcetype( OafMapperUtils.qualifier( From 7b5e04f37ec57971fbd588a9a6cca0314d06b045 Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Thu, 30 Nov 2023 14:36:50 +0100 Subject: [PATCH 34/60] removed Orcid intersection on DOIBoost --- .../preprocess/oozie_app/workflow.xml | 26 ---- .../doiboost/process/oozie_app/workflow.xml | 34 +----- .../doiboost/SparkGenerateDoiBoost.scala | 17 +-- .../dhp/enrich/orcid/oozie_app/workflow.xml | 2 +- .../dhp/enrich/orcid/AuthorEnricher.scala | 7 +- .../SparkEnrichGraphWithOrcidAuthors.scala | 106 ++++++++-------- .../dhp/enrich/orcid/EnrichOrcidTest.scala | 113 ++++++++++-------- 7 files changed, 128 insertions(+), 177 deletions(-) diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/preprocess/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/preprocess/oozie_app/workflow.xml index 40a17b486..ed6853229 100644 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/preprocess/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/preprocess/oozie_app/workflow.xml @@ -133,32 +133,6 @@ --targetPath${inputPathMAG}/dataset --masteryarn-cluster - - - - - - - - yarn-cluster - cluster - Convert ORCID to Dataset - eu.dnetlib.doiboost.orcid.SparkPreprocessORCID - dhp-doiboost-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.sql.shuffle.partitions=3840 - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - --sourcePath${inputPathOrcid} - --workingPath${workingPathOrcid} - --masteryarn-cluster - diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/process/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/process/oozie_app/workflow.xml index 29a12f4df..8f28d706d 100644 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/process/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/process/oozie_app/workflow.xml @@ -59,10 +59,10 @@
- - workingPathOrcid - the ORCID working path - + + + + @@ -170,32 +170,6 @@ --targetPath${workingPath}/uwPublication --masteryarn-cluster - - - - - - - - yarn-cluster - cluster - Convert ORCID to Dataset - eu.dnetlib.doiboost.orcid.SparkConvertORCIDToOAF - dhp-doiboost-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.sql.shuffle.partitions=3840 - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - - --workingPath${workingPathOrcid} - --targetPath${workingPath}/orcidPublication - --masteryarn-cluster - diff --git a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/SparkGenerateDoiBoost.scala b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/SparkGenerateDoiBoost.scala index 2cbd53097..07d6a0287 100644 --- a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/SparkGenerateDoiBoost.scala +++ b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/SparkGenerateDoiBoost.scala @@ -66,7 +66,7 @@ object SparkGenerateDoiBoost { Encoders.tuple(Encoders.STRING, mapEncoderPub) implicit val mapEncoderRel: Encoder[Relation] = Encoders.kryo[Relation] - logger.info("Phase 2) Join Crossref with UnpayWall") + logger.info("Phase 1) Join Crossref with UnpayWall") val crossrefPublication: Dataset[(String, Publication)] = spark.read.load(s"$workingDirPath/crossrefPublication").as[Publication].map(p => (p.getId, p)) @@ -91,20 +91,11 @@ object SparkGenerateDoiBoost { .write .mode(SaveMode.Overwrite) .save(s"$workingDirPath/firstJoin") - logger.info("Phase 3) Join Result with ORCID") - val fj: Dataset[(String, Publication)] = - spark.read.load(s"$workingDirPath/firstJoin").as[Publication].map(p => (p.getId, p)) - val orcidPublication: Dataset[(String, Publication)] = - spark.read.load(s"$workingDirPath/orcidPublication").as[Publication].map(p => (p.getId, p)) - fj.joinWith(orcidPublication, fj("_1").equalTo(orcidPublication("_1")), "left") - .map(applyMerge) - .write - .mode(SaveMode.Overwrite) - .save(s"$workingDirPath/secondJoin") - logger.info("Phase 4) Join Result with MAG") + + logger.info("Phase 2) Join Result with MAG") val sj: Dataset[(String, Publication)] = - spark.read.load(s"$workingDirPath/secondJoin").as[Publication].map(p => (p.getId, p)) + spark.read.load(s"$workingDirPath/firstJoin").as[Publication].map(p => (p.getId, p)) val magPublication: Dataset[(String, Publication)] = spark.read.load(s"$workingDirPath/magPublication").as[Publication].map(p => (p.getId, p)) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml index 1284cceda..87c4dcb4f 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml @@ -40,7 +40,7 @@ --orcidPath${orcidPath} --targetPath${targetPath} - --graphPath${graphPath}/publication + --graphPath${graphPath} --masteryarn diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/AuthorEnricher.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/AuthorEnricher.scala index a67de4b95..15513c8af 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/AuthorEnricher.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/AuthorEnricher.scala @@ -1,5 +1,6 @@ package eu.dnetlib.dhp.enrich.orcid +import eu.dnetlib.dhp.schema.common.ModelConstants import eu.dnetlib.dhp.schema.oaf.{Author, Publication} import eu.dnetlib.dhp.schema.sx.OafUtils import org.apache.spark.sql.Row @@ -13,9 +14,11 @@ object AuthorEnricher extends Serializable { a.setName(givenName) a.setSurname(familyName) a.setFullname(s"$givenName $familyName") - a.setPid(List(OafUtils.createSP(orcid, "ORCID", "ORCID")).asJava) + val pid = OafUtils.createSP(orcid, ModelConstants.ORCID, ModelConstants.ORCID) + pid.setDataInfo(OafUtils.generateDataInfo()) + pid.getDataInfo.setProvenanceaction(OafUtils.createQualifier("ORCID_ENRICHMENT", "ORCID_ENRICHMENT")) + a.setPid(List(pid).asJava) a - } def toOAFAuthor(r: Row): java.util.List[Author] = { diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala index 3c9e04a21..0d994d202 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala @@ -1,13 +1,11 @@ package eu.dnetlib.dhp.enrich.orcid -import com.fasterxml.jackson.databind.ObjectMapper import eu.dnetlib.dhp.application.AbstractScalaApplication import eu.dnetlib.dhp.oa.merge.AuthorMerger -import eu.dnetlib.dhp.schema.oaf.{Author, DataInfo, Instance, Publication, StructuredProperty} -import org.apache.spark.sql.{Dataset, Encoder, Encoders, Row, SaveMode, SparkSession} -import org.apache.spark.sql.functions.{col, collect_set, concat, explode, expr, first, flatten, lower, size, struct} +import eu.dnetlib.dhp.schema.oaf.{OtherResearchProduct, Publication, Result, Software} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql._ import org.slf4j.{Logger, LoggerFactory} -import org.apache.spark.sql.types._ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String], log: Logger) extends AbstractScalaApplication(propertyPath, args, log: Logger) { @@ -22,33 +20,49 @@ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String] log.info(s"orcidPath is '$orcidPath'") val targetPath = parser.get("targetPath") log.info(s"targetPath is '$targetPath'") - enrichResult(spark, graphPath, orcidPath, targetPath) + val orcidPublication: Dataset[Row] = generateOrcidTable(spark, orcidPath) + enrichResult( + spark, + s"$graphPath/publication", + orcidPublication, + s"$targetPath/publication", + Encoders.bean(classOf[Publication]) + ) + enrichResult( + spark, + s"$graphPath/dataset", + orcidPublication, + s"$targetPath/dataset", + Encoders.bean(classOf[eu.dnetlib.dhp.schema.oaf.Dataset]) + ) + enrichResult( + spark, + s"$graphPath/software", + orcidPublication, + s"$targetPath/software", + Encoders.bean(classOf[Software]) + ) + enrichResult( + spark, + s"$graphPath/otherresearchproduct", + orcidPublication, + s"$targetPath/otherresearchproduct", + Encoders.bean(classOf[OtherResearchProduct]) + ) } - def enrichResult(spark: SparkSession, graphPath: String, orcidPath: String, outputPath: String): Unit = { - val orcidPublication = generateOrcidTable(spark, orcidPath) + private def enrichResult[T <: Result]( + spark: SparkSession, + graphPath: String, + orcidPublication: Dataset[Row], + outputPath: String, + enc: Encoder[T] + ): Unit = { - - implicit val publicationEncoder = Encoders.bean(classOf[Publication]) - - val aschema = new StructType() - .add("id", StringType) - .add("dataInfo", Encoders.bean(classOf[DataInfo]).schema) - .add( - "author",Encoders.bean(classOf[Author]).schema - - ) - - val schema = new StructType() - .add("id", StringType) - .add("dataInfo", Encoders.bean(classOf[DataInfo]).schema) - .add( - "instance", - ArrayType(new StructType().add("pid", ArrayType(Encoders.bean(classOf[StructuredProperty]).schema))) - ) val entities = spark.read - .schema(schema) + .schema(enc.schema) .json(graphPath) + .select(col("id"), col("datainfo"), col("instance")) .where("datainfo.deletedbyinference = false") .drop("datainfo") .withColumn("instances", explode(col("instance"))) @@ -58,7 +72,8 @@ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String] col("pids.value").alias("pid_value"), col("id").alias("dnet_id") ) - val orcidDnet = orcidPublication + + val orcidDnet = orcidPublication .join( entities, lower(col("schema")).equalTo(lower(col("pid_schema"))) && @@ -69,36 +84,25 @@ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String] .agg(collect_set(orcidPublication("author")).alias("orcid_authors")) .select("dnet_id", "orcid_authors") .cache() + orcidDnet.count() + val result = spark.read.schema(enc.schema).json(graphPath).as[T](enc) - - orcidPublication - .join( - entities, - lower(col("schema")).equalTo(lower(col("pid_schema"))) && - lower(col("value")).equalTo(lower(col("pid_value"))), - "inner" - ) - .groupBy(col("dnet_id")).agg(collect_set(struct(col("pid_schema"), col("pid_value")))).write.mode("Overwrite").save("/user/sandro.labruzzo/enrich_pub") - - val publication = spark.read.schema(publicationEncoder.schema).json(graphPath).as[Publication] - - publication - .joinWith(orcidDnet, publication("id").equalTo(orcidDnet("dnet_id")), "left") + result + .joinWith(orcidDnet, result("id").equalTo(orcidDnet("dnet_id")), "left") .map { - case (p: Publication, null) => { - p - } - case (p: Publication, r: Row) => + case (r: T, null) => + r + case (p: T, r: Row) => p.setAuthor(AuthorMerger.enrichOrcid(p.getAuthor, AuthorEnricher.toOAFAuthor(r))) p - } + }(enc) .write .mode(SaveMode.Overwrite) .option("compression", "gzip") .json(outputPath) } - def generateOrcidTable(spark: SparkSession, inputPath: String): Dataset[Row] = { + private def generateOrcidTable(spark: SparkSession, inputPath: String): Dataset[Row] = { val orcidAuthors = spark.read.load(s"$inputPath/Authors").select("orcid", "familyName", "givenName", "creditName", "otherNames") val orcidWorks = spark.read @@ -107,14 +111,14 @@ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String] .where( "identifier.schema = 'doi' or identifier.schema ='pmid' or identifier.schema ='pmc' or identifier.schema ='arxiv' or identifier.schema ='handle'" ) - val orcidPublication =orcidAuthors + val orcidPublication = orcidAuthors .join(orcidWorks, orcidAuthors("orcid").equalTo(orcidWorks("orcid"))) .select( col("identifier.schema").alias("schema"), col("identifier.value").alias("value"), struct(orcidAuthors("orcid").alias("orcid"), col("givenName"), col("familyName")).alias("author") ) - orcidPublication + orcidPublication.cache() } } @@ -123,10 +127,8 @@ object SparkEnrichGraphWithOrcidAuthors { val log: Logger = LoggerFactory.getLogger(SparkEnrichGraphWithOrcidAuthors.getClass) def main(args: Array[String]): Unit = { - new SparkEnrichGraphWithOrcidAuthors("/eu/dnetlib/dhp/enrich/orcid/enrich_graph_orcid_parameters.json", args, log) .initialize() .run() - } } diff --git a/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/EnrichOrcidTest.scala b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/EnrichOrcidTest.scala index f58b06318..84483b1a2 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/EnrichOrcidTest.scala +++ b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/EnrichOrcidTest.scala @@ -1,78 +1,85 @@ package eu.dnetlib.dhp.enrich.orcid -import eu.dnetlib.dhp.schema.oaf.Publication + +import eu.dnetlib.dhp.schema.oaf.{Author, Publication} import org.apache.spark.sql.{Column, Encoder, Encoders, Row, SparkSession} import org.junit.jupiter.api.Test import org.slf4j.{Logger, LoggerFactory} import org.apache.spark.sql.functions._ -case class Pid(pidScheme: String, pidValue: String) {} - -case class AuthorPid(fullName: String, pids: List[Pid]) {} - -case class PubSummary(id: String, authorWithPids: List[AuthorPid]) - class EnrichOrcidTest { val log: Logger = LoggerFactory.getLogger(getClass) - def orcid_intersection_wrong(p: PubSummary): PubSummary = { - - if (p.authorWithPids.isEmpty) - null - else { - val incorrectAuthor = p.authorWithPids.filter(a => a.pids.filter(p => p.pidScheme != null && p.pidScheme.toLowerCase.contains("orcid")).map(p => p.pidValue.toLowerCase).distinct.size > 1) - if (incorrectAuthor.nonEmpty) { - PubSummary(p.id, incorrectAuthor) - } - else { - null - } - } - } - - def test() = { val spark = SparkSession.builder().master("local[*]").getOrCreate() - spark.sparkContext.setLogLevel("ERROR") +// spark.sparkContext.setLogLevel("ERROR") + +// new SparkEnrichGraphWithOrcidAuthors(null, null, null) +// .enrichResult( +// spark, +// "/Users/sandro/orcid_test/publication", +// "", +// "/tmp/graph/", +// Encoders.bean(classOf[Publication]) +// ) val schema = Encoders.bean(classOf[Publication]).schema +// +// val simplifyAuthor = udf((r: Seq[Row]) => { +// r +// .map(k => +// AuthorPid( +// k.getAs[String]("fullname"), +// k.getAs[Seq[Row]]("pid") +// .map(p => Pid(p.getAs[Row]("qualifier").getAs[String]("classid"), p.getAs[String]("value"))) +// .toList +// ) +// ) +// .filter(l => l.pids.nonEmpty) +// .toList +// }) +// +// val wrong_orcid_intersection = udf((a: Seq[Row]) => { +// a.map(author => { +// val pids_with_orcid: Seq[Row] = author +// .getAs[Seq[Row]]("pids") +// .filter(p => +// p.getAs[String]("pidScheme") != null && p.getAs[String]("pidScheme").toLowerCase.contains("orcid") +// ) +// if (pids_with_orcid.exists(p => p.getAs[String]("pidScheme").equals("ORCID"))) { +// if (pids_with_orcid.map(p => p.getAs[String]("pidValue").toLowerCase).distinct.size > 1) { +// AuthorPid( +// author.getAs[String]("fullName"), +// pids_with_orcid.map(p => Pid(p.getAs[String]("pidScheme"), p.getAs[String]("pidValue"))).toList +// ) +// +// } else +// null +// } else +// null +// }).filter(author => author != null) +// }) + + + Encoders + import spark.implicits._ + +// val enriched = spark.read +// .schema(schema) +// .json("/Users/sandro/orcid_test/publication_enriched") +// .select(col("id"), explode(col("author")).as("authors")) +// .withColumn("ap", col("authors.pid.qualifier.classid")) +// .withColumn("dp", col("authors.pid.datainfo.provenanceAction.classid")) +// +// .show() - val simplifyAuthor = udf((r: Seq[Row]) => { - r - .map(k => - AuthorPid(k.getAs[String]("fullname"), - k.getAs[Seq[Row]]("pid") - .map( - p => Pid(p.getAs[Row]("qualifier").getAs[String]("classid"), p.getAs[String]("value")) - ).toList) - ).filter(l => l.pids.nonEmpty) - .toList - } - ) - val wrong_orcid_intersection = udf((a: Seq[Row]) => { - a.map(author => { - val pids_with_orcid: Seq[Row] = author.getAs[Seq[Row]]("pids").filter(p => p.getAs[String]("pidScheme")!= null && p.getAs[String]("pidScheme").toLowerCase.contains("orcid")) - if (pids_with_orcid.exists(p => p.getAs[String]("pidScheme").equals("ORCID"))) { - if (pids_with_orcid.map(p => p.getAs[String]("pidValue").toLowerCase).distinct.size > 1) { - AuthorPid(author.getAs[String]("fullName"),pids_with_orcid.map(p => Pid(p.getAs[String]("pidScheme"),p.getAs[String]("pidValue"))).toList ) - } - else - null - } else - null - }).filter(author => author != null) - }) - val enriched = spark.read.schema(schema).json("/Users/sandro/orcid_test/publication_enriched").select(col("id"), simplifyAuthor(col("author")).alias("authors")) - .select(col("id"), wrong_orcid_intersection(col("authors")).alias("wi")).where("wi is not null") - enriched.show(20, 1000, true) } - } From f718caaac974ed8e931053c757aaef8d9945c38c Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Thu, 30 Nov 2023 14:51:00 +0100 Subject: [PATCH 35/60] Added copy of the untouched entities of the graph --- .../dhp/enrich/orcid/oozie_app/workflow.xml | 46 +++++++++++++++++-- 1 file changed, 41 insertions(+), 5 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml index 87c4dcb4f..a4330b966 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml @@ -1,4 +1,4 @@ - + orcidPath @@ -38,15 +38,51 @@ --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --orcidPath${orcidPath} - --targetPath${targetPath} - --graphPath${graphPath} - --masteryarn + --orcidPath + ${orcidPath} + --targetPath + ${targetPath} + --graphPath + ${graphPath} + --master + yarn + + + ${nameNode}/${graphPath}/datasource + ${nameNode}/${targetPath}/datasource + + + + + + + ${nameNode}/${graphPath}/organization + ${nameNode}/${targetPath}/organization + + + + + + + ${nameNode}/${graphPath}/project + ${nameNode}/${targetPath}/project + + + + + + + ${nameNode}/${graphPath}/relation + ${nameNode}/${targetPath}/relation + + + + \ No newline at end of file From cdfb7588dd4b23fc83e764ae490dd051c8e1576f Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Thu, 30 Nov 2023 15:31:42 +0100 Subject: [PATCH 36/60] code formatting --- .../eu/dnetlib/dhp/oa/merge/AuthorMerger.java | 551 +++++++++--------- .../PrepareAffiliationRelationsTest.java | 4 +- .../doiboost/SparkGenerateDoiBoost.scala | 1 - .../SparkEnrichGraphWithOrcidAuthors.scala | 2 +- .../dhp/enrich/orcid/EnrichOrcidTest.scala | 10 +- 5 files changed, 268 insertions(+), 300 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java index 62e6764b7..d7224af78 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java @@ -4,352 +4,329 @@ package eu.dnetlib.dhp.oa.merge; import java.text.Normalizer; import java.util.*; import java.util.stream.Collectors; - import org.apache.commons.lang3.StringUtils; -import org.jetbrains.annotations.NotNull; - import com.wcohen.ss.JaroWinkler; - import eu.dnetlib.dhp.schema.oaf.Author; import eu.dnetlib.dhp.schema.oaf.StructuredProperty; import eu.dnetlib.pace.model.Person; import scala.Tuple2; -class SimilarityCellInfo implements Comparable { - - public int authorPosition = 0; - public int orcidPosition = 0; - - public double maxColumnSimilarity = 0.0; - - public SimilarityCellInfo() { - } - - public void setValues(final int authPos, final int orcidPos, final double similarity) { - this.authorPosition = authPos; - this.orcidPosition = orcidPos; - this.maxColumnSimilarity = similarity; - } - - @Override - public int compareTo(@NotNull SimilarityCellInfo o) { - return Double.compare(maxColumnSimilarity, o.maxColumnSimilarity); - } -} public class AuthorMerger { - private static final Double THRESHOLD = 0.95; + private static final Double THRESHOLD = 0.95; - private AuthorMerger() { - } + private AuthorMerger() { + } - public static List merge(List> authors) { + public static List merge(List> authors) { - authors.sort((o1, o2) -> -Integer.compare(countAuthorsPids(o1), countAuthorsPids(o2))); + authors.sort((o1, o2) -> -Integer.compare(countAuthorsPids(o1), countAuthorsPids(o2))); - List author = new ArrayList<>(); + List author = new ArrayList<>(); - for (List a : authors) { - author = mergeAuthor(author, a); - } + for (List a : authors) { + author = mergeAuthor(author, a); + } - return author; + return author; - } + } - public static List mergeAuthor(final List a, final List b, Double threshold) { - int pa = countAuthorsPids(a); - int pb = countAuthorsPids(b); - List base; - List enrich; - int sa = authorsSize(a); - int sb = authorsSize(b); + public static List mergeAuthor(final List a, final List b, Double threshold) { + int pa = countAuthorsPids(a); + int pb = countAuthorsPids(b); + List base; + List enrich; + int sa = authorsSize(a); + int sb = authorsSize(b); - if (sa == sb) { - base = pa > pb ? a : b; - enrich = pa > pb ? b : a; - } else { - base = sa > sb ? a : b; - enrich = sa > sb ? b : a; - } - enrichPidFromList(base, enrich, threshold); - return base; - } + if (sa == sb) { + base = pa > pb ? a : b; + enrich = pa > pb ? b : a; + } else { + base = sa > sb ? a : b; + enrich = sa > sb ? b : a; + } + enrichPidFromList(base, enrich, threshold); + return base; + } - public static List mergeAuthor(final List a, final List b) { - return mergeAuthor(a, b, THRESHOLD); - } + public static List mergeAuthor(final List a, final List b) { + return mergeAuthor(a, b, THRESHOLD); + } - private static void enrichPidFromList(List base, List enrich, Double threshold) { - if (base == null || enrich == null) - return; + private static void enrichPidFromList(List base, List enrich, Double threshold) { + if (base == null || enrich == null) + return; - // (if an Author has more than 1 pid, it appears 2 times in the list) - final Map basePidAuthorMap = base - .stream() - .filter(a -> a.getPid() != null && !a.getPid().isEmpty()) - .flatMap( - a -> a - .getPid() - .stream() - .filter(Objects::nonNull) - .map(p -> new Tuple2<>(pidToComparableString(p), a))) - .collect(Collectors.toMap(Tuple2::_1, Tuple2::_2, (x1, x2) -> x1)); + // (if an Author has more than 1 pid, it appears 2 times in the list) + final Map basePidAuthorMap = base + .stream() + .filter(a -> a.getPid() != null && !a.getPid().isEmpty()) + .flatMap( + a -> a + .getPid() + .stream() + .filter(Objects::nonNull) + .map(p -> new Tuple2<>(pidToComparableString(p), a))) + .collect(Collectors.toMap(Tuple2::_1, Tuple2::_2, (x1, x2) -> x1)); - // (list of pid that are missing in the other list) - final List> pidToEnrich = enrich - .stream() - .filter(a -> a.getPid() != null && !a.getPid().isEmpty()) - .flatMap( - a -> a - .getPid() - .stream() - .filter(Objects::nonNull) - .filter(p -> !basePidAuthorMap.containsKey(pidToComparableString(p))) - .map(p -> new Tuple2<>(p, a))) - .collect(Collectors.toList()); + // (list of pid that are missing in the other list) + final List> pidToEnrich = enrich + .stream() + .filter(a -> a.getPid() != null && !a.getPid().isEmpty()) + .flatMap( + a -> a + .getPid() + .stream() + .filter(Objects::nonNull) + .filter(p -> !basePidAuthorMap.containsKey(pidToComparableString(p))) + .map(p -> new Tuple2<>(p, a))) + .collect(Collectors.toList()); - pidToEnrich - .forEach( - a -> { - Optional> simAuthor = base - .stream() - .map(ba -> new Tuple2<>(sim(ba, a._2()), ba)) - .max(Comparator.comparing(Tuple2::_1)); + pidToEnrich + .forEach( + a -> { + Optional> simAuthor = base + .stream() + .map(ba -> new Tuple2<>(sim(ba, a._2()), ba)) + .max(Comparator.comparing(Tuple2::_1)); - if (simAuthor.isPresent()) { - double th = threshold; - // increase the threshold if the surname is too short - if (simAuthor.get()._2().getSurname() != null - && simAuthor.get()._2().getSurname().length() <= 3 && threshold > 0.0) - th = 0.99; + if (simAuthor.isPresent()) { + double th = threshold; + // increase the threshold if the surname is too short + if (simAuthor.get()._2().getSurname() != null + && simAuthor.get()._2().getSurname().length() <= 3 && threshold > 0.0) + th = 0.99; - if (simAuthor.get()._1() > th) { - Author r = simAuthor.get()._2(); - if (r.getPid() == null) { - r.setPid(new ArrayList<>()); - } + if (simAuthor.get()._1() > th) { + Author r = simAuthor.get()._2(); + if (r.getPid() == null) { + r.setPid(new ArrayList<>()); + } - // TERRIBLE HACK but for some reason when we create and Array with Arrays.asList, - // it creates of fixed size, and the add method raise UnsupportedOperationException at - // java.util.AbstractList.add - final List tmp = new ArrayList<>(r.getPid()); - tmp.add(a._1()); - r.setPid(tmp); - } - } - }); - } + // TERRIBLE HACK but for some reason when we create and Array with Arrays.asList, + // it creates of fixed size, and the add method raise UnsupportedOperationException at + // java.util.AbstractList.add + final List tmp = new ArrayList<>(r.getPid()); + tmp.add(a._1()); + r.setPid(tmp); + } + } + }); + } - public static String normalizeFullName(final String fullname) { - return nfd(fullname) - .toLowerCase() - // do not compact the regexes in a single expression, would cause StackOverflowError - // in case - // of large input strings - .replaceAll("(\\W)+", " ") - .replaceAll("(\\p{InCombiningDiacriticalMarks})+", " ") - .replaceAll("(\\p{Punct})+", " ") - .replaceAll("(\\d)+", " ") - .replaceAll("(\\n)+", " ") + public static String normalizeFullName(final String fullname) { + return nfd(fullname) + .toLowerCase() + // do not compact the regexes in a single expression, would cause StackOverflowError + // in case + // of large input strings + .replaceAll("(\\W)+", " ") + .replaceAll("(\\p{InCombiningDiacriticalMarks})+", " ") + .replaceAll("(\\p{Punct})+", " ") + .replaceAll("(\\d)+", " ") + .replaceAll("(\\n)+", " ") - .trim(); - } + .trim(); + } - static int hammingDist(String str1, String str2) { - if (str1.length() != str2.length()) - return Math.max(str1.length(), str2.length()); - int i = 0, count = 0; - while (i < str1.length()) { - if (str1.charAt(i) != str2.charAt(i)) - count++; - i++; - } - return count; - } + static int hammingDist(String str1, String str2) { + if (str1.length() != str2.length()) + return Math.max(str1.length(), str2.length()); + int i = 0, count = 0; + while (i < str1.length()) { + if (str1.charAt(i) != str2.charAt(i)) + count++; + i++; + } + return count; + } - private static String authorFieldToBeCompared(Author author) { - if (StringUtils.isNotBlank(author.getSurname())) { - return author.getSurname(); + private static String authorFieldToBeCompared(Author author) { + if (StringUtils.isNotBlank(author.getSurname())) { + return author.getSurname(); - } - if (StringUtils.isNotBlank(author.getFullname())) { - return author.getFullname(); - } - return null; - } + } + if (StringUtils.isNotBlank(author.getFullname())) { + return author.getFullname(); + } + return null; + } - /** - * This method tries to figure out when two author are the same in the contest - * of ORCID enrichment - * @param left Author in the OAF entity - * @param right Author ORCID - * @return based on a heuristic on the names of the authors if they are the same. - */ - public static boolean checkORCIDSimilarity(final Author left, final Author right) { - final Person pl = parse(left); - final Person pr = parse(right); + /** + * This method tries to figure out when two author are the same in the contest + * of ORCID enrichment + * + * @param left Author in the OAF entity + * @param right Author ORCID + * @return based on a heuristic on the names of the authors if they are the same. + */ + public static boolean checkORCIDSimilarity(final Author left, final Author right) { + final Person pl = parse(left); + final Person pr = parse(right); - // If one of them didn't have a surname we verify if they have the fullName not empty - // and verify if the normalized version is equal - if (!(pl.getSurname() != null && pl.getSurname().stream().anyMatch(StringUtils::isNotBlank) && - pr.getSurname() != null && pr.getSurname().stream().anyMatch(StringUtils::isNotBlank))) { + // If one of them didn't have a surname we verify if they have the fullName not empty + // and verify if the normalized version is equal + if (!(pl.getSurname() != null && pl.getSurname().stream().anyMatch(StringUtils::isNotBlank) && + pr.getSurname() != null && pr.getSurname().stream().anyMatch(StringUtils::isNotBlank))) { - if (pl.getFullname() != null && !pl.getFullname().isEmpty() && pr.getFullname() != null - && !pr.getFullname().isEmpty()) { - return pl - .getFullname() - .stream() - .anyMatch( - fl -> pr.getFullname().stream().anyMatch(fr -> normalize(fl).equalsIgnoreCase(normalize(fr)))); - } else { - return false; - } - } - // The Authors have one surname in common - if (pl.getSurname().stream().anyMatch(sl -> pr.getSurname().stream().anyMatch(sr -> sr.equalsIgnoreCase(sl)))) { + if (pl.getFullname() != null && !pl.getFullname().isEmpty() && pr.getFullname() != null + && !pr.getFullname().isEmpty()) { + return pl + .getFullname() + .stream() + .anyMatch( + fl -> pr.getFullname().stream().anyMatch(fr -> normalize(fl).equalsIgnoreCase(normalize(fr)))); + } else { + return false; + } + } + // The Authors have one surname in common + if (pl.getSurname().stream().anyMatch(sl -> pr.getSurname().stream().anyMatch(sr -> sr.equalsIgnoreCase(sl)))) { - // If one of them has only a surname and is the same we can say that they are the same author - if ((pl.getName() == null || pl.getName().stream().allMatch(StringUtils::isBlank)) || - (pr.getName() == null || pr.getName().stream().allMatch(StringUtils::isBlank))) - return true; - // The authors have the same initials of Name in common - if (pl - .getName() - .stream() - .anyMatch( - nl -> pr - .getName() - .stream() - .anyMatch(nr -> nr.equalsIgnoreCase(nl)))) - return true; - } + // If one of them has only a surname and is the same we can say that they are the same author + if ((pl.getName() == null || pl.getName().stream().allMatch(StringUtils::isBlank)) || + (pr.getName() == null || pr.getName().stream().allMatch(StringUtils::isBlank))) + return true; + // The authors have the same initials of Name in common + if (pl + .getName() + .stream() + .anyMatch( + nl -> pr + .getName() + .stream() + .anyMatch(nr -> nr.equalsIgnoreCase(nl)))) + return true; + } - // Sometimes we noticed that publication have author wrote in inverse order Surname, Name - // We verify if we have an exact match between name and surname - if (pl.getSurname().stream().anyMatch(sl -> pr.getName().stream().anyMatch(nr -> nr.equalsIgnoreCase(sl))) && - pl.getName().stream().anyMatch(nl -> pr.getSurname().stream().anyMatch(sr -> sr.equalsIgnoreCase(nl)))) - return true; - else - return false; - } - // + // Sometimes we noticed that publication have author wrote in inverse order Surname, Name + // We verify if we have an exact match between name and surname + if (pl.getSurname().stream().anyMatch(sl -> pr.getName().stream().anyMatch(nr -> nr.equalsIgnoreCase(sl))) && + pl.getName().stream().anyMatch(nl -> pr.getSurname().stream().anyMatch(sr -> sr.equalsIgnoreCase(nl)))) + return true; + else + return false; + } + // - /** - * Method to enrich ORCID information in one list of authors based on another list - * @param baseAuthor the Author List in the OAF Entity - * @param orcidAuthor The list of ORCID Author intersected - * @return The Author List of the OAF Entity enriched with the orcid Author - */ - public static List enrichOrcid(List baseAuthor, List orcidAuthor) { + /** + * Method to enrich ORCID information in one list of authors based on another list + * + * @param baseAuthor the Author List in the OAF Entity + * @param orcidAuthor The list of ORCID Author intersected + * @return The Author List of the OAF Entity enriched with the orcid Author + */ + public static List enrichOrcid(List baseAuthor, List orcidAuthor) { - if (baseAuthor == null || baseAuthor.isEmpty()) - return orcidAuthor; + if (baseAuthor == null || baseAuthor.isEmpty()) + return orcidAuthor; - if (orcidAuthor == null || orcidAuthor.isEmpty()) - return baseAuthor; + if (orcidAuthor == null || orcidAuthor.isEmpty()) + return baseAuthor; - if (baseAuthor.size() == 1 && orcidAuthor.size() > 10) - return baseAuthor; + if (baseAuthor.size() == 1 && orcidAuthor.size() > 10) + return baseAuthor; - final List oAuthor = new ArrayList<>(); - oAuthor.addAll(orcidAuthor); + final List oAuthor = new ArrayList<>(); + oAuthor.addAll(orcidAuthor); - baseAuthor.forEach(ba -> { - Optional aMatch = oAuthor.stream().filter(oa -> checkORCIDSimilarity(ba, oa)).findFirst(); - if (aMatch.isPresent()) { - final Author sameAuthor = aMatch.get(); - addPid(ba, sameAuthor.getPid()); - oAuthor.remove(sameAuthor); - } - }); - return baseAuthor; - } + baseAuthor.forEach(ba -> { + Optional aMatch = oAuthor.stream().filter(oa -> checkORCIDSimilarity(ba, oa)).findFirst(); + if (aMatch.isPresent()) { + final Author sameAuthor = aMatch.get(); + addPid(ba, sameAuthor.getPid()); + oAuthor.remove(sameAuthor); + } + }); + return baseAuthor; + } - private static void addPid(final Author a, final List pids) { + private static void addPid(final Author a, final List pids) { - if (a.getPid() == null) { - a.setPid(new ArrayList<>()); - } + if (a.getPid() == null) { + a.setPid(new ArrayList<>()); + } - a.getPid().addAll(pids); + a.getPid().addAll(pids); - } + } - public static String pidToComparableString(StructuredProperty pid) { - final String classid = pid.getQualifier().getClassid() != null ? pid.getQualifier().getClassid().toLowerCase() - : ""; - return (pid.getQualifier() != null ? classid : "") - + (pid.getValue() != null ? pid.getValue().toLowerCase() : ""); - } + public static String pidToComparableString(StructuredProperty pid) { + final String classid = pid.getQualifier().getClassid() != null ? pid.getQualifier().getClassid().toLowerCase() + : ""; + return (pid.getQualifier() != null ? classid : "") + + (pid.getValue() != null ? pid.getValue().toLowerCase() : ""); + } - public static int countAuthorsPids(List authors) { - if (authors == null) - return 0; + public static int countAuthorsPids(List authors) { + if (authors == null) + return 0; - return (int) authors.stream().filter(AuthorMerger::hasPid).count(); - } + return (int) authors.stream().filter(AuthorMerger::hasPid).count(); + } - private static int authorsSize(List authors) { - if (authors == null) - return 0; - return authors.size(); - } + private static int authorsSize(List authors) { + if (authors == null) + return 0; + return authors.size(); + } - private static Double sim(Author a, Author b) { + private static Double sim(Author a, Author b) { - final Person pa = parse(a); - final Person pb = parse(b); + final Person pa = parse(a); + final Person pb = parse(b); - // if both are accurate (e.g. they have name and surname) - if (pa.isAccurate() & pb.isAccurate()) { - return new JaroWinkler().score(normalize(pa.getSurnameString()), normalize(pb.getSurnameString())) * 0.5 - + new JaroWinkler().score(normalize(pa.getNameString()), normalize(pb.getNameString())) * 0.5; - } else { - return new JaroWinkler() - .score(normalize(pa.getNormalisedFullname()), normalize(pb.getNormalisedFullname())); - } - } + // if both are accurate (e.g. they have name and surname) + if (pa.isAccurate() & pb.isAccurate()) { + return new JaroWinkler().score(normalize(pa.getSurnameString()), normalize(pb.getSurnameString())) * 0.5 + + new JaroWinkler().score(normalize(pa.getNameString()), normalize(pb.getNameString())) * 0.5; + } else { + return new JaroWinkler() + .score(normalize(pa.getNormalisedFullname()), normalize(pb.getNormalisedFullname())); + } + } - private static boolean hasPid(Author a) { - if (a == null || a.getPid() == null || a.getPid().isEmpty()) - return false; - return a.getPid().stream().anyMatch(p -> p != null && StringUtils.isNotBlank(p.getValue())); - } + private static boolean hasPid(Author a) { + if (a == null || a.getPid() == null || a.getPid().isEmpty()) + return false; + return a.getPid().stream().anyMatch(p -> p != null && StringUtils.isNotBlank(p.getValue())); + } - private static Person parse(Author author) { - if (StringUtils.isNotBlank(author.getSurname())) { - return new Person(author.getSurname() + ", " + author.getName(), false); - } else { - if (StringUtils.isNotBlank(author.getFullname())) - return new Person(author.getFullname(), false); - else - return new Person("", false); - } - } + private static Person parse(Author author) { + if (StringUtils.isNotBlank(author.getSurname())) { + return new Person(author.getSurname() + ", " + author.getName(), false); + } else { + if (StringUtils.isNotBlank(author.getFullname())) + return new Person(author.getFullname(), false); + else + return new Person("", false); + } + } - public static String normalize(final String s) { - String[] normalized = nfd(s) - .toLowerCase() - // do not compact the regexes in a single expression, would cause StackOverflowError - // in case - // of large input strings - .replaceAll("(\\W)+", " ") - .replaceAll("(\\p{InCombiningDiacriticalMarks})+", " ") - .replaceAll("(\\p{Punct})+", " ") - .replaceAll("(\\d)+", " ") - .replaceAll("(\\n)+", " ") - .trim() - .split(" "); + public static String normalize(final String s) { + String[] normalized = nfd(s) + .toLowerCase() + // do not compact the regexes in a single expression, would cause StackOverflowError + // in case + // of large input strings + .replaceAll("(\\W)+", " ") + .replaceAll("(\\p{InCombiningDiacriticalMarks})+", " ") + .replaceAll("(\\p{Punct})+", " ") + .replaceAll("(\\d)+", " ") + .replaceAll("(\\n)+", " ") + .trim() + .split(" "); - Arrays.sort(normalized); + Arrays.sort(normalized); - return String.join(" ", normalized); - } + return String.join(" ", normalized); + } - private static String nfd(final String s) { - return Normalizer.normalize(s, Normalizer.Form.NFD); - } + private static String nfd(final String s) { + return Normalizer.normalize(s, Normalizer.Form.NFD); + } } diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java index e2639996c..b87738879 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/bipaffiliations/PrepareAffiliationRelationsTest.java @@ -79,8 +79,8 @@ public class PrepareAffiliationRelationsTest { .getPath(); String pubmedAffiliationRelationsPath = getClass() - .getResource("/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror.json") - .getPath(); + .getResource("/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror.json") + .getPath(); String outputPath = workingDir.toString() + "/actionSet"; diff --git a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/SparkGenerateDoiBoost.scala b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/SparkGenerateDoiBoost.scala index 07d6a0287..9ffaeeeef 100644 --- a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/SparkGenerateDoiBoost.scala +++ b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/SparkGenerateDoiBoost.scala @@ -92,7 +92,6 @@ object SparkGenerateDoiBoost { .mode(SaveMode.Overwrite) .save(s"$workingDirPath/firstJoin") - logger.info("Phase 2) Join Result with MAG") val sj: Dataset[(String, Publication)] = spark.read.load(s"$workingDirPath/firstJoin").as[Publication].map(p => (p.getId, p)) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala index 0d994d202..645df8181 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala @@ -73,7 +73,7 @@ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String] col("id").alias("dnet_id") ) - val orcidDnet = orcidPublication + val orcidDnet = orcidPublication .join( entities, lower(col("schema")).equalTo(lower(col("pid_schema"))) && diff --git a/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/EnrichOrcidTest.scala b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/EnrichOrcidTest.scala index 84483b1a2..cb05332b2 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/EnrichOrcidTest.scala +++ b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/EnrichOrcidTest.scala @@ -6,13 +6,10 @@ import org.junit.jupiter.api.Test import org.slf4j.{Logger, LoggerFactory} import org.apache.spark.sql.functions._ - class EnrichOrcidTest { val log: Logger = LoggerFactory.getLogger(getClass) - - def test() = { val spark = SparkSession.builder().master("local[*]").getOrCreate() // spark.sparkContext.setLogLevel("ERROR") @@ -63,8 +60,7 @@ class EnrichOrcidTest { // }).filter(author => author != null) // }) - - Encoders + Encoders import spark.implicits._ // val enriched = spark.read @@ -76,10 +72,6 @@ class EnrichOrcidTest { // // .show() - - - - } } From 48430a32a63cc80b32bce14104b9f11037960adf Mon Sep 17 00:00:00 2001 From: dimitrispie Date: Fri, 1 Dec 2023 11:35:01 +0200 Subject: [PATCH 37/60] Update StatsAtomicActionsJob.java Added indi_funded_result_with_fundref indicator --- .../StatsAtomicActionsJob.java | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsAtomicActionsJob.java b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsAtomicActionsJob.java index a64a63c24..0c3c2f3c0 100644 --- a/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsAtomicActionsJob.java +++ b/dhp-workflows/dhp-stats-actionsets/src/main/java/eu/dnetlib/dhp/actionmanager/stats_actionsets/StatsAtomicActionsJob.java @@ -96,14 +96,16 @@ public class StatsAtomicActionsJob implements Serializable { .sql( String .format( - "select b.%s as id, is_gold, is_bronze_oa, is_hybrid,green_oa, in_diamond_journal,f.fundref as publicly_funded " - + "from %s.indi_pub_bronze_oa b " + - "left outer join %s.indi_pub_gold_oa g on g.id=b.id " + - "left outer join %s.indi_pub_hybrid h on b.id=h.id " + - "left outer join %s.indi_pub_green_oa gr on b.id=gr.id " + + "select r.%s as id, is_gold, is_bronze_oa, is_hybrid,green_oa, in_diamond_journal,f.publicly_funded as publicly_funded " + + + "from %s.publication r " + + "left outer join %s.indi_pub_bronze_oa b on r.id=b.id " + + "left outer join %s.indi_pub_gold_oa g on r.id=g.id " + + "left outer join %s.indi_pub_hybrid h on r.id=h.id " + + "left outer join %s.indi_pub_green_oa gr on r.id=gr.id " + "left outer join %s.indi_pub_diamond d on b.id=d.id " + - "left outer join %s.indi_funded_result_with_fundref f on b.id=f.id ", - resultAttributeName, dbname, dbname, dbname, dbname, dbname, dbname)) + "left outer join %s.indi_pub_publicly_funded f on r.id=f.id ", + resultAttributeName, dbname, dbname, dbname, dbname, dbname, dbname, dbname)) .as(Encoders.bean(StatsResultEnhancementModel.class)) .write() .mode(SaveMode.Overwrite) @@ -136,12 +138,12 @@ public class StatsAtomicActionsJob implements Serializable { r.setIsInDiamondJournal(usm.isIn_diamond_journal()); r.setIsGreen(usm.isGreen_oa()); r.setPubliclyFunded(usm.isPublicly_funded()); - if (usm.isIs_bronze_oa()) - r.setOpenAccessColor(OpenAccessColor.bronze); - else if (usm.isIs_gold()) + if (usm.isIs_gold()) r.setOpenAccessColor(OpenAccessColor.gold); else if (usm.isIs_hybrid()) r.setOpenAccessColor(OpenAccessColor.hybrid); + else if (usm.isIs_bronze_oa()) + r.setOpenAccessColor(OpenAccessColor.bronze); return r; }, Encoders.bean(Result.class)); } From bf0fd27c362bc07a0fe5579ac7787dfe0134cb30 Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Fri, 1 Dec 2023 12:16:42 +0100 Subject: [PATCH 38/60] Removed unused function Applied PR Comment of Giambattista in the PR --- .../java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java | 11 ----------- .../orcid/SparkEnrichGraphWithOrcidAuthors.scala | 8 ++++++-- 2 files changed, 6 insertions(+), 13 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java index d7224af78..0153e52d5 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java @@ -132,17 +132,6 @@ public class AuthorMerger { .trim(); } - static int hammingDist(String str1, String str2) { - if (str1.length() != str2.length()) - return Math.max(str1.length(), str2.length()); - int i = 0, count = 0; - while (i < str1.length()) { - if (str1.charAt(i) != str2.charAt(i)) - count++; - i++; - } - return count; - } private static String authorFieldToBeCompared(Author author) { if (StringUtils.isNotBlank(author.getSurname())) { diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala index 645df8181..4822059c6 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/SparkEnrichGraphWithOrcidAuthors.scala @@ -2,10 +2,12 @@ package eu.dnetlib.dhp.enrich.orcid import eu.dnetlib.dhp.application.AbstractScalaApplication import eu.dnetlib.dhp.oa.merge.AuthorMerger +import eu.dnetlib.dhp.schema.common.ModelSupport import eu.dnetlib.dhp.schema.oaf.{OtherResearchProduct, Publication, Result, Software} import org.apache.spark.sql.functions._ import org.apache.spark.sql._ import org.slf4j.{Logger, LoggerFactory} +import scala.collection.JavaConverters._ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String], log: Logger) extends AbstractScalaApplication(propertyPath, args, log: Logger) { @@ -21,6 +23,8 @@ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String] val targetPath = parser.get("targetPath") log.info(s"targetPath is '$targetPath'") val orcidPublication: Dataset[Row] = generateOrcidTable(spark, orcidPath) +// ModelSupport.entityTypes.entrySet().asScala.filter(k => k.getKey.getClass isInstance(Result)) + enrichResult( spark, s"$graphPath/publication", @@ -63,7 +67,7 @@ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String] .schema(enc.schema) .json(graphPath) .select(col("id"), col("datainfo"), col("instance")) - .where("datainfo.deletedbyinference = false") + .where("datainfo.deletedbyinference != true") .drop("datainfo") .withColumn("instances", explode(col("instance"))) .withColumn("pids", explode(col("instances.pid"))) @@ -109,7 +113,7 @@ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String] .load(s"$inputPath/Works") .select(col("orcid"), explode(col("pids")).alias("identifier")) .where( - "identifier.schema = 'doi' or identifier.schema ='pmid' or identifier.schema ='pmc' or identifier.schema ='arxiv' or identifier.schema ='handle'" + "identifier.schema IN('doi','pmid','pmc','arxiv','handle')" ) val orcidPublication = orcidAuthors .join(orcidWorks, orcidAuthors("orcid").equalTo(orcidWorks("orcid"))) From 76594ded23455ecf67addcbebd3b4ca45b73199e Mon Sep 17 00:00:00 2001 From: dimitrispie Date: Fri, 1 Dec 2023 13:38:19 +0200 Subject: [PATCH 39/60] Changes to indicators Fixes on open access colours indicators - indi_pub_green_oa - indi_pub_gold_oa - indi_pub_hybrid - indi_pub_bronze_oa - indi_pub_diamond --- .../scripts/step16-createIndicatorsTables.sql | 129 +++++++++++++----- .../scripts/step20-createMonitorDB.sql | 2 + .../scripts/step20-createMonitorDBAll.sql | 2 + .../graph/stats/oozie_app/scripts/step3.sql | 18 +++ .../graph/stats/oozie_app/scripts/step4.sql | 15 ++ .../graph/stats/oozie_app/scripts/step5.sql | 16 +++ .../graph/stats/oozie_app/scripts/step6.sql | 14 ++ 7 files changed, 162 insertions(+), 34 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql index 8180e6527..fea449de6 100755 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql @@ -1,6 +1,18 @@ -- Sprint 1 ---- drop table if exists ${stats_db_name}.indi_pub_green_oa purge; +--create table if not exists ${stats_db_name}.indi_pub_green_oa stored as parquet as +--select distinct p.id, coalesce(green_oa, 0) as green_oa +--from ${stats_db_name}.publication p +-- left outer join ( +-- select p.id, 1 as green_oa +-- from ${stats_db_name}.publication p +-- join ${stats_db_name}.result_instance ri on ri.id = p.id +-- join ${stats_db_name}.datasource on datasource.id = ri.hostedby +-- where datasource.type like '%Repository%' +-- and (ri.accessright = 'Open Access' +-- or ri.accessright = 'Embargo' or ri.accessright = 'Open Source')) tmp +-- on p.id= tmp.id; create table if not exists ${stats_db_name}.indi_pub_green_oa stored as parquet as select distinct p.id, coalesce(green_oa, 0) as green_oa from ${stats_db_name}.publication p @@ -11,7 +23,7 @@ from ${stats_db_name}.publication p join ${stats_db_name}.datasource on datasource.id = ri.hostedby where datasource.type like '%Repository%' and (ri.accessright = 'Open Access' - or ri.accessright = 'Embargo' or ri.accessright = 'Open Source')) tmp + or ri.accessright = 'Embargo' or ri.accessright = 'Open Source') and datasource.name!='Other') tmp on p.id= tmp.id; drop table if exists ${stats_db_name}.indi_pub_grey_lit purge; @@ -183,15 +195,24 @@ drop table if exists ${stats_db_name}.tmp purge; ---- Sprint 4 ---- drop table if exists ${stats_db_name}.indi_pub_diamond purge; +--create table if not exists ${stats_db_name}.indi_pub_diamond stored as parquet as +--select distinct pd.id, coalesce(in_diamond_journal, 0) as in_diamond_journal +--from ${stats_db_name}.publication_datasources pd +-- left outer join ( +-- select pd.id, 1 as in_diamond_journal from ${stats_db_name}.publication_datasources pd +-- join ${stats_db_name}.datasource d on d.id=pd.datasource +-- join STATS_EXT.plan_s_jn ps where (ps.issn_print=d.issn_printed and ps.issn_online=d.issn_online) +-- and (ps.journal_is_in_doaj=true or ps.journal_is_oa=true) and ps.has_apc=false) tmp +-- on pd.id=tmp.id; + create table if not exists ${stats_db_name}.indi_pub_diamond stored as parquet as select distinct pd.id, coalesce(in_diamond_journal, 0) as in_diamond_journal from ${stats_db_name}.publication_datasources pd - left outer join ( - select pd.id, 1 as in_diamond_journal from ${stats_db_name}.publication_datasources pd - join ${stats_db_name}.datasource d on d.id=pd.datasource - join STATS_EXT.plan_s_jn ps where (ps.issn_print=d.issn_printed and ps.issn_online=d.issn_online) - and (ps.journal_is_in_doaj=true or ps.journal_is_oa=true) and ps.has_apc=false) tmp - on pd.id=tmp.id; +left outer join (select pd.id, 1 as in_diamond_journal from ${stats_db_name}.publication_datasources pd +join ${stats_db_name}.datasource d on d.id=pd.datasource +join STATS_EXT.plan_s_jn ps where (ps.issn_print=d.issn_printed and ps.issn_online=d.issn_online) +and (ps.journal_is_in_doaj=true or ps.journal_is_oa=true) and ps.has_apc=false) tmp +on pd.id=tmp.id; drop table if exists ${stats_db_name}.indi_pub_in_transformative purge; @@ -312,28 +333,55 @@ drop table if exists ${stats_db_name}.indi_pub_gold_oa purge; -- JOIN gold_oa on issn.issn = gold_oa.issn) tmp -- on pd.id=tmp.id; +--create table if not exists ${stats_db_name}.indi_pub_gold_oa stored as parquet as +--with gold_oa as ( +--SELECT issn,issn_l from stats_ext.issn_gold_oa_dataset_v5), +--issn AS (SELECT * FROM +--(SELECT id,issn_printed as issn FROM ${stats_db_name}.datasource +--WHERE issn_printed IS NOT NULL +--UNION ALL +--SELECT id, issn_online as issn FROM ${stats_db_name}.datasource +--WHERE issn_online IS NOT NULL or id like '%doajarticles%') as issn +--WHERE LENGTH(issn) > 7), +--alljournals AS(select issn, issn_l from stats_ext.alljournals +--where journal_is_in_doaj=true or journal_is_oa=true) +--SELECT DISTINCT pd.id, coalesce(is_gold, 0) as is_gold +--FROM ${stats_db_name}.publication_datasources pd +--left outer join ( +--select pd.id, 1 as is_gold FROM ${stats_db_name}.publication_datasources pd +--JOIN issn on issn.id=pd.datasource +--JOIN gold_oa on issn.issn = gold_oa.issn +--join alljournals on issn.issn=alljournals.issn +--left outer join ${stats_db_name}.result_instance ri on ri.id=pd.id +--and ri.accessright!='Closed Access' and ri.accessright_uw='gold') tmp +--on pd.id=tmp.id; create table if not exists ${stats_db_name}.indi_pub_gold_oa stored as parquet as with gold_oa as ( -SELECT issn,issn_l from stats_ext.issn_gold_oa_dataset_v5), -issn AS (SELECT * FROM -(SELECT id,issn_printed as issn FROM ${stats_db_name}.datasource -WHERE issn_printed IS NOT NULL -UNION ALL -SELECT id, issn_online as issn FROM ${stats_db_name}.datasource -WHERE issn_online IS NOT NULL or id like '%doajarticles%') as issn -WHERE LENGTH(issn) > 7), -alljournals AS(select issn, issn_l from stats_ext.alljournals -where journal_is_in_doaj=true or journal_is_oa=true) +select distinct issn from ( + SELECT issn_l as issn from stats_ext.issn_gold_oa_dataset_v5 + UNION ALL + SELECT issn as issn from stats_ext.issn_gold_oa_dataset_v5 + UNION ALL + select issn from stats_ext.alljournals where journal_is_in_doaj=true or journal_is_oa=true + UNION ALL + select issn_l as issn from stats_ext.alljournals where journal_is_in_doaj=true or journal_is_oa=true) foo), +dd as ( +select distinct * from ( + select id, issn_printed as issn from ${stats_db_name}.datasource d where d.id like '%doajarticles%' + UNION ALL + select id, issn_online as issn from ${stats_db_name}.datasource d where d.id like '%doajarticles%' + UNION ALL + select id, issn_printed as issn from ${stats_db_name}.datasource d join gold_oa on gold_oa.issn=d.issn_printed + UNION ALL + select id, issn_online as issn from ${stats_db_name}.datasource d join gold_oa on gold_oa.issn=d.issn_online) foo +) SELECT DISTINCT pd.id, coalesce(is_gold, 0) as is_gold FROM ${stats_db_name}.publication_datasources pd left outer join ( -select pd.id, 1 as is_gold FROM ${stats_db_name}.publication_datasources pd -JOIN issn on issn.id=pd.datasource -JOIN gold_oa on issn.issn = gold_oa.issn -join alljournals on issn.issn=alljournals.issn -left outer join ${stats_db_name}.result_instance ri on ri.id=pd.id -and ri.accessright!='Closed Access' and ri.accessright_uw='gold') tmp -on pd.id=tmp.id; + select pd.id, 1 as is_gold + FROM ${stats_db_name}.publication_datasources pd + join dd on dd.id=pd.datasource + left outer join ${stats_db_name}.result_accessroute ra on ra.id = pd.id where ra.accessroute = 'gold') tmp on tmp.id=pd.id; drop table if exists ${stats_db_name}.indi_pub_hybrid_oa_with_cc purge; @@ -421,15 +469,26 @@ drop table if exists ${stats_db_name}.indi_pub_hybrid purge; -- where (gold_oa.journal_is_in_doaj=false or gold_oa.journal_is_oa=false))tmp -- on pd.id=tmp.id; +--create table if not exists ${stats_db_name}.indi_pub_hybrid stored as parquet as +--select distinct pd.id,coalesce(is_hybrid,0) is_hybrid from ${stats_db_name}.publication_datasources pd +--left outer join (select pd.id, 1 as is_hybrid from ${stats_db_name}.publication_datasources pd +--join ${stats_db_name}.datasource d on pd.datasource=d.id +--join ${stats_db_name}.result_instance ri on ri.id=pd.id +--join ${stats_db_name}.indi_pub_gold_oa indi_gold on indi_gold.id=pd.id +--join ${stats_db_name}.result_accessroute ra on ra.id=pd.id +--where d.type like '%Journal%' and ri.accessright!='Closed Access' and (ri.accessright_uw!='gold' +--or indi_gold.is_gold=0) and (ra.accessroute='hybrid' or ri.license is not null)) tmp +--on pd.id=tmp.id; + create table if not exists ${stats_db_name}.indi_pub_hybrid stored as parquet as -select distinct pd.id,coalesce(is_hybrid,0) is_hybrid from ${stats_db_name}.publication_datasources pd -left outer join (select pd.id, 1 as is_hybrid from ${stats_db_name}.publication_datasources pd -join ${stats_db_name}.datasource d on pd.datasource=d.id +select distinct pd.id,coalesce(is_hybrid,0) is_hybrid from ${stats_db_name}.publication pd +left outer join (select pd.id, 1 as is_hybrid from ${stats_db_name}.publication pd join ${stats_db_name}.result_instance ri on ri.id=pd.id join ${stats_db_name}.indi_pub_gold_oa indi_gold on indi_gold.id=pd.id join ${stats_db_name}.result_accessroute ra on ra.id=pd.id -where d.type like '%Journal%' and ri.accessright!='Closed Access' and (ri.accessright_uw!='gold' -or indi_gold.is_gold=0) and (ra.accessroute='hybrid' or ri.license is not null)) tmp +join ${stats_db_name}.datasource d on d.id=ri.hostedby +where indi_gold.is_gold=0 and ((d.type like '%Journal%' and ri.accessright!='Closed Access' and ri.accessright!='Restricted' and ri.license is not null) or +ra.accessroute='hybrid'))tmp on pd.id=tmp.id; drop table if exists ${stats_db_name}.indi_org_fairness purge; @@ -814,14 +873,16 @@ drop table if exists ${stats_db_name}.indi_pub_bronze_oa purge; --and ri.accessright='Open Access') tmp on tmp.id=p.id; create table ${stats_db_name}.indi_pub_bronze_oa stored as parquet as -select distinct pd.id,coalesce(is_bronze_oa,0) is_bronze_oa from ${stats_db_name}.publication_datasources pd -left outer join (select pd.id, 1 as is_bronze_oa from ${stats_db_name}.publication_datasources pd -join ${stats_db_name}.datasource d on pd.datasource=d.id +select distinct pd.id,coalesce(is_bronze_oa,0) is_bronze_oa from ${stats_db_name}.publication pd +left outer join (select pd.id, 1 as is_bronze_oa from ${stats_db_name}.publication pd join ${stats_db_name}.result_instance ri on ri.id=pd.id join ${stats_db_name}.indi_pub_gold_oa indi_gold on indi_gold.id=pd.id +join ${stats_db_name}.indi_pub_hybrid indi_hybrid on indi_hybrid.id=pd.id join ${stats_db_name}.result_accessroute ra on ra.id=pd.id -where d.type like '%Journal%' and ri.accessright!='Closed Access' and (ri.accessright_uw!='gold' -or indi_gold.is_gold=0) and (ra.accessroute='bronze' or ri.license is null)) tmp +join ${stats_db_name}.datasource d on d.id=ri.hostedby +where indi_gold.is_gold=0 and indi_hybrid.is_hybrid=0 +and ((d.type like '%Journal%' and ri.accessright!='Closed Access' +and ri.accessright!='Restricted' and ri.license is null) or ra.accessroute='bronze')) tmp on pd.id=tmp.id; CREATE TEMPORARY TABLE ${stats_db_name}.project_year_result_year as diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql index b52abd865..c61a19e5c 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql @@ -64,6 +64,8 @@ create table TARGET.result_accessroute stored as parquet as select * from SOURCE create table TARGET.result_orcid stored as parquet as select * from SOURCE.result_orcid orig where exists (select 1 from TARGET.result r where r.id=orig.id); +create table TARGET.result_instance stored as parquet as select * from SOURCE.result_instance orig where exists (select 1 from TARGET.result r where r.id=orig.id); + create view TARGET.foo1 as select * from SOURCE.result_result rr where rr.source in (select id from TARGET.result); create view TARGET.foo2 as select * from SOURCE.result_result rr where rr.target in (select id from TARGET.result); create table TARGET.result_result STORED AS PARQUET as select distinct * from (select * from TARGET.foo1 union all select * from TARGET.foo2) foufou; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql index 2b6a68514..167aac726 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql @@ -248,6 +248,8 @@ create table TARGET.indi_impact_measures stored as parquet as select * from SOUR create table TARGET.indi_pub_interdisciplinarity stored as parquet as select * from SOURCE.indi_pub_interdisciplinarity orig where exists (select 1 from TARGET.result r where r.id=orig.id); create table TARGET.result_apc_affiliations stored as parquet as select * from SOURCE.result_apc_affiliations orig where exists (select 1 from TARGET.result r where r.id=orig.id); +create table TARGET.result_instance stored as parquet as select * from SOURCE.result_instance orig where exists (select 1 from TARGET.result r where r.id=orig.id); +create table TARGET.result_orcid stored as parquet as select * from SOURCE.result_orcid orig where exists (select 1 from TARGET.result r where r.id=orig.id); create table TARGET.indi_is_project_result_after stored as parquet as select * from SOURCE.indi_is_project_result_after orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); create view TARGET.indi_is_funder_plan_s as select * from SOURCE.indi_is_funder_plan_s; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql index eb97263a7..0384de4ec 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step3.sql @@ -5,6 +5,8 @@ ------------------------------------------------------ -- Dataset temporary table supporting updates +DROP TABLE IF EXISTS ${stats_db_name}.dataset_tmp purge; + CREATE TABLE ${stats_db_name}.dataset_tmp ( id STRING, @@ -40,6 +42,8 @@ SELECT substr(d.id, 4) AS id, FROM ${openaire_db_name}.dataset d WHERE d.datainfo.deletedbyinference = FALSE and d.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_citations purge; + CREATE TABLE ${stats_db_name}.dataset_citations STORED AS PARQUET AS SELECT substr(d.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites FROM ${openaire_db_name}.dataset d @@ -47,12 +51,16 @@ FROM ${openaire_db_name}.dataset d WHERE xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != "" and d.datainfo.deletedbyinference = false and d.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_classifications purge; + CREATE TABLE ${stats_db_name}.dataset_classifications STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_concepts purge; + CREATE TABLE ${stats_db_name}.dataset_concepts STORED AS PARQUET AS SELECT substr(p.id, 4) as id, case when contexts.context.id RLIKE '^[^::]+::[^::]+::.+$' then contexts.context.id @@ -62,6 +70,8 @@ from ${openaire_db_name}.dataset p LATERAL VIEW explode(p.context) contexts as context where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_datasources purge; + CREATE TABLE ${stats_db_name}.dataset_datasources STORED AS PARQUET AS SELECT p.id, case when d.id IS NULL THEN 'other' ELSE p.datasource END AS datasource FROM ( @@ -74,23 +84,31 @@ FROM ( FROM ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference = false and d.datainfo.invisible=false) d ON p.datasource = d.id; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_languages purge; + CREATE TABLE ${stats_db_name}.dataset_languages STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, p.language.classname AS language FROM ${openaire_db_name}.dataset p where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_oids purge; + CREATE TABLE ${stats_db_name}.dataset_oids STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.originalid) oids AS ids where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_pids purge; + CREATE TABLE ${stats_db_name}.dataset_pids STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid FROM ${openaire_db_name}.dataset p LATERAL VIEW explode(p.pid) pids AS ppid where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.dataset_topics purge; + CREATE TABLE ${stats_db_name}.dataset_topics STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.dataset p diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql index 0d1f6323e..d8f4d65e4 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step4.sql @@ -5,6 +5,7 @@ -------------------------------------------------------- -- Software temporary table supporting updates +DROP TABLE IF EXISTS ${stats_db_name}.software_tmp purge; CREATE TABLE ${stats_db_name}.software_tmp ( id STRING, @@ -40,6 +41,8 @@ SELECT substr(s.id, 4) as id, from ${openaire_db_name}.software s where s.datainfo.deletedbyinference = false and s.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.software_citations purge; + CREATE TABLE ${stats_db_name}.software_citations STORED AS PARQUET AS SELECT substr(s.id, 4) as id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites FROM ${openaire_db_name}.software s @@ -47,6 +50,8 @@ FROM ${openaire_db_name}.software s where xpath_string(citation.value, "//citation/id[@type='openaire']/@value") != "" and s.datainfo.deletedbyinference = false and s.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.software_classifications purge; + CREATE TABLE ${stats_db_name}.software_classifications STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.software p @@ -62,6 +67,8 @@ FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.context) contexts AS context where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.software_datasources purge; + CREATE TABLE ${stats_db_name}.software_datasources STORED AS PARQUET AS SELECT p.id, CASE WHEN d.id IS NULL THEN 'other' ELSE p.datasource end as datasource FROM ( @@ -74,23 +81,31 @@ FROM ( FROM ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference = false and d.datainfo.invisible=false) d ON p.datasource = d.id; +DROP TABLE IF EXISTS ${stats_db_name}.software_languages purge; + CREATE TABLE ${stats_db_name}.software_languages STORED AS PARQUET AS select substr(p.id, 4) AS id, p.language.classname AS language FROM ${openaire_db_name}.software p where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.software_oids purge; + CREATE TABLE ${stats_db_name}.software_oids STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.originalid) oids AS ids where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.software_pids purge; + CREATE TABLE ${stats_db_name}.software_pids STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid FROM ${openaire_db_name}.software p LATERAL VIEW explode(p.pid) pids AS ppid where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.software_topics purge; + CREATE TABLE ${stats_db_name}.software_topics STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.software p diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql index 06b616d6a..fae0fbb63 100755 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step5.sql @@ -5,6 +5,8 @@ -------------------------------------------------------------------------------- -- Otherresearchproduct temporary table supporting updates +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_tmp purge; + CREATE TABLE ${stats_db_name}.otherresearchproduct_tmp ( id STRING, @@ -40,6 +42,8 @@ FROM ${openaire_db_name}.otherresearchproduct o WHERE o.datainfo.deletedbyinference = FALSE and o.datainfo.invisible=false; -- Otherresearchproduct_citations +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_citations purge; + CREATE TABLE ${stats_db_name}.otherresearchproduct_citations STORED AS PARQUET AS SELECT substr(o.id, 4) AS id, xpath_string(citation.value, "//citation/id[@type='openaire']/@value") AS cites FROM ${openaire_db_name}.otherresearchproduct o LATERAL VIEW explode(o.extrainfo) citations AS citation @@ -51,6 +55,8 @@ SELECT substr(p.id, 4) AS id, instancetype.classname AS type FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.instance.instancetype) instances AS instancetype where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_concepts purge; + CREATE TABLE ${stats_db_name}.otherresearchproduct_concepts STORED AS PARQUET AS SELECT substr(p.id, 4) as id, case when contexts.context.id RLIKE '^[^::]+::[^::]+::.+$' then contexts.context.id @@ -59,6 +65,8 @@ SELECT substr(p.id, 4) as id, case FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.context) contexts AS context where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_datasources purge; + CREATE TABLE ${stats_db_name}.otherresearchproduct_datasources STORED AS PARQUET AS SELECT p.id, CASE WHEN d.id IS NULL THEN 'other' ELSE p.datasource END AS datasource FROM (SELECT substr(p.id, 4) AS id, substr(instances.instance.hostedby.key, 4) AS datasource @@ -68,21 +76,29 @@ FROM (SELECT substr(p.id, 4) AS id, substr(instances.instance.hostedby.key, 4) A from ${openaire_db_name}.datasource d WHERE d.datainfo.deletedbyinference = false and d.datainfo.invisible=false) d on p.datasource = d.id; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_languages purge; + CREATE TABLE ${stats_db_name}.otherresearchproduct_languages STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, p.language.classname AS language FROM ${openaire_db_name}.otherresearchproduct p where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_oids purge; + CREATE TABLE ${stats_db_name}.otherresearchproduct_oids STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.originalid) oids AS ids where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_pids purge; + CREATE TABLE ${stats_db_name}.otherresearchproduct_pids STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, ppid.qualifier.classname AS type, ppid.value AS pid FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.pid) pids AS ppid where p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.otherresearchproduct_topics purge; + CREATE TABLE ${stats_db_name}.otherresearchproduct_topics STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, subjects.subject.qualifier.classname AS type, subjects.subject.value AS topic FROM ${openaire_db_name}.otherresearchproduct p LATERAL VIEW explode(p.subject) subjects AS subject diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql index e0522e149..e5b3f504e 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step6.sql @@ -3,29 +3,39 @@ -- Project table/view and Project related tables/views ------------------------------------------------------ ------------------------------------------------------ +DROP TABLE IF EXISTS ${stats_db_name}.project_oids purge; + CREATE TABLE ${stats_db_name}.project_oids STORED AS PARQUET AS SELECT substr(p.id, 4) AS id, oids.ids AS oid FROM ${openaire_db_name}.project p LATERAL VIEW explode(p.originalid) oids AS ids where p.datainfo.deletedbyinference=false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.project_organizations purge; + CREATE TABLE ${stats_db_name}.project_organizations STORED AS PARQUET AS SELECT substr(r.source, 4) AS id, substr(r.target, 4) AS organization from ${openaire_db_name}.relation r WHERE r.reltype = 'projectOrganization' and r.source like '40|%' and r.datainfo.deletedbyinference = false and r.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.project_results purge; + CREATE TABLE ${stats_db_name}.project_results STORED AS PARQUET AS SELECT substr(r.target, 4) AS id, substr(r.source, 4) AS result, r.datainfo.provenanceaction.classname as provenance FROM ${openaire_db_name}.relation r WHERE r.reltype = 'resultProject' and r.target like '40|%' and r.datainfo.deletedbyinference = false and r.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.project_classification purge; + create table ${stats_db_name}.project_classification STORED AS PARQUET as select substr(p.id, 4) as id, class.h2020programme.code, class.level1, class.level2, class.level3 from ${openaire_db_name}.project p lateral view explode(p.h2020classification) classifs as class where p.datainfo.deletedbyinference=false and p.datainfo.invisible=false and class.h2020programme is not null; +DROP TABLE IF EXISTS ${stats_db_name}.project_tmp purge; + CREATE TABLE ${stats_db_name}.project_tmp ( id STRING, @@ -80,12 +90,16 @@ SELECT substr(p.id, 4) AS id, FROM ${openaire_db_name}.project p WHERE p.datainfo.deletedbyinference = false and p.datainfo.invisible=false; +DROP TABLE IF EXISTS ${stats_db_name}.funder purge; + create table ${stats_db_name}.funder STORED AS PARQUET as select distinct xpath_string(fund, '//funder/id') as id, xpath_string(fund, '//funder/name') as name, xpath_string(fund, '//funder/shortname') as shortname from ${openaire_db_name}.project p lateral view explode(p.fundingtree.value) fundingtree as fund; +DROP TABLE IF EXISTS ${stats_db_name}.project_organization_contribution purge; + CREATE TABLE ${stats_db_name}.project_organization_contribution STORED AS PARQUET AS SELECT distinct substr(r.source, 4) AS project, substr(r.target, 4) AS organization, properties[0].value contribution, properties[1].value currency From a397112cb86a6f26798b18ce159837b032878fe1 Mon Sep 17 00:00:00 2001 From: dimitrispie Date: Fri, 1 Dec 2023 15:00:18 +0200 Subject: [PATCH 40/60] Add new indicator Add indi_pub_publicly_funded --- .../scripts/step16-createIndicatorsTables.sql | 21 +++++++++++++++++++ .../scripts/step20-createMonitorDB.sql | 1 + .../scripts/step20-createMonitorDBAll.sql | 1 + 3 files changed, 23 insertions(+) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql index fea449de6..5aa14e2c2 100755 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step16-createIndicatorsTables.sql @@ -1177,3 +1177,24 @@ select allresults.ri_initiative, result_findable.no_result_findable/allresults.n from allresults join result_findable on result_findable.ri_initiative=allresults.ri_initiative; +create table if not exists ${stats_db_name}.indi_pub_publicly_funded stored as parquet as +with org_names_pids as +(select org.id,name, pid from ${stats_db_name}.organization org +join ${stats_db_name}.organization_pids op on org.id=op.id), +publicly_funded_orgs as +(select distinct name from +(select pf.name from stats_ext.insitutions_for_publicly_funded pf +join ${stats_db_name}.fundref f on f.name=pf.name where f.type='government' +union all +select pf.name from stats_ext.insitutions_for_publicly_funded pf +join ${stats_db_name}.project p on p.funder=pf.name +union all +select pf.name from stats_ext.insitutions_for_publicly_funded pf +join org_names_pids op on (op.name=pf.name or op.pid=pf.ror) +and pf.publicly_funded='yes') foo) +select distinct p.id, coalesce(publicly_funded, 0) as publicly_funded +from ${stats_db_name}.publication p +left outer join ( +select distinct ro.id, 1 as publicly_funded from result_organization ro +join ${stats_db_name}.organization o on o.id=ro.organization +join publicly_funded_orgs pfo on o.name=pfo.name) tmp on p.id=tmp.id; \ No newline at end of file diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql index c61a19e5c..cc8348f26 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB.sql @@ -63,6 +63,7 @@ create table TARGET.result_fos stored as parquet as select * from SOURCE.result_ create table TARGET.result_accessroute stored as parquet as select * from SOURCE.result_accessroute orig where exists (select 1 from TARGET.result r where r.id=orig.id); create table TARGET.result_orcid stored as parquet as select * from SOURCE.result_orcid orig where exists (select 1 from TARGET.result r where r.id=orig.id); +create table TARGET.indi_pub_publicly_funded stored as parquet as select * from SOURCE.indi_pub_publicly_funded orig where exists (select 1 from TARGET.result r where r.id=orig.id); create table TARGET.result_instance stored as parquet as select * from SOURCE.result_instance orig where exists (select 1 from TARGET.result r where r.id=orig.id); diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql index 167aac726..68417f3e0 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql @@ -250,6 +250,7 @@ create table TARGET.indi_pub_interdisciplinarity stored as parquet as select * f create table TARGET.result_apc_affiliations stored as parquet as select * from SOURCE.result_apc_affiliations orig where exists (select 1 from TARGET.result r where r.id=orig.id); create table TARGET.result_instance stored as parquet as select * from SOURCE.result_instance orig where exists (select 1 from TARGET.result r where r.id=orig.id); create table TARGET.result_orcid stored as parquet as select * from SOURCE.result_orcid orig where exists (select 1 from TARGET.result r where r.id=orig.id); +create table TARGET.indi_pub_publicly_funded stored as parquet as select * from SOURCE.indi_pub_publicly_funded orig where exists (select 1 from TARGET.result r where r.id=orig.id); create table TARGET.indi_is_project_result_after stored as parquet as select * from SOURCE.indi_is_project_result_after orig where exists (select 1 from TARGET.result r where r.id=orig.result_id); create view TARGET.indi_is_funder_plan_s as select * from SOURCE.indi_is_funder_plan_s; From c9d995dde0d48b56e99af43e4c91273817def678 Mon Sep 17 00:00:00 2001 From: dimitrispie Date: Fri, 1 Dec 2023 15:44:35 +0200 Subject: [PATCH 41/60] New institutions added --- .../stats/oozie_app/scripts/step20-createMonitorDBAll.sql | 5 ++++- .../scripts/step20-createMonitorDB_institutions.sql | 4 +++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql index 68417f3e0..42812d159 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDBAll.sql @@ -78,7 +78,10 @@ create table TARGET.result stored as parquet as 'openorgs____::4d4051b56708688235252f1d8fddb8c1', -- Iscte - Instituto Universitário de Lisboa 'openorgs____::5d55fb216b14691cf68218daf5d78cd9', -- Munster Technological University 'openorgs____::0fccc7640f0cb44d5cd1b06b312a06b9', -- Cardiff University - 'openorgs____::8839b55dae0c84d56fd533f52d5d483a' -- Leibniz Institute of Ecological Urban and Regional Development + 'openorgs____::8839b55dae0c84d56fd533f52d5d483a', -- Leibniz Institute of Ecological Urban and Regional Development + 'openorgs____::526468206bca24c1c90da6a312295cf4', -- Cyprus University of Technology + 'openorgs____::b5ca9d4340e26454e367e2908ef3872f' -- Alma Mater Studiorum University of Bologna + ) )) foo; create view if not exists TARGET.category as select * from SOURCE.category; diff --git a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql index d2f08b391..2c0ac337c 100644 --- a/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql +++ b/dhp-workflows/dhp-stats-update/src/main/resources/eu/dnetlib/dhp/oa/graph/stats/oozie_app/scripts/step20-createMonitorDB_institutions.sql @@ -58,5 +58,7 @@ create table TARGET.result stored as parquet as 'openorgs____::4d4051b56708688235252f1d8fddb8c1', -- Iscte - Instituto Universitário de Lisboa 'openorgs____::5d55fb216b14691cf68218daf5d78cd9', -- Munster Technological University 'openorgs____::0fccc7640f0cb44d5cd1b06b312a06b9', -- Cardiff University - 'openorgs____::8839b55dae0c84d56fd533f52d5d483a' -- Leibniz Institute of Ecological Urban and Regional Development + 'openorgs____::8839b55dae0c84d56fd533f52d5d483a', -- Leibniz Institute of Ecological Urban and Regional Development + 'openorgs____::526468206bca24c1c90da6a312295cf4', -- Cyprus University of Technology + 'openorgs____::b5ca9d4340e26454e367e2908ef3872f' -- Alma Mater Studiorum University of Bologna ))) foo; \ No newline at end of file From 33cb483c756808d9fc2363d5b27fdb5eeb35c355 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 1 Dec 2023 15:03:05 +0100 Subject: [PATCH 42/60] using objectSubType as originalType in Crossref2Oaf, code formatting --- .../dhp/datacite/DataciteToOAFTransformation.scala | 10 ++++++---- .../main/scala/eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala | 1 - .../eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala | 5 ++--- .../eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala | 4 ++-- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala index ee3660918..7603715b9 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/datacite/DataciteToOAFTransformation.scala @@ -176,7 +176,8 @@ object DataciteToOAFTransformation { vocabularies.getSynonymAsQualifier( ModelConstants.DNET_RESULT_TYPOLOGIES, typeQualifier.getClassid - ), resourceType + ), + resourceType ) } if (schemaOrg != null && schemaOrg.nonEmpty) { @@ -188,7 +189,8 @@ object DataciteToOAFTransformation { vocabularies.getSynonymAsQualifier( ModelConstants.DNET_RESULT_TYPOLOGIES, typeQualifier.getClassid - ), schemaOrg + ), + schemaOrg ) } @@ -203,7 +205,8 @@ object DataciteToOAFTransformation { vocabularies.getSynonymAsQualifier( ModelConstants.DNET_RESULT_TYPOLOGIES, typeQualifier.getClassid - ), resourceTypeGeneral + ), + resourceTypeGeneral ) } @@ -228,7 +231,6 @@ object DataciteToOAFTransformation { itm.setVocabularyName(ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1) i.setInstanceTypeMapping(List(itm).asJava) - typeQualifiers._2.getClassname match { case "dataset" => val r = new OafDataset diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala index 89bc8d948..df356548a 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/BioDBToOAF.scala @@ -593,7 +593,6 @@ object BioDBToOAF { itm.setVocabularyName(ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1) i.setInstanceTypeMapping(List(itm).asJava) - i.setCollectedfrom(collectedFromMap("ebi")) d.setInstance(List(i).asJava) i.setDateofacceptance( diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala index f3f8b4f02..d59d73bd0 100644 --- a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/sx/bio/pubmed/PubMedToOaf.scala @@ -195,7 +195,7 @@ object PubMedToOaf { pubmedInstance.setInstanceTypeMapping(List(itm).asJava) } else { val i_type = article.getPublicationTypes.asScala - .map(s => (s.getValue,getVocabularyTerm(ModelConstants.DNET_PUBLICATION_RESOURCE, vocabularies, s.getValue))) + .map(s => (s.getValue, getVocabularyTerm(ModelConstants.DNET_PUBLICATION_RESOURCE, vocabularies, s.getValue))) .find(q => q._2 != null) if (i_type.isDefined) { @@ -205,8 +205,7 @@ object PubMedToOaf { itm.setOriginalType(i_type.get._1) itm.setVocabularyName(ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1) pubmedInstance.setInstanceTypeMapping(List(itm).asJava) - } - else + } else return null } val result = createResult(pubmedInstance.getInstancetype, vocabularies) diff --git a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala index 65e395357..53576fc10 100644 --- a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala @@ -107,7 +107,7 @@ case object Crossref2Oaf { .map(f => f.id) } - def mappingResult(result: Result, json: JValue, cobjCategory: String, originalType:String): Result = { + def mappingResult(result: Result, json: JValue, cobjCategory: String, originalType: String): Result = { implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats //MAPPING Crossref DOI into PID @@ -372,7 +372,7 @@ case object Crossref2Oaf { objectType, mappingCrossrefSubType.getOrElse(objectSubType, "0038 Other literature type") ) - mappingResult(result, json, cOBJCategory, originalType) + mappingResult(result, json, cOBJCategory, objectSubType) if (result == null || result.getId == null) return List() From d33f578e544c23eb8e1a1dd4edfd79b0a42d854a Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 1 Dec 2023 15:14:17 +0100 Subject: [PATCH 43/60] code formatting --- .../eu/dnetlib/dhp/oa/merge/AuthorMerger.java | 511 +++++++++--------- 1 file changed, 256 insertions(+), 255 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java index 0153e52d5..0461c9353 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/oa/merge/AuthorMerger.java @@ -4,318 +4,319 @@ package eu.dnetlib.dhp.oa.merge; import java.text.Normalizer; import java.util.*; import java.util.stream.Collectors; + import org.apache.commons.lang3.StringUtils; + import com.wcohen.ss.JaroWinkler; + import eu.dnetlib.dhp.schema.oaf.Author; import eu.dnetlib.dhp.schema.oaf.StructuredProperty; import eu.dnetlib.pace.model.Person; import scala.Tuple2; - public class AuthorMerger { - private static final Double THRESHOLD = 0.95; + private static final Double THRESHOLD = 0.95; - private AuthorMerger() { - } + private AuthorMerger() { + } - public static List merge(List> authors) { + public static List merge(List> authors) { - authors.sort((o1, o2) -> -Integer.compare(countAuthorsPids(o1), countAuthorsPids(o2))); + authors.sort((o1, o2) -> -Integer.compare(countAuthorsPids(o1), countAuthorsPids(o2))); - List author = new ArrayList<>(); + List author = new ArrayList<>(); - for (List a : authors) { - author = mergeAuthor(author, a); - } + for (List a : authors) { + author = mergeAuthor(author, a); + } - return author; + return author; - } + } - public static List mergeAuthor(final List a, final List b, Double threshold) { - int pa = countAuthorsPids(a); - int pb = countAuthorsPids(b); - List base; - List enrich; - int sa = authorsSize(a); - int sb = authorsSize(b); + public static List mergeAuthor(final List a, final List b, Double threshold) { + int pa = countAuthorsPids(a); + int pb = countAuthorsPids(b); + List base; + List enrich; + int sa = authorsSize(a); + int sb = authorsSize(b); - if (sa == sb) { - base = pa > pb ? a : b; - enrich = pa > pb ? b : a; - } else { - base = sa > sb ? a : b; - enrich = sa > sb ? b : a; - } - enrichPidFromList(base, enrich, threshold); - return base; - } + if (sa == sb) { + base = pa > pb ? a : b; + enrich = pa > pb ? b : a; + } else { + base = sa > sb ? a : b; + enrich = sa > sb ? b : a; + } + enrichPidFromList(base, enrich, threshold); + return base; + } - public static List mergeAuthor(final List a, final List b) { - return mergeAuthor(a, b, THRESHOLD); - } + public static List mergeAuthor(final List a, final List b) { + return mergeAuthor(a, b, THRESHOLD); + } - private static void enrichPidFromList(List base, List enrich, Double threshold) { - if (base == null || enrich == null) - return; + private static void enrichPidFromList(List base, List enrich, Double threshold) { + if (base == null || enrich == null) + return; - // (if an Author has more than 1 pid, it appears 2 times in the list) - final Map basePidAuthorMap = base - .stream() - .filter(a -> a.getPid() != null && !a.getPid().isEmpty()) - .flatMap( - a -> a - .getPid() - .stream() - .filter(Objects::nonNull) - .map(p -> new Tuple2<>(pidToComparableString(p), a))) - .collect(Collectors.toMap(Tuple2::_1, Tuple2::_2, (x1, x2) -> x1)); + // (if an Author has more than 1 pid, it appears 2 times in the list) + final Map basePidAuthorMap = base + .stream() + .filter(a -> a.getPid() != null && !a.getPid().isEmpty()) + .flatMap( + a -> a + .getPid() + .stream() + .filter(Objects::nonNull) + .map(p -> new Tuple2<>(pidToComparableString(p), a))) + .collect(Collectors.toMap(Tuple2::_1, Tuple2::_2, (x1, x2) -> x1)); - // (list of pid that are missing in the other list) - final List> pidToEnrich = enrich - .stream() - .filter(a -> a.getPid() != null && !a.getPid().isEmpty()) - .flatMap( - a -> a - .getPid() - .stream() - .filter(Objects::nonNull) - .filter(p -> !basePidAuthorMap.containsKey(pidToComparableString(p))) - .map(p -> new Tuple2<>(p, a))) - .collect(Collectors.toList()); + // (list of pid that are missing in the other list) + final List> pidToEnrich = enrich + .stream() + .filter(a -> a.getPid() != null && !a.getPid().isEmpty()) + .flatMap( + a -> a + .getPid() + .stream() + .filter(Objects::nonNull) + .filter(p -> !basePidAuthorMap.containsKey(pidToComparableString(p))) + .map(p -> new Tuple2<>(p, a))) + .collect(Collectors.toList()); - pidToEnrich - .forEach( - a -> { - Optional> simAuthor = base - .stream() - .map(ba -> new Tuple2<>(sim(ba, a._2()), ba)) - .max(Comparator.comparing(Tuple2::_1)); + pidToEnrich + .forEach( + a -> { + Optional> simAuthor = base + .stream() + .map(ba -> new Tuple2<>(sim(ba, a._2()), ba)) + .max(Comparator.comparing(Tuple2::_1)); - if (simAuthor.isPresent()) { - double th = threshold; - // increase the threshold if the surname is too short - if (simAuthor.get()._2().getSurname() != null - && simAuthor.get()._2().getSurname().length() <= 3 && threshold > 0.0) - th = 0.99; + if (simAuthor.isPresent()) { + double th = threshold; + // increase the threshold if the surname is too short + if (simAuthor.get()._2().getSurname() != null + && simAuthor.get()._2().getSurname().length() <= 3 && threshold > 0.0) + th = 0.99; - if (simAuthor.get()._1() > th) { - Author r = simAuthor.get()._2(); - if (r.getPid() == null) { - r.setPid(new ArrayList<>()); - } + if (simAuthor.get()._1() > th) { + Author r = simAuthor.get()._2(); + if (r.getPid() == null) { + r.setPid(new ArrayList<>()); + } - // TERRIBLE HACK but for some reason when we create and Array with Arrays.asList, - // it creates of fixed size, and the add method raise UnsupportedOperationException at - // java.util.AbstractList.add - final List tmp = new ArrayList<>(r.getPid()); - tmp.add(a._1()); - r.setPid(tmp); - } - } - }); - } + // TERRIBLE HACK but for some reason when we create and Array with Arrays.asList, + // it creates of fixed size, and the add method raise UnsupportedOperationException at + // java.util.AbstractList.add + final List tmp = new ArrayList<>(r.getPid()); + tmp.add(a._1()); + r.setPid(tmp); + } + } + }); + } - public static String normalizeFullName(final String fullname) { - return nfd(fullname) - .toLowerCase() - // do not compact the regexes in a single expression, would cause StackOverflowError - // in case - // of large input strings - .replaceAll("(\\W)+", " ") - .replaceAll("(\\p{InCombiningDiacriticalMarks})+", " ") - .replaceAll("(\\p{Punct})+", " ") - .replaceAll("(\\d)+", " ") - .replaceAll("(\\n)+", " ") + public static String normalizeFullName(final String fullname) { + return nfd(fullname) + .toLowerCase() + // do not compact the regexes in a single expression, would cause StackOverflowError + // in case + // of large input strings + .replaceAll("(\\W)+", " ") + .replaceAll("(\\p{InCombiningDiacriticalMarks})+", " ") + .replaceAll("(\\p{Punct})+", " ") + .replaceAll("(\\d)+", " ") + .replaceAll("(\\n)+", " ") - .trim(); - } + .trim(); + } + private static String authorFieldToBeCompared(Author author) { + if (StringUtils.isNotBlank(author.getSurname())) { + return author.getSurname(); - private static String authorFieldToBeCompared(Author author) { - if (StringUtils.isNotBlank(author.getSurname())) { - return author.getSurname(); + } + if (StringUtils.isNotBlank(author.getFullname())) { + return author.getFullname(); + } + return null; + } - } - if (StringUtils.isNotBlank(author.getFullname())) { - return author.getFullname(); - } - return null; - } + /** + * This method tries to figure out when two author are the same in the contest + * of ORCID enrichment + * + * @param left Author in the OAF entity + * @param right Author ORCID + * @return based on a heuristic on the names of the authors if they are the same. + */ + public static boolean checkORCIDSimilarity(final Author left, final Author right) { + final Person pl = parse(left); + final Person pr = parse(right); - /** - * This method tries to figure out when two author are the same in the contest - * of ORCID enrichment - * - * @param left Author in the OAF entity - * @param right Author ORCID - * @return based on a heuristic on the names of the authors if they are the same. - */ - public static boolean checkORCIDSimilarity(final Author left, final Author right) { - final Person pl = parse(left); - final Person pr = parse(right); + // If one of them didn't have a surname we verify if they have the fullName not empty + // and verify if the normalized version is equal + if (!(pl.getSurname() != null && pl.getSurname().stream().anyMatch(StringUtils::isNotBlank) && + pr.getSurname() != null && pr.getSurname().stream().anyMatch(StringUtils::isNotBlank))) { - // If one of them didn't have a surname we verify if they have the fullName not empty - // and verify if the normalized version is equal - if (!(pl.getSurname() != null && pl.getSurname().stream().anyMatch(StringUtils::isNotBlank) && - pr.getSurname() != null && pr.getSurname().stream().anyMatch(StringUtils::isNotBlank))) { + if (pl.getFullname() != null && !pl.getFullname().isEmpty() && pr.getFullname() != null + && !pr.getFullname().isEmpty()) { + return pl + .getFullname() + .stream() + .anyMatch( + fl -> pr.getFullname().stream().anyMatch(fr -> normalize(fl).equalsIgnoreCase(normalize(fr)))); + } else { + return false; + } + } + // The Authors have one surname in common + if (pl.getSurname().stream().anyMatch(sl -> pr.getSurname().stream().anyMatch(sr -> sr.equalsIgnoreCase(sl)))) { - if (pl.getFullname() != null && !pl.getFullname().isEmpty() && pr.getFullname() != null - && !pr.getFullname().isEmpty()) { - return pl - .getFullname() - .stream() - .anyMatch( - fl -> pr.getFullname().stream().anyMatch(fr -> normalize(fl).equalsIgnoreCase(normalize(fr)))); - } else { - return false; - } - } - // The Authors have one surname in common - if (pl.getSurname().stream().anyMatch(sl -> pr.getSurname().stream().anyMatch(sr -> sr.equalsIgnoreCase(sl)))) { + // If one of them has only a surname and is the same we can say that they are the same author + if ((pl.getName() == null || pl.getName().stream().allMatch(StringUtils::isBlank)) || + (pr.getName() == null || pr.getName().stream().allMatch(StringUtils::isBlank))) + return true; + // The authors have the same initials of Name in common + if (pl + .getName() + .stream() + .anyMatch( + nl -> pr + .getName() + .stream() + .anyMatch(nr -> nr.equalsIgnoreCase(nl)))) + return true; + } - // If one of them has only a surname and is the same we can say that they are the same author - if ((pl.getName() == null || pl.getName().stream().allMatch(StringUtils::isBlank)) || - (pr.getName() == null || pr.getName().stream().allMatch(StringUtils::isBlank))) - return true; - // The authors have the same initials of Name in common - if (pl - .getName() - .stream() - .anyMatch( - nl -> pr - .getName() - .stream() - .anyMatch(nr -> nr.equalsIgnoreCase(nl)))) - return true; - } + // Sometimes we noticed that publication have author wrote in inverse order Surname, Name + // We verify if we have an exact match between name and surname + if (pl.getSurname().stream().anyMatch(sl -> pr.getName().stream().anyMatch(nr -> nr.equalsIgnoreCase(sl))) && + pl.getName().stream().anyMatch(nl -> pr.getSurname().stream().anyMatch(sr -> sr.equalsIgnoreCase(nl)))) + return true; + else + return false; + } + // - // Sometimes we noticed that publication have author wrote in inverse order Surname, Name - // We verify if we have an exact match between name and surname - if (pl.getSurname().stream().anyMatch(sl -> pr.getName().stream().anyMatch(nr -> nr.equalsIgnoreCase(sl))) && - pl.getName().stream().anyMatch(nl -> pr.getSurname().stream().anyMatch(sr -> sr.equalsIgnoreCase(nl)))) - return true; - else - return false; - } - // + /** + * Method to enrich ORCID information in one list of authors based on another list + * + * @param baseAuthor the Author List in the OAF Entity + * @param orcidAuthor The list of ORCID Author intersected + * @return The Author List of the OAF Entity enriched with the orcid Author + */ + public static List enrichOrcid(List baseAuthor, List orcidAuthor) { - /** - * Method to enrich ORCID information in one list of authors based on another list - * - * @param baseAuthor the Author List in the OAF Entity - * @param orcidAuthor The list of ORCID Author intersected - * @return The Author List of the OAF Entity enriched with the orcid Author - */ - public static List enrichOrcid(List baseAuthor, List orcidAuthor) { + if (baseAuthor == null || baseAuthor.isEmpty()) + return orcidAuthor; - if (baseAuthor == null || baseAuthor.isEmpty()) - return orcidAuthor; + if (orcidAuthor == null || orcidAuthor.isEmpty()) + return baseAuthor; - if (orcidAuthor == null || orcidAuthor.isEmpty()) - return baseAuthor; + if (baseAuthor.size() == 1 && orcidAuthor.size() > 10) + return baseAuthor; - if (baseAuthor.size() == 1 && orcidAuthor.size() > 10) - return baseAuthor; + final List oAuthor = new ArrayList<>(); + oAuthor.addAll(orcidAuthor); - final List oAuthor = new ArrayList<>(); - oAuthor.addAll(orcidAuthor); + baseAuthor.forEach(ba -> { + Optional aMatch = oAuthor.stream().filter(oa -> checkORCIDSimilarity(ba, oa)).findFirst(); + if (aMatch.isPresent()) { + final Author sameAuthor = aMatch.get(); + addPid(ba, sameAuthor.getPid()); + oAuthor.remove(sameAuthor); + } + }); + return baseAuthor; + } - baseAuthor.forEach(ba -> { - Optional aMatch = oAuthor.stream().filter(oa -> checkORCIDSimilarity(ba, oa)).findFirst(); - if (aMatch.isPresent()) { - final Author sameAuthor = aMatch.get(); - addPid(ba, sameAuthor.getPid()); - oAuthor.remove(sameAuthor); - } - }); - return baseAuthor; - } + private static void addPid(final Author a, final List pids) { - private static void addPid(final Author a, final List pids) { + if (a.getPid() == null) { + a.setPid(new ArrayList<>()); + } - if (a.getPid() == null) { - a.setPid(new ArrayList<>()); - } + a.getPid().addAll(pids); - a.getPid().addAll(pids); + } - } + public static String pidToComparableString(StructuredProperty pid) { + final String classid = pid.getQualifier().getClassid() != null ? pid.getQualifier().getClassid().toLowerCase() + : ""; + return (pid.getQualifier() != null ? classid : "") + + (pid.getValue() != null ? pid.getValue().toLowerCase() : ""); + } - public static String pidToComparableString(StructuredProperty pid) { - final String classid = pid.getQualifier().getClassid() != null ? pid.getQualifier().getClassid().toLowerCase() - : ""; - return (pid.getQualifier() != null ? classid : "") - + (pid.getValue() != null ? pid.getValue().toLowerCase() : ""); - } + public static int countAuthorsPids(List authors) { + if (authors == null) + return 0; - public static int countAuthorsPids(List authors) { - if (authors == null) - return 0; + return (int) authors.stream().filter(AuthorMerger::hasPid).count(); + } - return (int) authors.stream().filter(AuthorMerger::hasPid).count(); - } + private static int authorsSize(List authors) { + if (authors == null) + return 0; + return authors.size(); + } - private static int authorsSize(List authors) { - if (authors == null) - return 0; - return authors.size(); - } + private static Double sim(Author a, Author b) { - private static Double sim(Author a, Author b) { + final Person pa = parse(a); + final Person pb = parse(b); - final Person pa = parse(a); - final Person pb = parse(b); + // if both are accurate (e.g. they have name and surname) + if (pa.isAccurate() & pb.isAccurate()) { + return new JaroWinkler().score(normalize(pa.getSurnameString()), normalize(pb.getSurnameString())) * 0.5 + + new JaroWinkler().score(normalize(pa.getNameString()), normalize(pb.getNameString())) * 0.5; + } else { + return new JaroWinkler() + .score(normalize(pa.getNormalisedFullname()), normalize(pb.getNormalisedFullname())); + } + } - // if both are accurate (e.g. they have name and surname) - if (pa.isAccurate() & pb.isAccurate()) { - return new JaroWinkler().score(normalize(pa.getSurnameString()), normalize(pb.getSurnameString())) * 0.5 - + new JaroWinkler().score(normalize(pa.getNameString()), normalize(pb.getNameString())) * 0.5; - } else { - return new JaroWinkler() - .score(normalize(pa.getNormalisedFullname()), normalize(pb.getNormalisedFullname())); - } - } + private static boolean hasPid(Author a) { + if (a == null || a.getPid() == null || a.getPid().isEmpty()) + return false; + return a.getPid().stream().anyMatch(p -> p != null && StringUtils.isNotBlank(p.getValue())); + } - private static boolean hasPid(Author a) { - if (a == null || a.getPid() == null || a.getPid().isEmpty()) - return false; - return a.getPid().stream().anyMatch(p -> p != null && StringUtils.isNotBlank(p.getValue())); - } + private static Person parse(Author author) { + if (StringUtils.isNotBlank(author.getSurname())) { + return new Person(author.getSurname() + ", " + author.getName(), false); + } else { + if (StringUtils.isNotBlank(author.getFullname())) + return new Person(author.getFullname(), false); + else + return new Person("", false); + } + } - private static Person parse(Author author) { - if (StringUtils.isNotBlank(author.getSurname())) { - return new Person(author.getSurname() + ", " + author.getName(), false); - } else { - if (StringUtils.isNotBlank(author.getFullname())) - return new Person(author.getFullname(), false); - else - return new Person("", false); - } - } + public static String normalize(final String s) { + String[] normalized = nfd(s) + .toLowerCase() + // do not compact the regexes in a single expression, would cause StackOverflowError + // in case + // of large input strings + .replaceAll("(\\W)+", " ") + .replaceAll("(\\p{InCombiningDiacriticalMarks})+", " ") + .replaceAll("(\\p{Punct})+", " ") + .replaceAll("(\\d)+", " ") + .replaceAll("(\\n)+", " ") + .trim() + .split(" "); - public static String normalize(final String s) { - String[] normalized = nfd(s) - .toLowerCase() - // do not compact the regexes in a single expression, would cause StackOverflowError - // in case - // of large input strings - .replaceAll("(\\W)+", " ") - .replaceAll("(\\p{InCombiningDiacriticalMarks})+", " ") - .replaceAll("(\\p{Punct})+", " ") - .replaceAll("(\\d)+", " ") - .replaceAll("(\\n)+", " ") - .trim() - .split(" "); + Arrays.sort(normalized); - Arrays.sort(normalized); + return String.join(" ", normalized); + } - return String.join(" ", normalized); - } - - private static String nfd(final String s) { - return Normalizer.normalize(s, Normalizer.Form.NFD); - } + private static String nfd(final String s) { + return Normalizer.normalize(s, Normalizer.Form.NFD); + } } From 511a98dd800d2697134b0c01f36f7b09f2caa8f1 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 1 Dec 2023 16:21:53 +0100 Subject: [PATCH 44/60] fixed doiboost process workflow, removed references to the ProcessORCID step --- .../eu/dnetlib/dhp/doiboost/process/oozie_app/workflow.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/process/oozie_app/workflow.xml b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/process/oozie_app/workflow.xml index 8f28d706d..84cbc5232 100644 --- a/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/process/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-doiboost/src/main/resources/eu/dnetlib/dhp/doiboost/process/oozie_app/workflow.xml @@ -84,7 +84,6 @@ ${wf:conf('resumeFrom') eq 'Skip'} ${wf:conf('resumeFrom') eq 'PreprocessMag'} ${wf:conf('resumeFrom') eq 'PreprocessUW'} - ${wf:conf('resumeFrom') eq 'ProcessORCID'} ${wf:conf('resumeFrom') eq 'CreateDOIBoost'} ${wf:conf('resumeFrom') eq 'GenerateActionSet'} From 3caf6ff27ef91aa6b684fdb91ea4d7e1274f9b3d Mon Sep 17 00:00:00 2001 From: Sandro La Bruzzo Date: Fri, 1 Dec 2023 16:33:56 +0100 Subject: [PATCH 45/60] Extracted the correct original type to pass to instanceTypeMapping in Crossref Mapping --- .../scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala index 53576fc10..91118c636 100644 --- a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala @@ -372,7 +372,9 @@ case object Crossref2Oaf { objectType, mappingCrossrefSubType.getOrElse(objectSubType, "0038 Other literature type") ) - mappingResult(result, json, cOBJCategory, objectSubType) + + val originalType = if (mappingCrossrefSubType.contains(objectType)) objectType else objectSubType + mappingResult(result, json, cOBJCategory, originalType) if (result == null || result.getId == null) return List() From 178a14c491ea24847d02e3eb2295a08198b37d9b Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Sun, 3 Dec 2023 13:31:58 +0100 Subject: [PATCH 46/60] code formatting --- .../main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala index 91118c636..ee857e2c4 100644 --- a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala +++ b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/crossref/Crossref2Oaf.scala @@ -373,7 +373,7 @@ case object Crossref2Oaf { mappingCrossrefSubType.getOrElse(objectSubType, "0038 Other literature type") ) - val originalType = if (mappingCrossrefSubType.contains(objectType)) objectType else objectSubType + val originalType = if (mappingCrossrefSubType.contains(objectType)) objectType else objectSubType mappingResult(result, json, cOBJCategory, originalType) if (result == null || result.getId == null) return List() From 2a233a89aa49c2ec59c95ba86d256ab46eb3db9b Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Sun, 3 Dec 2023 13:32:52 +0100 Subject: [PATCH 47/60] [graph grouping] added isLookupUrl to the workflow definition, passed to the grouping spark aciton --- .../eu/dnetlib/dhp/oa/graph/group/oozie_app/workflow.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/group/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/group/oozie_app/workflow.xml index 190788c9d..2f44bea9d 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/group/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/group/oozie_app/workflow.xml @@ -12,6 +12,10 @@ graphOutputPath path of the output graph + + isLookupUrl + the URL address of the lookUp service + filterInvisible whether filter out invisible entities after merge @@ -98,6 +102,7 @@ --graphInputPath${graphBasePath} --checkpointPath${workingPath}/grouped_entities --outputPath${graphOutputPath} + --isLookupUrl${isLookupUrl} --filterInvisible${filterInvisible} From e6086efc53804cf3e29d29fd6b2f1b8f958f9130 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Sun, 3 Dec 2023 13:33:20 +0100 Subject: [PATCH 48/60] avoid NPEs in Vocabulary.getTermBySynonym --- .../java/eu/dnetlib/dhp/common/vocabulary/Vocabulary.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/common/vocabulary/Vocabulary.java b/dhp-common/src/main/java/eu/dnetlib/dhp/common/vocabulary/Vocabulary.java index 2ab23bda6..94e7f5221 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/common/vocabulary/Vocabulary.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/common/vocabulary/Vocabulary.java @@ -63,7 +63,10 @@ public class Vocabulary implements Serializable { } public VocabularyTerm getTermBySynonym(final String syn) { - return getTerm(synonyms.get(syn.toLowerCase())); + return Optional + .ofNullable(syn) + .map(s -> getTerm(synonyms.get(s.toLowerCase()))) + .orElse(null); } public Qualifier getTermAsQualifier(final String termId) { From 74b185d07b3655f6e964ce5ae0610695d1685903 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Sun, 3 Dec 2023 16:18:20 +0100 Subject: [PATCH 49/60] avoid NPEs --- .../dhp/schema/oaf/utils/GraphCleaningFunctions.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java index 7cb32eab5..c65f36de9 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java @@ -943,17 +943,17 @@ public class GraphCleaningFunctions extends CleaningFunctions { private static Qualifier getMetaResourceType(final List instances, final VocabularyGroup vocs) { if (vocs.vocabularyExists(OPENAIRE_META_RESOURCE_TYPE)) { - Optional instanceTypeMapping = instances + Optional itm = instances .stream() .flatMap( i -> Optional.ofNullable(i.getInstanceTypeMapping()).map(Collection::stream).orElse(Stream.empty())) .filter(t -> OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(t.getVocabularyName())) .findFirst(); - if (!instanceTypeMapping.isPresent()) { + if (!itm.isPresent() || Objects.isNull(itm.get().getTypeCode())) { return null; } else { - final String typeCode = instanceTypeMapping.get().getTypeCode(); + final String typeCode = itm.get().getTypeCode(); return Optional .ofNullable(vocs.lookupTermBySynonym(OPENAIRE_META_RESOURCE_TYPE, typeCode)) .orElseThrow( From 7c3041b276c2c27f235f68e3ebdc4ae3b862dd5b Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Sun, 3 Dec 2023 16:49:49 +0100 Subject: [PATCH 50/60] avoid NPEs --- .../oaf/utils/GraphCleaningFunctions.java | 50 +++++++++++-------- 1 file changed, 29 insertions(+), 21 deletions(-) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java index c65f36de9..c25d67f25 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/GraphCleaningFunctions.java @@ -941,28 +941,36 @@ public class GraphCleaningFunctions extends CleaningFunctions { } private static Qualifier getMetaResourceType(final List instances, final VocabularyGroup vocs) { + return Optional + .ofNullable(instances) + .map(ii -> { + if (vocs.vocabularyExists(OPENAIRE_META_RESOURCE_TYPE)) { + Optional itm = ii + .stream() + .filter(Objects::nonNull) + .flatMap( + i -> Optional + .ofNullable(i.getInstanceTypeMapping()) + .map(Collection::stream) + .orElse(Stream.empty())) + .filter(t -> OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(t.getVocabularyName())) + .findFirst(); - if (vocs.vocabularyExists(OPENAIRE_META_RESOURCE_TYPE)) { - Optional itm = instances - .stream() - .flatMap( - i -> Optional.ofNullable(i.getInstanceTypeMapping()).map(Collection::stream).orElse(Stream.empty())) - .filter(t -> OPENAIRE_COAR_RESOURCE_TYPES_3_1.equals(t.getVocabularyName())) - .findFirst(); - - if (!itm.isPresent() || Objects.isNull(itm.get().getTypeCode())) { - return null; - } else { - final String typeCode = itm.get().getTypeCode(); - return Optional - .ofNullable(vocs.lookupTermBySynonym(OPENAIRE_META_RESOURCE_TYPE, typeCode)) - .orElseThrow( - () -> new IllegalStateException("unable to find a synonym for '" + typeCode + "' in " + - OPENAIRE_META_RESOURCE_TYPE)); - } - } else { - throw new IllegalStateException("vocabulary '" + OPENAIRE_META_RESOURCE_TYPE + "' not available"); - } + if (!itm.isPresent() || Objects.isNull(itm.get().getTypeCode())) { + return null; + } else { + final String typeCode = itm.get().getTypeCode(); + return Optional + .ofNullable(vocs.lookupTermBySynonym(OPENAIRE_META_RESOURCE_TYPE, typeCode)) + .orElseThrow( + () -> new IllegalStateException("unable to find a synonym for '" + typeCode + "' in " + + OPENAIRE_META_RESOURCE_TYPE)); + } + } else { + throw new IllegalStateException("vocabulary '" + OPENAIRE_META_RESOURCE_TYPE + "' not available"); + } + }) + .orElse(null); } } From 3c3bdb83187d572c101447def9e5f75febe8efa0 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 5 Dec 2023 09:08:48 +0100 Subject: [PATCH 51/60] [bulktagging] fixed workflow parameters --- .../dnetlib/dhp/bulktag/SparkBulkTagJob.java | 2 +- .../dhp/bulktag/input_bulkTag_parameters.json | 5 ----- .../dhp/bulktag/oozie_app/workflow.xml | 21 ++++--------------- 3 files changed, 5 insertions(+), 23 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java index f01063955..5d1b2b38d 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/bulktag/SparkBulkTagJob.java @@ -135,7 +135,7 @@ public class SparkBulkTagJob { ModelSupport.entityTypes .keySet() .parallelStream() - .filter(e -> ModelSupport.isResult(e)) + .filter(ModelSupport::isResult) .forEach(e -> { removeOutputDir(spark, outputPath + e.name()); ResultTagger resultTagger = new ResultTagger(); diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json index d88904cd9..ce1a8ecab 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/input_bulkTag_parameters.json @@ -1,5 +1,4 @@ [ - { "paramName":"s", "paramLongName":"sourcePath", @@ -12,7 +11,6 @@ "paramDescription": "the json path associated to each selection field", "paramRequired": true }, - { "paramName": "out", "paramLongName": "outputPath", @@ -25,19 +23,16 @@ "paramDescription": "true if the spark session is managed, false otherwise", "paramRequired": false }, - { "paramName": "tg", "paramLongName": "taggingConf", "paramDescription": "this parameter is intended for testing purposes only. It is a possible tagging configuration obtained via the XQUERY. Intended to be removed", "paramRequired": false }, - { "paramName": "bu", "paramLongName": "baseURL", "paramDescription": "this parameter is to specify the api to be queried (beta or production)", "paramRequired": false } - ] \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml index 56eed4703..0d4d1f046 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml @@ -12,21 +12,10 @@ outputPath the output path - - - postgresURL - the url of the postgress server to query + baseURL + the community API base URL - - postgresUser - the username to access the postgres db - - - postgresPassword - the postgres password - - @@ -104,7 +93,7 @@ yarn-cluster cluster - bulkTagging-publication + bulkTagging-result eu.dnetlib.dhp.bulktag.SparkBulkTagJob dhp-enrichment-${projectVersion}.jar @@ -120,14 +109,12 @@ --sourcePath${sourcePath}/ --outputPath${outputPath}/ --pathMap${pathMap} - --production${production} + --baseURL${baseURL} - - \ No newline at end of file From c5b72531301c4317ebc63261e1148b10c1c6f240 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 5 Dec 2023 09:13:33 +0100 Subject: [PATCH 52/60] [community_organization propagation] fixed workflow parameters --- .../oozie_app/workflow.xml | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml index d5d75fbc1..dfa762ac6 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml @@ -4,11 +4,14 @@ sourcePath the source path - outputPath the output path + + baseURL + the community API base URL + @@ -99,13 +102,12 @@ --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} --sourcePath${sourcePath}/relation --outputPath${workingDir}/preparedInfo/resultCommunityList --hive_metastore_uris${hive_metastore_uris} - --production${production} + --baseURL${baseURL} @@ -115,7 +117,7 @@ yarn cluster - community2resultfromorganization-Publication + community2resultfromorganization eu.dnetlib.dhp.resulttocommunityfromorganization.SparkResultToCommunityFromOrganizationJob dhp-enrichment-${projectVersion}.jar From 321922772b71c70e074eb97efb46d920f1031752 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Tue, 5 Dec 2023 16:37:04 +0100 Subject: [PATCH 53/60] added serialization for the new fields imported for the Irish tender --- .../oa/provision/utils/XmlRecordFactory.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java index 3b810ba8c..4d9d9c341 100644 --- a/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java +++ b/dhp-workflows/dhp-graph-provision/src/main/java/eu/dnetlib/dhp/oa/provision/utils/XmlRecordFactory.java @@ -434,6 +434,24 @@ public class XmlRecordFactory implements Serializable { XmlSerializationUtils .asXmlElement("processingchargecurrency", r.getProcessingchargecurrency().getValue())); } + if (r.getIsGreen() != null) { + metadata + .add(XmlSerializationUtils.asXmlElement("isgreen", Boolean.toString(r.getIsGreen()))); + } + if (r.getOpenAccessColor() != null) { + metadata + .add(XmlSerializationUtils.asXmlElement("openaccesscolor", r.getOpenAccessColor().toString())); + } + if (r.getIsInDiamondJournal() != null) { + metadata + .add( + XmlSerializationUtils + .asXmlElement("isindiamondjournal", Boolean.toString(r.getIsInDiamondJournal()))); + } + if (r.getPubliclyFunded() != null) { + metadata + .add(XmlSerializationUtils.asXmlElement("publiclyfunded", Boolean.toString(r.getPubliclyFunded()))); + } } switch (type) { From 431c6bb08a2713df4cb1dbe0abeb60011c409349 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 6 Dec 2023 11:06:46 +0100 Subject: [PATCH 54/60] [dedup] added isLookupUrl to the graph consistency workflow definition, required now by the entity grouping phase --- .../dnetlib/dhp/oa/dedup/consistency/oozie_app/workflow.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/consistency/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/consistency/oozie_app/workflow.xml index 5e2fc0a01..306229e79 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/consistency/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/consistency/oozie_app/workflow.xml @@ -16,6 +16,10 @@ filterInvisible whether filter out invisible entities after merge + + isLookupUrl + the URL address of the lookUp service + sparkDriverMemory heap memory for driver process @@ -128,6 +132,7 @@ --graphInputPath${graphBasePath} --checkpointPath${workingPath}/grouped_entities --outputPath${graphOutputPath} + --isLookupUrl${isLookupUrl} --filterInvisible${filterInvisible} From 259c69e446c855a3f3da8cfec532f2ac0e2ff87e Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 6 Dec 2023 19:41:53 +0100 Subject: [PATCH 55/60] [orcid enrichment] fixed workflow definition --- .../dhp/enrich/orcid/oozie_app/workflow.xml | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml index a4330b966..ce117b5e9 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/enrich/orcid/oozie_app/workflow.xml @@ -38,16 +38,12 @@ --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --orcidPath - ${orcidPath} - --targetPath - ${targetPath} - --graphPath - ${graphPath} - --master - yarn + --orcidPath${orcidPath} + --targetPath${targetPath} + --graphPath${graphPath} + --masteryarn - + From c0cde53bf647d00bf5d633ef0ef906b4984d35b5 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 7 Dec 2023 10:08:35 +0100 Subject: [PATCH 56/60] [bulktagging] setting first step of bulktaggin as the copy of the entities and relations not involved in the tagging' --- .../resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml index 0d4d1f046..03373eda0 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/bulktag/oozie_app/workflow.xml @@ -29,7 +29,7 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] From 336fb31d8778cbfbb75a555165b55ee48518a9ba Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 7 Dec 2023 10:27:25 +0100 Subject: [PATCH 57/60] [community_result_propagation] adjusting starting poit of workflow --- .../resulttocommunityfromorganization/oozie_app/workflow.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml index dfa762ac6..e342bce23 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromorganization/oozie_app/workflow.xml @@ -25,7 +25,7 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] From c381bacee01b7f5a3918e0e77d4a55d8e969c04b Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 7 Dec 2023 14:07:11 +0100 Subject: [PATCH 58/60] [enrichment] passing the community API base URL --- .../resulttocommunityfromproject/oozie_app/workflow.xml | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/oozie_app/workflow.xml index 21cc2d887..d0784c94d 100644 --- a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/resulttocommunityfromproject/oozie_app/workflow.xml @@ -4,11 +4,14 @@ sourcePath the source path - outputPath the output path + + baseURL + the community API base URL + @@ -104,7 +107,7 @@ --sourcePath${sourcePath}/relation --outputPath${workingDir}/preparedInfo/resultCommunityList - --production${production} + --baseURL${baseURL} From 70eb1796b20c1001d4c49655e72dbbe0c791252a Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 7 Dec 2023 14:08:04 +0100 Subject: [PATCH 59/60] logging typo --- .../resulttocommunityfromproject/PrepareResultCommunitySet.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java index 7fed2606b..467e11a96 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromproject/PrepareResultCommunitySet.java @@ -53,7 +53,7 @@ public class PrepareResultCommunitySet { log.info("outputPath: {}", outputPath); final String baseURL = parser.get("baseURL"); - log.info("baseUEL: {}", baseURL); + log.info("baseURL: {}", baseURL); final CommunityEntityMap projectsMap = Utils.getCommunityProjects(baseURL); From cb71a7936bb98f498693fe75335c60947f5cbef6 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 7 Dec 2023 23:09:54 +0100 Subject: [PATCH 60/60] [graph cleaning] avoid stack overflow error when navigating Oaf objects declaring an Enum --- .../dhp/oa/graph/clean/OafCleaner.java | 1 + .../graph/clean/CleanGraphSparkJobTest.java | 32 +++++++++++++++++++ .../dhp/oa/graph/clean/graph/orp/orp.json | 1 + 3 files changed, 34 insertions(+) create mode 100644 dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/graph/orp/orp.json diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/OafCleaner.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/OafCleaner.java index 102a1fa85..4cfec6fe7 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/OafCleaner.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/clean/OafCleaner.java @@ -59,6 +59,7 @@ public class OafCleaner implements Serializable { private static boolean isPrimitive(Object o) { return Objects.isNull(o) || o.getClass().isPrimitive() + || o.getClass().isEnum() || o instanceof Class || o instanceof Integer || o instanceof Double diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/clean/CleanGraphSparkJobTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/clean/CleanGraphSparkJobTest.java index a1a4c29cd..4ae3f82c2 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/clean/CleanGraphSparkJobTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/clean/CleanGraphSparkJobTest.java @@ -690,6 +690,38 @@ public class CleanGraphSparkJobTest { } + @Test + void testClean_ORP() throws Exception { + final String prefix = "gcube "; + + new CleanGraphSparkJob( + args( + "/eu/dnetlib/dhp/oa/graph/input_clean_graph_parameters.json", + new String[] { + "--inputPath", graphInputPath + "/orp", + "--outputPath", graphOutputPath + "/orp", + "--isLookupUrl", "lookupurl", + "--graphTableClassName", OtherResearchProduct.class.getCanonicalName(), + "--deepClean", "true", + "--contextId", "sobigdata", + "--verifyParam", "gCube ", + "--masterDuplicatePath", dsMasterDuplicatePath, + "--country", "NL", + "--verifyCountryParam", "10.17632", + "--collectedfrom", "NARCIS", + "--hostedBy", Objects + .requireNonNull( + getClass() + .getResource("/eu/dnetlib/dhp/oa/graph/clean/hostedBy")) + .getPath() + })).run(false, isLookUpService); + + Dataset orp = read(spark, graphOutputPath + "/orp", OtherResearchProduct.class); + + assertEquals(1, orp.count()); + + } + @Test void testCleanCfHbSparkJob() throws Exception { diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/graph/orp/orp.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/graph/orp/orp.json new file mode 100644 index 000000000..85045e361 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/clean/graph/orp/orp.json @@ -0,0 +1 @@ +{"collectedfrom":[{"key":"10|openaire____::fffd45256148b1de3114788f21179083","value":"JAIRO","dataInfo":null}],"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.8","inferenceprovenance":"dedup-result-decisiontree-v3","provenanceaction":{"classid":"sysimport:dedup","classname":"sysimport:dedup","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":1701811731058,"id":"50|dedup_wf_001::5ff833ef4a38d08e840935d5c52fc479","originalId":["oai:irdb.nii.ac.jp:01292:0003046643","50|jairo_______::5ff833ef4a38d08e840935d5c52fc479","oai:irdb.nii.ac.jp:0108/00008334","50|jairo_______::f542b980618630c53c3f8052588dca25"],"pid":[],"dateofcollection":"2023-07-12T02:35:49.772Z","dateoftransformation":"2023-07-12T13:51:55.356Z","extraInfo":[],"oaiprovenance":{"originDescription":{"harvestDate":"2023-07-12T02:35:49.772Z","altered":true,"baseURL":"https%3A%2F%2Firdb.nii.ac.jp%2Foai","identifier":"oai:irdb.nii.ac.jp:01292:0003046643","datestamp":"2023-07-08T10:54:57Z","metadataNamespace":""}},"measures":[{"id":"influence","unit":[{"key":"score","value":"3.244804E-9","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"key":"class","value":"C5","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]},{"id":"popularity","unit":[{"key":"score","value":"6.0055183E-10","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"key":"class","value":"C5","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]},{"id":"influence_alt","unit":[{"key":"score","value":"0","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"key":"class","value":"C5","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]},{"id":"popularity_alt","unit":[{"key":"score","value":"0.0","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"key":"class","value":"C5","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]},{"id":"impulse","unit":[{"key":"score","value":"0","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"key":"class","value":"C5","dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"measure:bip","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]}],"processingchargeamount":null,"processingchargecurrency":null,"author":[{"fullname":"Shimazaki, Kazushi","name":"Kazushi","surname":"Shimazaki","rank":2,"pid":[],"affiliation":null}],"resulttype":{"classid":"other","classname":"other","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"metaResourceType":null,"language":{"classid":"jpn","classname":"Japanese","schemeid":"dnet:languages","schemename":"dnet:languages"},"country":[],"subject":[{"value":"Reinforced concrete structure","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"damage level","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"柱","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"shear span ratio","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"損傷レベル","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"column","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"鉄筋コンクリート構造","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"残存軸耐力","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Residual Axis Load Capacity","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"せん断スパン比","qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":[{"value":"Eximental study odual axis loadity of reinforced concrete column","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"RC造柱の地震後の残存軸耐力に関する研究","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"relevantdate":[],"description":[{"value":"Shear failure of columns may lead to collapse for existence buildings.After a severe earthquake,some buildings collapsed by loss of axial capacity of the columns,however, some buildings have stayed even many columns were fractured by shear.For the examinaton of building collapse,it is required to investigate the axial load capacity of the columns after shear failure and to analyze the building performance using the pertinent analytical model for the columns.This paper investigates the analytical model for the collapse analysis of a building based on Yoshimura’s concept of the failure surface contraction. The reduction rate was obtained as a function of the maximum deformation,shear margin ratio,and so on,using the regression analydid of experimental results. The tested results are explained by this formula with good agreement.","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"Article","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"研究報告書ページ(113)-(118)","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"学術フロンティア研究プロジェクト(2005年度~2009年度) 2008年度 研究成果報告書","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofacceptance":{"value":"2008-03-31","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"publisher":{"value":"神奈川大学","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"embargoenddate":null,"source":[],"fulltext":[{"value":"https://kanagawa-u.repo.nii.ac.jp/record/4315/files/2008-28.pdf","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"http://klibredb.lib.kanagawa-u.ac.jp/dspace/bitstream/10487/10662/1/2008-28.pdf","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"format":[{"value":"application/pdf","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"contributor":[],"resourcetype":{"classid":"UNKNOWN","classname":"Unknown","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"coverage":[],"bestaccessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"context":[],"externalReference":[],"instance":[{"license":null,"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes","openAccessRoute":null},"instancetype":{"classid":"0000","classname":"Unknown","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"instanceTypeMapping":[{"originalType":"Research Paper","typeCode":null,"typeLabel":null,"vocabularyName":"openaire::coar_resource_types_3_1"}],"hostedby":{"key":"10|openaire____::fffd45256148b1de3114788f21179083","value":"JAIRO","dataInfo":null},"url":["https://kanagawa-u.repo.nii.ac.jp/records/4315"],"distributionlocation":"","collectedfrom":{"key":"10|openaire____::fffd45256148b1de3114788f21179083","value":"JAIRO","dataInfo":null},"pid":[],"alternateIdentifier":[],"dateofacceptance":{"value":"2008-03-31","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":{"classid":"0002","classname":"nonPeerReviewed","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"measures":null,"fulltext":"https://kanagawa-u.repo.nii.ac.jp/record/4315/files/2008-28.pdf"},{"license":null,"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes","openAccessRoute":null},"instancetype":{"classid":"0000","classname":"Unknown","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"instanceTypeMapping":[{"originalType":"Article","typeCode":"http://purl.org/coar/resource_type/c_2df8fbb1","typeLabel":"research article","vocabularyName":"openaire::coar_resource_types_3_1"},{"originalType":"http://purl.org/coar/resource_type/c_2df8fbb1","typeCode":"Article","typeLabel":"Article","vocabularyName":"openaire::user_resource_types"}],"hostedby":{"key":"10|openaire____::fffd45256148b1de3114788f21179083","value":"JAIRO","dataInfo":null},"url":["http://hdl.handle.net/10487/10662"],"distributionlocation":"","collectedfrom":{"key":"10|openaire____::fffd45256148b1de3114788f21179083","value":"JAIRO","dataInfo":null},"pid":[],"alternateIdentifier":[],"dateofacceptance":{"value":"2008-03-31","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":"","provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},"processingchargeamount":null,"processingchargecurrency":null,"refereed":{"classid":"0002","classname":"nonPeerReviewed","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"measures":null,"fulltext":"http://klibredb.lib.kanagawa-u.ac.jp/dspace/bitstream/10487/10662/1/2008-28.pdf"}],"eoscifguidelines":[],"openAccessColor":"bronze","publiclyFunded":false,"contactperson":[],"contactgroup":[],"tool":[],"isGreen":true,"isInDiamondJournal":false} \ No newline at end of file