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 0461c9353..b413a0bb9 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 @@ -145,105 +145,6 @@ public class AuthorMerger { 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); - - // 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 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; - } - // - - /** - * 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 (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 -> 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) { - - 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() : ""; 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 0225a5063..eadca231e 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 @@ -1,24 +1,6 @@ package eu.dnetlib.dhp.oa.merge; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; -import static org.apache.spark.sql.functions.col; -import static org.apache.spark.sql.functions.when; - -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ForkJoinPool; -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.api.java.function.ReduceFunction; -import org.apache.spark.sql.*; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.common.HdfsSupport; import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; @@ -26,169 +8,186 @@ 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.schema.oaf.utils.MergeUtils; import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.ReduceFunction; +import org.apache.spark.sql.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import scala.Tuple2; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ForkJoinPool; +import java.util.stream.Collectors; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.when; + /** * Groups the graph content by entity identifier to ensure ID uniqueness */ public class GroupEntitiesSparkJob { - private static final Logger log = LoggerFactory.getLogger(GroupEntitiesSparkJob.class); + private static final Logger log = LoggerFactory.getLogger(GroupEntitiesSparkJob.class); - private static final Encoder OAFENTITY_KRYO_ENC = Encoders.kryo(OafEntity.class); + private static final Encoder OAFENTITY_KRYO_ENC = Encoders.kryo(OafEntity.class); - private ArgumentApplicationParser parser; + private ArgumentApplicationParser parser; - public GroupEntitiesSparkJob(ArgumentApplicationParser parser) { - this.parser = parser; - } + public GroupEntitiesSparkJob(ArgumentApplicationParser parser) { + this.parser = parser; + } - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils - .toString( - GroupEntitiesSparkJob.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/merge/group_graph_entities_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + String jsonConfiguration = IOUtils + .toString( + GroupEntitiesSparkJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/merge/group_graph_entities_parameters.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); + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - final String isLookupUrl = parser.get("isLookupUrl"); - log.info("isLookupUrl: {}", isLookupUrl); + final String isLookupUrl = parser.get("isLookupUrl"); + log.info("isLookupUrl: {}", isLookupUrl); - final ISLookUpService isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl); + final ISLookUpService isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl); - new GroupEntitiesSparkJob(parser).run(isSparkSessionManaged, isLookupService); - } + new GroupEntitiesSparkJob(parser).run(isSparkSessionManaged, isLookupService); + } - public void run(Boolean isSparkSessionManaged, ISLookUpService isLookUpService) - throws ISLookUpException { + public void run(Boolean isSparkSessionManaged, ISLookUpService isLookUpService) + throws ISLookUpException { - String graphInputPath = parser.get("graphInputPath"); - log.info("graphInputPath: {}", graphInputPath); + String graphInputPath = parser.get("graphInputPath"); + log.info("graphInputPath: {}", graphInputPath); - String checkpointPath = parser.get("checkpointPath"); - log.info("checkpointPath: {}", checkpointPath); + String checkpointPath = parser.get("checkpointPath"); + log.info("checkpointPath: {}", checkpointPath); - String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - boolean filterInvisible = Boolean.parseBoolean(parser.get("filterInvisible")); - log.info("filterInvisible: {}", 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()); + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); - final VocabularyGroup vocs = VocabularyGroup.loadVocsFromIS(isLookUpService); + final VocabularyGroup vocs = VocabularyGroup.loadVocsFromIS(isLookUpService); - runWithSparkSession( - conf, - isSparkSessionManaged, - spark -> { - HdfsSupport.remove(checkpointPath, spark.sparkContext().hadoopConfiguration()); - groupEntities(spark, graphInputPath, checkpointPath, outputPath, filterInvisible, vocs); - }); - } + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + HdfsSupport.remove(checkpointPath, spark.sparkContext().hadoopConfiguration()); + groupEntities(spark, graphInputPath, checkpointPath, outputPath, filterInvisible, vocs); + }); + } - private static void groupEntities( - SparkSession spark, - String inputPath, - String checkpointPath, - String outputPath, - boolean filterInvisible, VocabularyGroup vocs) { + private static void groupEntities( + SparkSession spark, + String inputPath, + String checkpointPath, + String outputPath, + boolean filterInvisible, VocabularyGroup vocs) { - Dataset allEntities = spark.emptyDataset(OAFENTITY_KRYO_ENC); + Dataset allEntities = spark.emptyDataset(OAFENTITY_KRYO_ENC); - for (Map.Entry e : ModelSupport.entityTypes.entrySet()) { - String entity = e.getKey().name(); - Class entityClass = e.getValue(); - String entityInputPath = inputPath + "/" + entity; + for (Map.Entry e : ModelSupport.entityTypes.entrySet()) { + String entity = e.getKey().name(); + Class entityClass = e.getValue(); + String entityInputPath = inputPath + "/" + entity; - if (!HdfsSupport.exists(entityInputPath, spark.sparkContext().hadoopConfiguration())) { - continue; - } + if (!HdfsSupport.exists(entityInputPath, spark.sparkContext().hadoopConfiguration())) { + continue; + } - allEntities = allEntities - .union( - ((Dataset) spark - .read() - .schema(Encoders.bean(entityClass).schema()) - .json(entityInputPath) - .filter("length(id) > 0") - .as(Encoders.bean(entityClass))) - .map((MapFunction) r -> r, OAFENTITY_KRYO_ENC)); - } + allEntities = allEntities + .union( + ((Dataset) spark + .read() + .schema(Encoders.bean(entityClass).schema()) + .json(entityInputPath) + .filter("length(id) > 0") + .as(Encoders.bean(entityClass))) + .map((MapFunction) r -> r, OAFENTITY_KRYO_ENC)); + } - Dataset groupedEntities = allEntities - .map( - (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)); + Dataset groupedEntities = allEntities + .map( + (MapFunction) entity -> GraphCleaningFunctions + .applyCoarVocabularies(entity, vocs), + OAFENTITY_KRYO_ENC) + .groupByKey((MapFunction) OafEntity::getId, Encoders.STRING()) + .reduceGroups((ReduceFunction) MergeUtils::checkedMerge) + .map( + (MapFunction, Tuple2>) t -> new Tuple2<>( + t._2().getClass().getName(), t._2()), + Encoders.tuple(Encoders.STRING(), OAFENTITY_KRYO_ENC)); - // pivot on "_1" (classname of the entity) - // created columns containing only entities of the same class - for (Map.Entry e : ModelSupport.entityTypes.entrySet()) { - String entity = e.getKey().name(); - Class entityClass = e.getValue(); + // pivot on "_1" (classname of the entity) + // created columns containing only entities of the same class + for (Map.Entry e : ModelSupport.entityTypes.entrySet()) { + String entity = e.getKey().name(); + Class entityClass = e.getValue(); - groupedEntities = groupedEntities - .withColumn( - entity, - when(col("_1").equalTo(entityClass.getName()), col("_2"))); - } + groupedEntities = groupedEntities + .withColumn( + entity, + when(col("_1").equalTo(entityClass.getName()), col("_2"))); + } - groupedEntities - .drop("_1", "_2") - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .save(checkpointPath); + groupedEntities + .drop("_1", "_2") + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .save(checkpointPath); - ForkJoinPool parPool = new ForkJoinPool(ModelSupport.entityTypes.size()); + ForkJoinPool parPool = new ForkJoinPool(ModelSupport.entityTypes.size()); - ModelSupport.entityTypes - .entrySet() - .stream() - .map(e -> parPool.submit(() -> { - String entity = e.getKey().name(); - Class entityClass = e.getValue(); + ModelSupport.entityTypes + .entrySet() + .stream() + .map(e -> parPool.submit(() -> { + String entity = e.getKey().name(); + Class entityClass = e.getValue(); - spark - .read() - .load(checkpointPath) - .select(col(entity).as("value")) - .filter("value IS NOT NULL") - .as(OAFENTITY_KRYO_ENC) - .map((MapFunction) r -> r, (Encoder) Encoders.bean(entityClass)) - .filter(filterInvisible ? "dataInfo.invisible != TRUE" : "TRUE") - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .json(outputPath + "/" + entity); - })) - .collect(Collectors.toList()) - .forEach(t -> { - try { - t.get(); - } catch (InterruptedException | ExecutionException e) { - throw new RuntimeException(e); - } - }); - } + spark + .read() + .load(checkpointPath) + .select(col(entity).as("value")) + .filter("value IS NOT NULL") + .as(OAFENTITY_KRYO_ENC) + .map((MapFunction) r -> r, (Encoder) Encoders.bean(entityClass)) + .filter(filterInvisible ? "dataInfo.invisible != TRUE" : "TRUE") + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath + "/" + entity); + })) + .collect(Collectors.toList()) + .forEach(t -> { + try { + t.get(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + }); + } } diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeComparator.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeComparator.java new file mode 100644 index 000000000..ffb0e600a --- /dev/null +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeComparator.java @@ -0,0 +1,79 @@ +package eu.dnetlib.dhp.schema.oaf.utils; + +// +// Source code recreated from a .class file by IntelliJ IDEA +// (powered by FernFlower decompiler) +// + + +import eu.dnetlib.dhp.schema.common.EntityType; +import eu.dnetlib.dhp.schema.oaf.KeyValue; +import eu.dnetlib.dhp.schema.oaf.Oaf; +import eu.dnetlib.dhp.schema.oaf.OafEntity; +import eu.dnetlib.dhp.schema.oaf.Result; + +import java.util.Comparator; +import java.util.HashSet; +import java.util.Optional; +import java.util.stream.Collectors; + +public class MergeComparator implements Comparator { + public MergeComparator() { + } + + public int compare(Oaf left, Oaf right) { + // nulls at the end + if (left == null && right == null) { + return 0; + } else if (left == null) { + return -1; + } else if (right == null) { + return 1; + } + + // invisible + if (left.getDataInfo() != null && left.getDataInfo().getInvisible() == true) { + if (right.getDataInfo() != null && right.getDataInfo().getInvisible() == false) { + return -1; + } + } + + // collectedfrom + HashSet lCf = getCollectedFromIds(left); + HashSet rCf = getCollectedFromIds(right); + if (lCf.contains("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2") && !rCf.contains("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2")) { + return -1; + } else if (!lCf.contains("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2") && rCf.contains("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2")) { + return 1; + } + + + + SubEntityType lClass = SubEntityType.fromClass(left.getClass()); + SubEntityType rClass = SubEntityType.fromClass(right.getClass()); + return lClass.ordinal() - rClass.ordinal(); + + } + + protected HashSet getCollectedFromIds(Oaf left) { + return (HashSet) Optional.ofNullable(left.getCollectedfrom()).map((cf) -> { + return (HashSet) cf.stream().map(KeyValue::getKey).collect(Collectors.toCollection(HashSet::new)); + }).orElse(new HashSet()); + } + + enum SubEntityType { + publication, dataset, software, otherresearchproduct, datasource, organization, project; + + /** + * Resolves the EntityType, given the relative class name + * + * @param clazz the given class name + * @param actual OafEntity subclass + * @return the EntityType associated to the given class + */ + public static SubEntityType fromClass(Class clazz) { + return valueOf(clazz.getSimpleName().toLowerCase()); + } + } + +} diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java new file mode 100644 index 000000000..072a86bf1 --- /dev/null +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtils.java @@ -0,0 +1,707 @@ + +package eu.dnetlib.dhp.schema.oaf.utils; + +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.*; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; + +import java.text.ParseException; +import java.util.*; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static com.google.common.base.Objects.firstNonNull; +import static com.google.common.base.Preconditions.checkArgument; + +public class MergeUtils { + + public static T checkedMerge(final T left, final T right) { + return (T) merge(left, right, false); + } + + public static Oaf merge(final Oaf left, final Oaf right) { + return merge(left, right, false); + } + + public static Oaf merge(final Oaf left, final Oaf right, boolean checkDelegatedAuthority) { + if (sameClass(left, right, OafEntity.class)) { + return mergeEntities(left, right, checkDelegatedAuthority); + } else if (sameClass(left, right, Relation.class)) { + return mergeRelation((Relation) left, (Relation) right); + } else { + throw new RuntimeException( + String + .format( + "MERGE_FROM_AND_GET incompatible types: %s, %s", + left.getClass().getCanonicalName(), right.getClass().getCanonicalName())); + } + } + + private static boolean sameClass(Object left, Object right, Class cls) { + return cls.isAssignableFrom(left.getClass()) && cls.isAssignableFrom(right.getClass()); + } + + private static Oaf mergeEntities(Oaf left, Oaf right, boolean checkDelegatedAuthority) { + + if (sameClass(left, right, Result.class)) { + if (!left.getClass().equals(right.getClass()) || checkDelegatedAuthority) { + return mergeResultsOfDifferentTypes((Result)left, (Result) right); + } + + if (sameClass(left, right, Publication.class)) { + return mergePublication((Publication) left, (Publication) right); + } + if (sameClass(left, right, Dataset.class)) { + return mergeDataset((Dataset) left, (Dataset) right); + } + if (sameClass(left, right, OtherResearchProduct.class)) { + return mergeORP((OtherResearchProduct) left, (OtherResearchProduct) right); + } + if (sameClass(left, right, Software.class)) { + return mergeSoftware((Software) left, (Software) right); + } + + return mergeResult((Result) left, (Result) right); + } else if (sameClass(left, right, Datasource.class)) { + // TODO + final int trust = compareTrust(left, right); + return mergeOafEntityFields((Datasource) left, (Datasource) right, trust); + } else if (sameClass(left, right, Organization.class)) { + return mergeOrganization((Organization) left, (Organization) right); + } else if (sameClass(left, right, Project.class)) { + return mergeProject((Project) left, (Project) right); + } else { + throw new RuntimeException( + String + .format( + "MERGE_FROM_AND_GET incompatible types: %s, %s", + left.getClass().getCanonicalName(), right.getClass().getCanonicalName())); + } + } + + /** + * This method is used in the global result grouping phase. It checks if one of the two is from a delegated authority + * https://graph.openaire.eu/docs/data-model/pids-and-identifiers#delegated-authorities and in that case it prefers + * such version. + *

+ * Otherwise, it considers a resulttype priority order implemented in {@link ResultTypeComparator} + * and proceeds with the canonical property merging. + * + * @param left + * @param right + * @return + */ + private static T mergeResultsOfDifferentTypes(T left, T right) { + + final boolean leftFromDelegatedAuthority = isFromDelegatedAuthority(left); + final boolean rightFromDelegatedAuthority = isFromDelegatedAuthority(right); + + if (leftFromDelegatedAuthority && !rightFromDelegatedAuthority) { + return left; + } + if (!leftFromDelegatedAuthority && rightFromDelegatedAuthority) { + return right; + } + //TODO: raise trust to have preferred fields from one or the other?? + if (new ResultTypeComparator().compare(left, right) < 0) { + return mergeResult(left, right); + } else { + return mergeResult(right, left); + } + } + + private static DataInfo chooseDataInfo(DataInfo left, DataInfo right, int trust) { + if (trust > 0) { + return left; + } else if (trust == 0) { + if (left == null || (left.getInvisible() != null && left.getInvisible().equals(Boolean.TRUE))) { + return right; + } else { + return left; + } + } else { + return right; + } + } + + private static String chooseString(String left, String right, int trust) { + if (trust > 0) { + return left; + } else if (trust == 0) { + return StringUtils.isNotBlank(left) ? left : right; + } else { + return right; + } + } + + private static T chooseReference(T left, T right, int trust) { + if (trust > 0) { + return left; + } else if (trust == 0) { + return left != null ? left : right; + } else { + return right; + } + } + + private static Long max(Long left, Long right) { + if (left == null) + return right; + if (right == null) + return left; + + return Math.max(left, right); + } + + // trust ?? + private static Boolean booleanOR(Boolean a, Boolean b) { + if (a == null) { + return b; + } else if (b == null) { + return a; + } + + return a || b; + } + + + private static List unionDistinctLists(final List left, final List right, int trust) { + if (left == null) { + return right; + } else if (right == null) { + return left; + } + + List h = trust >= 0 ? left : right; + List l = trust >= 0 ? right : left; + + return Stream.concat(h.stream(), l.stream()) + .filter(Objects::nonNull) + .distinct() + .collect(Collectors.toList()); + } + + private static List unionDistinctListOfString(final List l, final List r) { + if (l == null) { + return r; + } else if (r == null) { + return l; + } + + return Stream.concat(l.stream(), r.stream()) + .filter(StringUtils::isNotBlank) + .distinct() + .collect(Collectors.toList()); + } + + //TODO review + private static List mergeKeyValue(List left, List right, int trust) { + if (trust < 0) { + List s = left; + left = right; + right = s; + } + + HashMap values = new HashMap<>(); + left.forEach(kv -> values.put(kv.getKey(), kv)); + right.forEach(kv -> values.putIfAbsent(kv.getKey(), kv)); + + return new ArrayList<>(values.values()); + } + + private static List unionTitle(List left, List right, int trust) { + if (left == null) { + return right; + } else if (right == null) { + return left; + } + + List h = trust >= 0 ? left : right; + List l = trust >= 0 ? right : left; + + return Stream.concat(h.stream(), l.stream()) + .filter(Objects::isNull) + .distinct() + .collect(Collectors.toList()); + } + + /** + * Internal utility that merges the common OafEntity fields + * + * @param merged + * @param enrich + * @param + * @return + */ + private static T mergeOafFields(T merged, T enrich, int trust) { + + //TODO: union of all values, but what does it mean with KeyValue pairs??? + merged.setCollectedfrom(mergeKeyValue(merged.getCollectedfrom(), enrich.getCollectedfrom(), trust)); + merged.setDataInfo(chooseDataInfo(merged.getDataInfo(), enrich.getDataInfo(), trust)); + merged.setLastupdatetimestamp(max(merged.getLastupdatetimestamp(), enrich.getLastupdatetimestamp())); + + return merged; + } + + /** + * Internal utility that merges the common OafEntity fields + * + * @param original + * @param enrich + * @param + * @return + */ + private static T mergeOafEntityFields(T original, T enrich, int trust) { + final T merged = mergeOafFields(original, enrich, trust); + + merged.setOriginalId(unionDistinctListOfString(merged.getOriginalId(), enrich.getOriginalId())); + merged.setPid(unionDistinctLists(merged.getPid(), enrich.getPid(), trust)); + // dateofcollection mettere today quando si fa merge + merged.setDateofcollection(chooseString(merged.getDateofcollection(), enrich.getDateofcollection(), trust)); + // setDateoftransformation mettere vuota in dedup, nota per Claudio + merged.setDateoftransformation(chooseString(merged.getDateoftransformation(), enrich.getDateoftransformation(), trust)); + // TODO: was missing in OafEntity.merge + merged.setExtraInfo(unionDistinctLists(merged.getExtraInfo(), enrich.getExtraInfo(), trust)); + //oaiprovenanze da mettere a null quando si genera merge + merged.setOaiprovenance(chooseReference(merged.getOaiprovenance(), enrich.getOaiprovenance(), trust)); + merged.setMeasures(unionDistinctLists(merged.getMeasures(), enrich.getMeasures(), trust)); + + return merged; + } + + + public static T mergeRelation(T original, T enrich) { + int trust = compareTrust(original, enrich); + T merge = mergeOafFields(original, enrich, trust); + + checkArgument(Objects.equals(merge.getSource(), enrich.getSource()), "source ids must be equal"); + checkArgument(Objects.equals(merge.getTarget(), enrich.getTarget()), "target ids must be equal"); + checkArgument(Objects.equals(merge.getRelType(), enrich.getRelType()), "relType(s) must be equal"); + checkArgument( + Objects.equals(merge.getSubRelType(), enrich.getSubRelType()), "subRelType(s) must be equal"); + checkArgument(Objects.equals(merge.getRelClass(), enrich.getRelClass()), "relClass(es) must be equal"); + + //merge.setProvenance(mergeLists(merge.getProvenance(), enrich.getProvenance())); + + //TODO: trust ?? + merge.setValidated(booleanOR(merge.getValidated(), enrich.getValidated())); + try { + merge.setValidationDate(ModelSupport.oldest(merge.getValidationDate(), enrich.getValidationDate())); + } catch (ParseException e) { + throw new IllegalArgumentException(String + .format( + "invalid validation date format in relation [s:%s, t:%s]: %s", merge.getSource(), + merge.getTarget(), + merge.getValidationDate())); + } + + // TODO keyvalue merge + merge.setProperties(mergeKeyValue(merge.getProperties(), enrich.getProperties(), trust)); + + return merge; + } + + public static T mergeResult(T original, T enrich) { + final int trust = compareTrust(original, enrich); + T merge = mergeOafEntityFields(original, enrich, trust); + + if (merge.getProcessingchargeamount() == null || StringUtils.isBlank(merge.getProcessingchargeamount().getValue())) { + merge.setProcessingchargeamount(enrich.getProcessingchargeamount()); + merge.setProcessingchargecurrency(enrich.getProcessingchargecurrency()); + } + + // author = usare la stessa logica che in dedup + merge.setAuthor(chooseReference(merge.getAuthor(), enrich.getAuthor(), trust)); + // il primo che mi arriva secondo l'ordinamento per priorita' + merge.setResulttype(chooseReference(merge.getResulttype(), enrich.getResulttype(), trust)); + // gestito come il resulttype perche' e' un subtype + merge.setMetaResourceType(chooseReference(merge.getMetaResourceType(), enrich.getMetaResourceType(), trust)); + // spostiamo nell'instance e qui prendo il primo che arriva + merge.setLanguage(chooseReference(merge.getLanguage(), enrich.getLanguage(), trust)); + // country lasicamo,o cosi' -> parentesi sul datainfo + merge.setCountry(unionDistinctLists(merge.getCountry(), enrich.getCountry(), trust)); + //ok + merge.setSubject(unionDistinctLists(merge.getSubject(), enrich.getSubject(), trust)); + // union per priority quindi vanno in append + merge.setTitle(unionTitle(merge.getTitle(), enrich.getTitle(), trust)); + //ok + merge.setRelevantdate(unionDistinctLists(merge.getRelevantdate(), enrich.getRelevantdate(), trust)); + // prima trust e poi longest list + merge.setDescription(longestLists(merge.getDescription(), enrich.getDescription())); + // trust piu' alto e poi piu' vecchia + merge.setDateofacceptance(chooseReference(merge.getDateofacceptance(), enrich.getDateofacceptance(), trust)); + // ok, ma publisher va messo ripetibile + merge.setPublisher(chooseReference(merge.getPublisher(), enrich.getPublisher(), trust)); + // ok + merge.setEmbargoenddate(chooseReference(merge.getEmbargoenddate(), enrich.getEmbargoenddate(), trust)); + // ok + merge.setSource(unionDistinctLists(merge.getSource(), enrich.getSource(), trust)); + // ok + merge.setFulltext(unionDistinctLists(merge.getFulltext(), enrich.getFulltext(), trust)); + // ok + merge.setFormat(unionDistinctLists(merge.getFormat(), enrich.getFormat(), trust)); + // ok + merge.setContributor(unionDistinctLists(merge.getContributor(), enrich.getContributor(), trust)); + + // prima prendo l'higher trust, su questo prendo il valore migliore nelle istanze TODO + // trust maggiore ma a parita' di trust il piu' specifico (base del vocabolario) + // vedi note + merge.setResourcetype(firstNonNull(merge.getResourcetype(), enrich.getResourcetype())); + + // ok + merge.setCoverage(unionDistinctLists(merge.getCoverage(), enrich.getCoverage(), trust)); + + // most open ok + if (enrich.getBestaccessright() != null + && new AccessRightComparator<>() + .compare(enrich.getBestaccessright(), merge.getBestaccessright()) < 0) { + merge.setBestaccessright(enrich.getBestaccessright()); + } + + // TODO merge of datainfo given same id + merge.setContext(unionDistinctLists(merge.getContext(), enrich.getContext(), trust)); + + //ok + merge.setExternalReference(unionDistinctLists(merge.getExternalReference(), enrich.getExternalReference(), trust)); + + //instance enrichment or union + // review instance equals => add pid to comparision + if (!isAnEnrichment(merge) && !isAnEnrichment(enrich)) + merge.setInstance(unionDistinctLists(merge.getInstance(), enrich.getInstance(), trust)); + else { + final List enrichmentInstances = isAnEnrichment(merge) ? merge.getInstance() + : enrich.getInstance(); + final List enrichedInstances = isAnEnrichment(merge) ? enrich.getInstance() + : merge.getInstance(); + if (isAnEnrichment(merge)) + merge.setDataInfo(enrich.getDataInfo()); + merge.setInstance(enrichInstances(enrichedInstances, enrichmentInstances)); + } + + merge.setEoscifguidelines(unionDistinctLists(merge.getEoscifguidelines(), enrich.getEoscifguidelines(), trust)); + merge.setIsGreen(booleanOR(merge.getIsGreen(), enrich.getIsGreen())); + // OK but should be list of values + merge.setOpenAccessColor(chooseReference(merge.getOpenAccessColor(), enrich.getOpenAccessColor(), trust)); + merge.setIsInDiamondJournal(booleanOR(merge.getIsInDiamondJournal(), enrich.getIsInDiamondJournal())); + merge.setPubliclyFunded(booleanOR(merge.getPubliclyFunded(), enrich.getPubliclyFunded())); + + return merge; + } + + private static T mergeORP(T original, T enrich) { + int trust = compareTrust(original, enrich); + final T merge = mergeResult(original, enrich); + + merge.setContactperson(unionDistinctLists(merge.getContactperson(), enrich.getContactperson(), trust)); + merge.setContactgroup(unionDistinctLists(merge.getContactgroup(), enrich.getContactgroup(), trust)); + merge.setTool(unionDistinctLists(merge.getTool(), enrich.getTool(), trust)); + + return merge; + } + + private static T mergeSoftware(T original, T enrich) { + int trust = compareTrust(original, enrich); + final T merge = mergeResult(original, enrich); + + merge.setDocumentationUrl(unionDistinctLists(merge.getDocumentationUrl(), enrich.getDocumentationUrl(), trust)); + merge.setLicense(unionDistinctLists(merge.getLicense(), enrich.getLicense(), trust)); + merge.setCodeRepositoryUrl(chooseReference(merge.getCodeRepositoryUrl(), enrich.getCodeRepositoryUrl(), trust)); + merge.setProgrammingLanguage(chooseReference(merge.getProgrammingLanguage(), enrich.getProgrammingLanguage(), trust)); + + return merge; + } + + private static T mergeDataset(T original, T enrich) { + int trust = compareTrust(original, enrich); + T merge = mergeResult(original, enrich); + + merge.setStoragedate(chooseReference(merge.getStoragedate(), enrich.getStoragedate(), trust)); + merge.setDevice(chooseReference(merge.getDevice(), enrich.getDevice(), trust)); + merge.setSize(chooseReference(merge.getSize(), enrich.getSize(), trust)); + merge.setVersion(chooseReference(merge.getVersion(), enrich.getVersion(), trust)); + merge.setLastmetadataupdate(chooseReference(merge.getLastmetadataupdate(), enrich.getLastmetadataupdate(), trust)); + merge.setMetadataversionnumber(chooseReference(merge.getMetadataversionnumber(), enrich.getMetadataversionnumber(), trust)); + merge.setGeolocation(unionDistinctLists(merge.getGeolocation(), enrich.getGeolocation(), trust)); + + return merge; + } + + public static T mergePublication(T original, T enrich) { + final int trust = compareTrust(original, enrich); + T merged = mergeResult(original, enrich); + + merged.setJournal(chooseReference(merged.getJournal(), enrich.getJournal(), trust)); + + return merged; + } + + private static T mergeOrganization(T left, T enrich) { + int trust = compareTrust(left, enrich); + T merged = mergeOafEntityFields(left, enrich, trust); + + merged.setLegalshortname(chooseReference(merged.getLegalshortname(), enrich.getLegalshortname(), trust)); + merged.setLegalname(chooseReference(merged.getLegalname(), enrich.getLegalname(), trust)); + merged.setAlternativeNames(unionDistinctLists(enrich.getAlternativeNames(), merged.getAlternativeNames(), trust)); + merged.setWebsiteurl(chooseReference(merged.getWebsiteurl(), enrich.getWebsiteurl(), trust)); + merged.setLogourl(chooseReference(merged.getLogourl(), enrich.getLogourl(), trust)); + merged.setEclegalbody(chooseReference(merged.getEclegalbody(), enrich.getEclegalbody(), trust)); + merged.setEclegalperson(chooseReference(merged.getEclegalperson(), enrich.getEclegalperson(), trust)); + merged.setEcnonprofit(chooseReference(merged.getEcnonprofit(), enrich.getEcnonprofit(), trust)); + merged.setEcresearchorganization(chooseReference(merged.getEcresearchorganization(), enrich.getEcresearchorganization(), trust)); + merged.setEchighereducation(chooseReference(merged.getEchighereducation(), enrich.getEchighereducation(), trust)); + merged.setEcinternationalorganizationeurinterests(chooseReference(merged.getEcinternationalorganizationeurinterests(), enrich.getEcinternationalorganizationeurinterests(), trust)); + merged.setEcinternationalorganization(chooseReference(merged.getEcinternationalorganization(), enrich.getEcinternationalorganization(), trust)); + merged.setEcenterprise(chooseReference(merged.getEcenterprise(), enrich.getEcenterprise(), trust)); + merged.setEcsmevalidated(chooseReference(merged.getEcsmevalidated(), enrich.getEcsmevalidated(), trust)); + merged.setEcnutscode(chooseReference(merged.getEcnutscode(), enrich.getEcnutscode(), trust)); + merged.setCountry(chooseReference(merged.getCountry(), enrich.getCountry(), trust)); + + return merged; + } + + public static T mergeProject(T original, T enrich) { + int trust = compareTrust(original, enrich); + T merged = mergeOafEntityFields(original, enrich, trust); + + merged.setWebsiteurl(chooseReference(merged.getWebsiteurl(), enrich.getWebsiteurl(), trust)); + merged.setCode(chooseReference(merged.getCode(), enrich.getCode(), trust)); + merged.setAcronym(chooseReference(merged.getAcronym(), enrich.getAcronym(), trust)); + merged.setTitle(chooseReference(merged.getTitle(), enrich.getTitle(), trust)); + merged.setStartdate(chooseReference(merged.getStartdate(), enrich.getStartdate(), trust)); + merged.setEnddate(chooseReference(merged.getEnddate(), enrich.getEnddate(), trust)); + merged.setCallidentifier(chooseReference(merged.getCallidentifier(), enrich.getCallidentifier(), trust)); + merged.setKeywords(chooseReference(merged.getKeywords(), enrich.getKeywords(), trust)); + merged.setDuration(chooseReference(merged.getDuration(), enrich.getDuration(), trust)); + merged.setEcsc39(chooseReference(merged.getEcsc39(), enrich.getEcsc39(), trust)); + merged.setOamandatepublications(chooseReference(merged.getOamandatepublications(), enrich.getOamandatepublications(), trust)); + merged.setEcarticle29_3(chooseReference(merged.getEcarticle29_3(), enrich.getEcarticle29_3(), trust)); + merged.setSubjects(unionDistinctLists(merged.getSubjects(), enrich.getSubjects(), trust)); + merged.setFundingtree(unionDistinctLists(merged.getFundingtree(), enrich.getFundingtree(), trust)); + merged.setContracttype(chooseReference(merged.getContracttype(), enrich.getContracttype(), trust)); + merged.setOptional1(chooseReference(merged.getOptional1(), enrich.getOptional1(), trust)); + merged.setOptional2(chooseReference(merged.getOptional2(), enrich.getOptional2(), trust)); + merged.setJsonextrainfo(chooseReference(merged.getJsonextrainfo(), enrich.getJsonextrainfo(), trust)); + merged.setContactfullname(chooseReference(merged.getContactfullname(), enrich.getContactfullname(), trust)); + merged.setContactfax(chooseReference(merged.getContactfax(), enrich.getContactfax(), trust)); + merged.setContactphone(chooseReference(merged.getContactphone(), enrich.getContactphone(), trust)); + merged.setContactemail(chooseReference(merged.getContactemail(), enrich.getContactemail(), trust)); + merged.setSummary(chooseReference(merged.getSummary(), enrich.getSummary(), trust)); + merged.setCurrency(chooseReference(merged.getCurrency(), enrich.getCurrency(), trust)); + + //missin in Project.merge + merged.setTotalcost(chooseReference(merged.getTotalcost(), enrich.getTotalcost(), trust)); + merged.setFundedamount(chooseReference(merged.getFundedamount(), enrich.getFundedamount(), trust)); + + // trust ?? + if (enrich.getH2020topiccode() != null && StringUtils.isEmpty(merged.getH2020topiccode())) { + merged.setH2020topiccode(enrich.getH2020topiccode()); + merged.setH2020topicdescription(enrich.getH2020topicdescription()); + } + + merged.setH2020classification(unionDistinctLists(merged.getH2020classification(), enrich.getH2020classification(), trust)); + + return merged; + } + + + /** + * Longest lists list. + * + * @param a the a + * @param b the b + * @return the list + */ + public static List> longestLists(List> a, List> b) { + if (a == null || b == null) + return a == null ? b : a; + + return a.size() >= b.size() ? a : b; + } + + /** + * This main method apply the enrichment of the instances + * + * @param toEnrichInstances the instances that could be enriched + * @param enrichmentInstances the enrichment instances + * @return list of instances possibly enriched + */ + private static List enrichInstances(final List toEnrichInstances, + final List enrichmentInstances) { + final List enrichmentResult = new ArrayList<>(); + + if (toEnrichInstances == null) { + return enrichmentResult; + } + if (enrichmentInstances == null) { + return enrichmentResult; + } + Map ri = toInstanceMap(enrichmentInstances); + + toEnrichInstances.forEach(i -> { + final List e = findEnrichmentsByPID(i.getPid(), ri); + if (e != null && e.size() > 0) { + e.forEach(enr -> applyEnrichment(i, enr)); + } else { + final List a = findEnrichmentsByPID(i.getAlternateIdentifier(), ri); + if (a != null && a.size() > 0) { + a.forEach(enr -> applyEnrichment(i, enr)); + } + } + enrichmentResult.add(i); + }); + return enrichmentResult; + } + + /** + * This method converts the list of instance enrichments + * into a Map where the key is the normalized identifier + * and the value is the instance itself + * + * @param ri the list of enrichment instances + * @return the result map + */ + private static Map toInstanceMap(final List ri) { + return ri + .stream() + .filter(i -> i.getPid() != null || i.getAlternateIdentifier() != null) + .flatMap(i -> { + final List> result = new ArrayList<>(); + if (i.getPid() != null) + i + .getPid() + .stream() + .filter(MergeUtils::validPid) + .forEach(p -> result.add(new ImmutablePair<>(extractKeyFromPid(p), i))); + if (i.getAlternateIdentifier() != null) + i + .getAlternateIdentifier() + .stream() + .filter(MergeUtils::validPid) + .forEach(p -> result.add(new ImmutablePair<>(extractKeyFromPid(p), i))); + return result.stream(); + }) + .collect( + Collectors + .toMap( + Pair::getLeft, + Pair::getRight, + (a, b) -> a)); + } + + private static boolean isFromDelegatedAuthority(Result r) { + return Optional + .ofNullable(r.getInstance()) + .map( + instance -> instance + .stream() + .filter(i -> Objects.nonNull(i.getCollectedfrom())) + .map(i -> i.getCollectedfrom().getKey()) + .anyMatch(cfId -> IdentifierFactory.delegatedAuthorityDatasourceIds().contains(cfId))) + .orElse(false); + } + + /** + * Valid pid boolean. + * + * @param p the p + * @return the boolean + */ + private static boolean validPid(final StructuredProperty p) { + return p.getValue() != null && p.getQualifier() != null && p.getQualifier().getClassid() != null; + } + + /** + * Normalize pid string. + * + * @param pid the pid + * @return the string + */ + private static String extractKeyFromPid(final StructuredProperty pid) { + if (pid == null) + return null; + final StructuredProperty normalizedPid = CleaningFunctions.normalizePidValue(pid); + + return String.format("%s::%s", normalizedPid.getQualifier().getClassid(), normalizedPid.getValue()); + } + + /** + * This utility method finds the list of enrichment instances + * that match one or more PIDs in the input list + * + * @param pids the list of PIDs + * @param enrichments the List of enrichment instances having the same pid + * @return the list + */ + private static List findEnrichmentsByPID(final List pids, + final Map enrichments) { + if (pids == null || enrichments == null) + return null; + return pids + .stream() + .map(MergeUtils::extractKeyFromPid) + .map(enrichments::get) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + } + + /** + * Is an enrichment boolean. + * + * @param e the e + * @return the boolean + */ + private static boolean isAnEnrichment(OafEntity e) { + return e.getDataInfo() != null && + e.getDataInfo().getProvenanceaction() != null + && ModelConstants.PROVENANCE_ENRICH.equalsIgnoreCase(e.getDataInfo().getProvenanceaction().getClassid()); + } + + /** + * This method apply enrichment on a single instance + * The enrichment consists of replacing values on + * single attribute only if in the current instance is missing + * The only repeatable field enriched is measures + * + * @param merge the current instance + * @param enrichment the enrichment instance + */ + private static void applyEnrichment(final Instance merge, final Instance enrichment) { + if (merge == null || enrichment == null) + return; + + merge.setLicense(firstNonNull(merge.getLicense(), enrichment.getLicense())); + merge.setAccessright(firstNonNull(merge.getAccessright(), enrichment.getAccessright())); + merge.setInstancetype(firstNonNull(merge.getInstancetype(), enrichment.getInstancetype())); + merge.setInstanceTypeMapping(firstNonNull(merge.getInstanceTypeMapping(), enrichment.getInstanceTypeMapping())); + merge.setHostedby(firstNonNull(merge.getHostedby(), enrichment.getHostedby())); + merge.setUrl(unionDistinctLists(merge.getUrl(), enrichment.getUrl(), 0)); + merge.setDistributionlocation(firstNonNull(merge.getDistributionlocation(), enrichment.getDistributionlocation())); + merge.setCollectedfrom(firstNonNull(merge.getCollectedfrom(), enrichment.getCollectedfrom())); + // pid and alternateId are used for matching + merge.setDateofacceptance(firstNonNull(merge.getDateofacceptance(), enrichment.getDateofacceptance())); + merge.setProcessingchargeamount(firstNonNull(merge.getProcessingchargeamount(), enrichment.getProcessingchargeamount())); + merge.setProcessingchargecurrency(firstNonNull(merge.getProcessingchargecurrency(), enrichment.getProcessingchargecurrency())); + merge.setRefereed(firstNonNull(merge.getRefereed(), enrichment.getRefereed())); + merge.setMeasures(unionDistinctLists(merge.getMeasures(), enrichment.getMeasures(), 0)); + merge.setFulltext(firstNonNull(merge.getFulltext(), enrichment.getFulltext())); + } + + private static int compareTrust(Oaf a, Oaf b) { + String left = Optional + .ofNullable(a.getDataInfo()) + .map(DataInfo::getTrust) + .orElse("0.0"); + + String right = Optional + .ofNullable(b.getDataInfo()) + .map(DataInfo::getTrust) + .orElse("0.0"); + + return left.compareTo(right); + } + +} 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 4cecd0895..417516b0f 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.ModelSupport; import eu.dnetlib.dhp.schema.oaf.*; public class OafMapperUtils { @@ -22,65 +21,6 @@ public class OafMapperUtils { private OafMapperUtils() { } - public static Oaf merge(final Oaf left, final Oaf right) { - if (ModelSupport.isSubClass(left, OafEntity.class)) { - return mergeEntities((OafEntity) left, (OafEntity) right); - } else if (ModelSupport.isSubClass(left, Relation.class)) { - ((Relation) left).mergeFrom((Relation) right); - } else { - throw new IllegalArgumentException("invalid Oaf type:" + left.getClass().getCanonicalName()); - } - return left; - } - - public static OafEntity mergeEntities(OafEntity left, OafEntity right) { - if (ModelSupport.isSubClass(left, Result.class)) { - return mergeResults((Result) left, (Result) right); - } else if (ModelSupport.isSubClass(left, Datasource.class)) { - left.mergeFrom(right); - } else if (ModelSupport.isSubClass(left, Organization.class)) { - left.mergeFrom(right); - } else if (ModelSupport.isSubClass(left, Project.class)) { - left.mergeFrom(right); - } else { - throw new IllegalArgumentException("invalid OafEntity subtype:" + left.getClass().getCanonicalName()); - } - return left; - } - - public static Result mergeResults(Result left, Result right) { - - final boolean leftFromDelegatedAuthority = isFromDelegatedAuthority(left); - final boolean rightFromDelegatedAuthority = isFromDelegatedAuthority(right); - - if (leftFromDelegatedAuthority && !rightFromDelegatedAuthority) { - return left; - } - if (!leftFromDelegatedAuthority && rightFromDelegatedAuthority) { - return right; - } - - if (new ResultTypeComparator().compare(left, right) < 0) { - left.mergeFrom(right); - return left; - } else { - right.mergeFrom(left); - return right; - } - } - - private static boolean isFromDelegatedAuthority(Result r) { - return Optional - .ofNullable(r.getInstance()) - .map( - instance -> instance - .stream() - .filter(i -> Objects.nonNull(i.getCollectedfrom())) - .map(i -> i.getCollectedfrom().getKey()) - .anyMatch(cfId -> IdentifierFactory.delegatedAuthorityDatasourceIds().contains(cfId))) - .orElse(false); - } - public static KeyValue keyValue(final String k, final String v) { final KeyValue kv = new KeyValue(); kv.setKey(k); diff --git a/dhp-common/src/test/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtilsTest.java b/dhp-common/src/test/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtilsTest.java new file mode 100644 index 000000000..ef35b50fb --- /dev/null +++ b/dhp-common/src/test/java/eu/dnetlib/dhp/schema/oaf/utils/MergeUtilsTest.java @@ -0,0 +1,111 @@ + +package eu.dnetlib.dhp.schema.oaf.utils; + +import static org.junit.jupiter.api.Assertions.*; +import static org.junit.jupiter.api.Assertions.assertEquals; + +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.stream.Collectors; + +import eu.dnetlib.dhp.schema.common.ModelSupport; +import org.apache.commons.io.IOUtils; +import org.junit.jupiter.api.Test; + +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.Dataset; +import eu.dnetlib.dhp.schema.oaf.KeyValue; +import eu.dnetlib.dhp.schema.oaf.Publication; +import eu.dnetlib.dhp.schema.oaf.Result; + +public class MergeUtilsTest { + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + + @Test + void testMergePubs() throws IOException { + Publication p1 = read("publication_1.json", Publication.class); + Publication p2 = read("publication_2.json", Publication.class); + Dataset d1 = read("dataset_1.json", Dataset.class); + Dataset d2 = read("dataset_2.json", Dataset.class); + + assertEquals(1, p1.getCollectedfrom().size()); + assertEquals(ModelConstants.CROSSREF_ID, p1.getCollectedfrom().get(0).getKey()); + assertEquals(1, d2.getCollectedfrom().size()); + assertFalse(cfId(d2.getCollectedfrom()).contains(ModelConstants.CROSSREF_ID)); + + assertEquals(1, p2.getCollectedfrom().size()); + assertFalse(cfId(p2.getCollectedfrom()).contains(ModelConstants.CROSSREF_ID)); + assertEquals(1, d1.getCollectedfrom().size()); + assertTrue(cfId(d1.getCollectedfrom()).contains(ModelConstants.CROSSREF_ID)); + + final Result p1d2 = MergeUtils.checkedMerge(p1, d2); + assertEquals(ModelConstants.PUBLICATION_RESULTTYPE_CLASSID, p1d2.getResulttype().getClassid()); + assertTrue(p1d2 instanceof Publication); + assertEquals(p1.getId(), p1d2.getId()); + } + + @Test + void testMergePubs_1() throws IOException { + Publication p2 = read("publication_2.json", Publication.class); + Dataset d1 = read("dataset_1.json", Dataset.class); + + final Result p2d1 = MergeUtils.checkedMerge(p2, d1); + assertEquals((ModelConstants.DATASET_RESULTTYPE_CLASSID), p2d1.getResulttype().getClassid()); + assertTrue(p2d1 instanceof Dataset); + assertEquals(d1.getId(), p2d1.getId()); + assertEquals(2, p2d1.getCollectedfrom().size()); + } + + @Test + void testMergePubs_2() throws IOException { + Publication p1 = read("publication_1.json", Publication.class); + Publication p2 = read("publication_2.json", Publication.class); + + Result p1p2 = MergeUtils.checkedMerge(p1, p2); + assertTrue(p1p2 instanceof Publication); + assertEquals(p1.getId(), p1p2.getId()); + assertEquals(2, p1p2.getCollectedfrom().size()); + } + + @Test + void testDelegatedAuthority_1() throws IOException { + Dataset d1 = read("dataset_2.json", Dataset.class); + Dataset d2 = read("dataset_delegated.json", Dataset.class); + + assertEquals(1, d2.getCollectedfrom().size()); + assertTrue(cfId(d2.getCollectedfrom()).contains(ModelConstants.ZENODO_OD_ID)); + + Result res = (Result) MergeUtils.merge(d1, d2, true); + + assertEquals(d2, res); + } + + @Test + void testDelegatedAuthority_2() throws IOException { + Dataset p1 = read("publication_1.json", Dataset.class); + Dataset d2 = read("dataset_delegated.json", Dataset.class); + + assertEquals(1, d2.getCollectedfrom().size()); + assertTrue(cfId(d2.getCollectedfrom()).contains(ModelConstants.ZENODO_OD_ID)); + + Result res = (Result) MergeUtils.merge(p1, d2, true); + + assertEquals(d2, res); + } + + protected HashSet cfId(List collectedfrom) { + return collectedfrom.stream().map(KeyValue::getKey).collect(Collectors.toCollection(HashSet::new)); + } + + protected T read(String filename, Class clazz) throws IOException { + final String json = IOUtils.toString(getClass().getResourceAsStream(filename)); + return OBJECT_MAPPER.readValue(json, clazz); + } + +} diff --git a/dhp-common/src/test/java/eu/dnetlib/dhp/schema/oaf/utils/OafMapperUtilsTest.java b/dhp-common/src/test/java/eu/dnetlib/dhp/schema/oaf/utils/OafMapperUtilsTest.java index 9111ac2df..2bbc3bc3e 100644 --- a/dhp-common/src/test/java/eu/dnetlib/dhp/schema/oaf/utils/OafMapperUtilsTest.java +++ b/dhp-common/src/test/java/eu/dnetlib/dhp/schema/oaf/utils/OafMapperUtilsTest.java @@ -149,7 +149,7 @@ class OafMapperUtilsTest { void testDate() { final String date = GraphCleaningFunctions.cleanDate("23-FEB-1998"); assertNotNull(date); - System.out.println(date); + assertEquals("1998-02-23", date); } @Test @@ -166,8 +166,8 @@ class OafMapperUtilsTest { assertEquals( ModelConstants.PUBLICATION_RESULTTYPE_CLASSID, - OafMapperUtils - .mergeResults(p1, d2) + MergeUtils + .mergeResult(p1, d2) .getResulttype() .getClassid()); @@ -178,8 +178,8 @@ class OafMapperUtilsTest { assertEquals( ModelConstants.DATASET_RESULTTYPE_CLASSID, - OafMapperUtils - .mergeResults(p2, d1) + MergeUtils + .mergeResult(p2, d1) .getResulttype() .getClassid()); } @@ -192,7 +192,7 @@ class OafMapperUtilsTest { assertEquals(1, d2.getCollectedfrom().size()); assertTrue(cfId(d2.getCollectedfrom()).contains(ModelConstants.ZENODO_OD_ID)); - Result res = OafMapperUtils.mergeResults(d1, d2); + Result res = MergeUtils.mergeResult(d1, d2); assertEquals(d2, res); diff --git a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/InstanceTypeMatch.java b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/InstanceTypeMatch.java index 34ebcf7a7..42459ef00 100644 --- a/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/InstanceTypeMatch.java +++ b/dhp-pace-core/src/main/java/eu/dnetlib/pace/tree/InstanceTypeMatch.java @@ -23,15 +23,18 @@ public class InstanceTypeMatch extends AbstractListComparator { // jolly types translationMap.put("Conference object", "*"); + translationMap.put("Research", "*"); translationMap.put("Other literature type", "*"); translationMap.put("Unknown", "*"); translationMap.put("UNKNOWN", "*"); // article types translationMap.put("Article", "Article"); + translationMap.put("Journal", "Article"); translationMap.put("Data Paper", "Article"); translationMap.put("Software Paper", "Article"); translationMap.put("Preprint", "Article"); + translationMap.put("Part of book or chapter of book", "Article"); // thesis types translationMap.put("Thesis", "Thesis"); diff --git a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/MergeAndGet.java b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/MergeAndGet.java index eccfa445c..6de631173 100644 --- a/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/MergeAndGet.java +++ b/dhp-workflows/dhp-actionmanager/src/main/java/eu/dnetlib/dhp/actionmanager/promote/MergeAndGet.java @@ -1,14 +1,13 @@ package eu.dnetlib.dhp.actionmanager.promote; -import static eu.dnetlib.dhp.schema.common.ModelSupport.isSubClass; +import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.SerializableSupplier; +import eu.dnetlib.dhp.schema.oaf.Oaf; +import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils; import java.util.function.BiFunction; -import eu.dnetlib.dhp.common.FunctionalInterfaceSupport.SerializableSupplier; -import eu.dnetlib.dhp.schema.oaf.Oaf; -import eu.dnetlib.dhp.schema.oaf.OafEntity; -import eu.dnetlib.dhp.schema.oaf.Relation; +import static eu.dnetlib.dhp.schema.common.ModelSupport.isSubClass; /** OAF model merging support. */ public class MergeAndGet { @@ -46,20 +45,7 @@ public class MergeAndGet { } private static G mergeFromAndGet(G x, A y) { - if (isSubClass(x, Relation.class) && isSubClass(y, Relation.class)) { - ((Relation) x).mergeFrom((Relation) y); - return x; - } else if (isSubClass(x, OafEntity.class) - && isSubClass(y, OafEntity.class) - && isSubClass(x, y)) { - ((OafEntity) x).mergeFrom((OafEntity) y); - return x; - } - throw new RuntimeException( - String - .format( - "MERGE_FROM_AND_GET incompatible types: %s, %s", - x.getClass().getCanonicalName(), y.getClass().getCanonicalName())); + return (G) MergeUtils.merge(x, y); } @SuppressWarnings("unchecked") diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipmodel/score/deserializers/BipProjectModel.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipmodel/score/deserializers/BipProjectModel.java index 680e12504..a41540564 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipmodel/score/deserializers/BipProjectModel.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/bipmodel/score/deserializers/BipProjectModel.java @@ -34,6 +34,11 @@ public class BipProjectModel { String totalCitationCount; + public String getProjectId() { + return projectId; + } + + // each project bip measure has exactly one value, hence one key-value pair private Measure createMeasure(String measureId, String measureValue) { diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/project/SparkAtomicActionJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/project/SparkAtomicActionJob.java index a32d04e1f..21f3044a5 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/project/SparkAtomicActionJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/project/SparkAtomicActionJob.java @@ -1,12 +1,20 @@ package eu.dnetlib.dhp.actionmanager.project; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; - -import java.util.Arrays; -import java.util.Objects; -import java.util.Optional; - +import com.fasterxml.jackson.databind.ObjectMapper; +import eu.dnetlib.dhp.actionmanager.project.utils.model.CSVProgramme; +import eu.dnetlib.dhp.actionmanager.project.utils.model.CSVProject; +import eu.dnetlib.dhp.actionmanager.project.utils.model.JsonTopic; +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.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.H2020Classification; +import eu.dnetlib.dhp.schema.oaf.H2020Programme; +import eu.dnetlib.dhp.schema.oaf.OafEntity; +import eu.dnetlib.dhp.schema.oaf.Project; +import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils; +import eu.dnetlib.dhp.utils.DHPUtils; import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.SequenceFileOutputFormat; @@ -18,24 +26,14 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - -import com.fasterxml.jackson.databind.ObjectMapper; - -import eu.dnetlib.dhp.actionmanager.project.utils.model.CSVProgramme; -import eu.dnetlib.dhp.actionmanager.project.utils.model.CSVProject; -import eu.dnetlib.dhp.actionmanager.project.utils.model.EXCELTopic; -import eu.dnetlib.dhp.actionmanager.project.utils.model.JsonTopic; -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.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.H2020Classification; -import eu.dnetlib.dhp.schema.oaf.H2020Programme; -import eu.dnetlib.dhp.schema.oaf.OafEntity; -import eu.dnetlib.dhp.schema.oaf.Project; -import eu.dnetlib.dhp.utils.DHPUtils; import scala.Tuple2; +import java.util.Arrays; +import java.util.Objects; +import java.util.Optional; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + /** * Class that makes the ActionSet. To prepare the AS two joins are needed * @@ -160,9 +158,11 @@ public class SparkAtomicActionJob { (MapFunction) OafEntity::getId, Encoders.STRING()) .mapGroups((MapGroupsFunction) (s, it) -> { - Project first = it.next(); - it.forEachRemaining(first::mergeFrom); - return first; + Project merge = it.next(); + while (it.hasNext()) { + merge = MergeUtils.mergeProject(merge, it.next()); + } + return merge; }, Encoders.bean(Project.class)) .toJavaRDD() .map(p -> new AtomicAction(Project.class, p)) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java index c35fbb497..98caa1741 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/CollectorWorker.java @@ -19,6 +19,7 @@ import org.slf4j.LoggerFactory; import eu.dnetlib.dhp.aggregation.common.ReporterCallback; import eu.dnetlib.dhp.aggregation.common.ReportingJob; import eu.dnetlib.dhp.collection.plugin.CollectorPlugin; +import eu.dnetlib.dhp.collection.plugin.base.BaseCollectorPlugin; import eu.dnetlib.dhp.collection.plugin.file.FileCollectorPlugin; import eu.dnetlib.dhp.collection.plugin.file.FileGZipCollectorPlugin; import eu.dnetlib.dhp.collection.plugin.mongodb.MDStoreCollectorPlugin; @@ -120,6 +121,8 @@ public class CollectorWorker extends ReportingJob { return new FileCollectorPlugin(fileSystem); case fileGzip: return new FileGZipCollectorPlugin(fileSystem); + case baseDump: + return new BaseCollectorPlugin(this.fileSystem); case other: final CollectorPlugin.NAME.OTHER_NAME plugin = Optional .ofNullable(api.getParams().get("other_plugin_type")) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ORCIDWorker.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ORCIDWorker.java new file mode 100644 index 000000000..2767a3e41 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/ORCIDWorker.java @@ -0,0 +1,244 @@ + +package eu.dnetlib.dhp.collection.orcid; + +import java.io.IOException; +import java.io.InputStream; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.concurrent.BlockingQueue; + +import javax.swing.*; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.apache.http.HttpHeaders; +import org.jetbrains.annotations.NotNull; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.common.collection.HttpClientParams; + +public class ORCIDWorker extends Thread { + + final static Logger log = LoggerFactory.getLogger(ORCIDWorker.class); + + public static String JOB_COMPLETE = "JOB_COMPLETE"; + + private static final String userAgent = "Mozilla/5.0 (compatible; OAI; +http://www.openaire.eu)"; + + private final BlockingQueue queue; + + private boolean hasComplete = false; + + private final SequenceFile.Writer employments; + + private final SequenceFile.Writer summary; + private final SequenceFile.Writer works; + + private final String token; + + private final String id; + + public static ORCIDWorkerBuilder builder() { + return new ORCIDWorkerBuilder(); + } + + public ORCIDWorker(String id, BlockingQueue myqueue, SequenceFile.Writer employments, + SequenceFile.Writer summary, SequenceFile.Writer works, String token) { + this.id = id; + this.queue = myqueue; + this.employments = employments; + this.summary = summary; + this.works = works; + this.token = token; + } + + public static String retrieveURL(final String id, final String apiUrl, String token) { + try { + final HttpURLConnection urlConn = getHttpURLConnection(apiUrl, token); + if (urlConn.getResponseCode() > 199 && urlConn.getResponseCode() < 300) { + InputStream input = urlConn.getInputStream(); + return IOUtils.toString(input); + } else { + log + .error( + "Thread {} UNABLE TO DOWNLOAD FROM THIS URL {} , status code {}", id, apiUrl, + urlConn.getResponseCode()); + } + } catch (Exception e) { + log.error("Thread {} Error on retrieving URL {} {}", id, apiUrl, e); + } + return null; + } + + @NotNull + private static HttpURLConnection getHttpURLConnection(String apiUrl, String token) throws IOException { + final HttpURLConnection urlConn = (HttpURLConnection) new URL(apiUrl).openConnection(); + final HttpClientParams clientParams = new HttpClientParams(); + urlConn.setInstanceFollowRedirects(false); + urlConn.setReadTimeout(clientParams.getReadTimeOut() * 1000); + urlConn.setConnectTimeout(clientParams.getConnectTimeOut() * 1000); + urlConn.addRequestProperty(HttpHeaders.USER_AGENT, userAgent); + urlConn.addRequestProperty(HttpHeaders.AUTHORIZATION, String.format("Bearer %s", token)); + return urlConn; + } + + private static String generateSummaryURL(final String orcidId) { + return "https://api.orcid.org/v3.0/" + orcidId + "/record"; + } + + private static String generateWorksURL(final String orcidId) { + return "https://api.orcid.org/v3.0/" + orcidId + "/works"; + } + + private static String generateEmploymentsURL(final String orcidId) { + return "https://api.orcid.org/v3.0/" + orcidId + "/employments"; + } + + private static void writeResultToSequenceFile(String id, String url, String token, String orcidId, + SequenceFile.Writer file) throws IOException { + final String response = retrieveURL(id, url, token); + if (response != null) { + if (orcidId == null) { + log.error("Thread {} {} {}", id, orcidId, response); + throw new RuntimeException("null items "); + } + + if (file == null) { + log.error("Thread {} file is null for {} URL:{}", id, url, orcidId); + } else { + file.append(new Text(orcidId), new Text(response)); + file.hflush(); + } + + } else + log.error("Thread {} response is null for {} URL:{}", id, url, orcidId); + + } + + @Override + public void run() { + final Text key = new Text(); + final Text value = new Text(); + long start; + long total_time; + String orcidId = ""; + int requests = 0; + if (summary == null || employments == null || works == null) + throw new RuntimeException("Null files"); + + while (!hasComplete) { + try { + + orcidId = queue.take(); + + if (orcidId.equalsIgnoreCase(JOB_COMPLETE)) { + hasComplete = true; + } else { + start = System.currentTimeMillis(); + writeResultToSequenceFile(id, generateSummaryURL(orcidId), token, orcidId, summary); + total_time = System.currentTimeMillis() - start; + requests++; + if (total_time < 1000) { + // I know making a sleep on a thread is bad, but we need to stay to 24 requests per seconds, + // hence + // the time between two http request in a thread must be 1 second + Thread.sleep(1000L - total_time); + } + start = System.currentTimeMillis(); + writeResultToSequenceFile(id, generateWorksURL(orcidId), token, orcidId, works); + total_time = System.currentTimeMillis() - start; + requests++; + if (total_time < 1000) { + // I know making a sleep on a thread is bad, but we need to stay to 24 requests per seconds, + // hence + // the time between two http request in a thread must be 1 second + Thread.sleep(1000L - total_time); + } + start = System.currentTimeMillis(); + writeResultToSequenceFile(id, generateEmploymentsURL(orcidId), token, orcidId, employments); + total_time = System.currentTimeMillis() - start; + requests++; + if (total_time < 1000) { + // I know making a sleep on a thread is bad, but we need to stay to 24 requests per seconds, + // hence + // the time between two http request in a thread must be 1 second + Thread.sleep(1000L - total_time); + } + if (requests % 30 == 0) { + log.info("Thread {} Downloaded {}", id, requests); + } + } + + } catch (Throwable e) { + + log.error("Thread {} Unable to save ORICD: {} item error", id, orcidId, e); + + } + + } + try { + works.close(); + summary.close(); + employments.close(); + } catch (Throwable e) { + throw new RuntimeException(e); + } + + log.info("Thread {} COMPLETE ", id); + log.info("Thread {} Downloaded {}", id, requests); + + } + + public static class ORCIDWorkerBuilder { + + private String id; + private SequenceFile.Writer employments; + private SequenceFile.Writer summary; + private SequenceFile.Writer works; + private BlockingQueue queue; + + private String token; + + public ORCIDWorkerBuilder withId(final String id) { + this.id = id; + return this; + } + + public ORCIDWorkerBuilder withEmployments(final SequenceFile.Writer sequenceFile) { + this.employments = sequenceFile; + return this; + } + + public ORCIDWorkerBuilder withSummary(final SequenceFile.Writer sequenceFile) { + this.summary = sequenceFile; + return this; + } + + public ORCIDWorkerBuilder withWorks(final SequenceFile.Writer sequenceFile) { + this.works = sequenceFile; + return this; + } + + public ORCIDWorkerBuilder withAccessToken(final String accessToken) { + this.token = accessToken; + return this; + } + + public ORCIDWorkerBuilder withBlockingQueue(final BlockingQueue queue) { + this.queue = queue; + return this; + } + + public ORCIDWorker build() { + if (this.summary == null || this.works == null || this.employments == null || StringUtils.isEmpty(token) + || queue == null) + throw new RuntimeException("Unable to build missing required params"); + return new ORCIDWorker(id, queue, employments, summary, works, token); + } + + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/OrcidGetUpdatesFile.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/OrcidGetUpdatesFile.java new file mode 100644 index 000000000..d9afd0796 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/orcid/OrcidGetUpdatesFile.java @@ -0,0 +1,171 @@ + +package eu.dnetlib.dhp.collection.orcid; + +import static eu.dnetlib.dhp.utils.DHPUtils.getHadoopConfiguration; + +import java.io.*; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; + +import org.apache.commons.compress.archivers.tar.TarArchiveEntry; +import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; +import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.common.collection.HttpClientParams; + +public class OrcidGetUpdatesFile { + + private static Logger log = LoggerFactory.getLogger(OrcidGetUpdatesFile.class); + + public static void main(String[] args) throws Exception { + + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + Objects + .requireNonNull( + OrcidGetUpdatesFile.class + .getResourceAsStream( + "/eu/dnetlib/dhp/collection/orcid/download_orcid_update_parameter.json"))) + + ); + parser.parseArgument(args); + + final String namenode = parser.get("namenode"); + log.info("got variable namenode: {}", namenode); + + final String master = parser.get("master"); + log.info("got variable master: {}", master); + + final String targetPath = parser.get("targetPath"); + log.info("got variable targetPath: {}", targetPath); + + final String apiURL = parser.get("apiURL"); + log.info("got variable apiURL: {}", apiURL); + + final String accessToken = parser.get("accessToken"); + log.info("got variable accessToken: {}", accessToken); + + final String graphPath = parser.get("graphPath"); + log.info("got variable graphPath: {}", graphPath); + + final SparkSession spark = SparkSession + .builder() + .appName(OrcidGetUpdatesFile.class.getName()) + .master(master) + .getOrCreate(); + + final String latestDate = spark + .read() + .load(graphPath + "/Authors") + .selectExpr("max(lastModifiedDate)") + .first() + .getString(0); + + log.info("latest date is {}", latestDate); + + final FileSystem fileSystem = FileSystem.get(getHadoopConfiguration(namenode)); + + new OrcidGetUpdatesFile().readTar(fileSystem, accessToken, apiURL, targetPath, latestDate); + + } + + private SequenceFile.Writer createFile(Path aPath, FileSystem fileSystem) throws IOException { + return SequenceFile + .createWriter( + fileSystem.getConf(), + SequenceFile.Writer.file(aPath), + SequenceFile.Writer.keyClass(Text.class), + SequenceFile.Writer.valueClass(Text.class)); + } + + private ORCIDWorker createWorker(final String id, final String targetPath, final BlockingQueue queue, + final String accessToken, FileSystem fileSystem) throws Exception { + return ORCIDWorker + .builder() + .withId(id) + .withEmployments(createFile(new Path(String.format("%s/employments_%s", targetPath, id)), fileSystem)) + .withSummary(createFile(new Path(String.format("%s/summary_%s", targetPath, id)), fileSystem)) + .withWorks(createFile(new Path(String.format("%s/works_%s", targetPath, id)), fileSystem)) + .withAccessToken(accessToken) + .withBlockingQueue(queue) + .build(); + } + + public void readTar(FileSystem fileSystem, final String accessToken, final String apiURL, final String targetPath, + final String startDate) throws Exception { + final HttpURLConnection urlConn = (HttpURLConnection) new URL(apiURL).openConnection(); + final HttpClientParams clientParams = new HttpClientParams(); + urlConn.setInstanceFollowRedirects(false); + urlConn.setReadTimeout(clientParams.getReadTimeOut() * 1000); + urlConn.setConnectTimeout(clientParams.getConnectTimeOut() * 1000); + if (urlConn.getResponseCode() > 199 && urlConn.getResponseCode() < 300) { + InputStream input = urlConn.getInputStream(); + + Path hdfsWritePath = new Path("/tmp/orcid_updates.tar.gz"); + final FSDataOutputStream fsDataOutputStream = fileSystem.create(hdfsWritePath, true); + IOUtils.copy(input, fsDataOutputStream); + fsDataOutputStream.flush(); + fsDataOutputStream.close(); + FSDataInputStream updateFile = fileSystem.open(hdfsWritePath); + TarArchiveInputStream tais = new TarArchiveInputStream(new GzipCompressorInputStream( + new BufferedInputStream( + updateFile.getWrappedStream()))); + TarArchiveEntry entry; + + BlockingQueue queue = new ArrayBlockingQueue(3000); + final List workers = new ArrayList<>(); + for (int i = 0; i < 22; i++) { + workers.add(createWorker("" + i, targetPath, queue, accessToken, fileSystem)); + } + workers.forEach(Thread::start); + + while ((entry = tais.getNextTarEntry()) != null) { + + if (entry.isFile()) { + + BufferedReader br = new BufferedReader(new InputStreamReader(tais)); + System.out.println(br.readLine()); + br + .lines() + .map(l -> l.split(",")) + .filter(s -> StringUtils.compare(s[3].substring(0, 10), startDate) > 0) + .map(s -> s[0]) + .forEach(s -> { + try { + queue.put(s); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }); + + } + } + + for (int i = 0; i < 22; i++) { + queue.put(ORCIDWorker.JOB_COMPLETE); + } + for (ORCIDWorker worker : workers) { + worker.join(); + } + } + + } +} 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 index 159b8a5fc..ea108821e 100644 --- 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 @@ -1,11 +1,15 @@ package eu.dnetlib.dhp.collection.orcid; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; +import java.util.*; +import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; +import org.dom4j.Document; +import org.dom4j.DocumentFactory; +import org.dom4j.DocumentHelper; +import org.dom4j.Node; +import org.jetbrains.annotations.NotNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,8 +44,8 @@ public class OrcidParser { 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_EMPLOYMENT = "employment"; + private static final String NS_EMPLOYMENT_URL = "http://www.orcid.org/ns/employment"; private static final String NS_EXTERNAL = "external-identifier"; private static final String NS_EXTERNAL_URL = "http://www.orcid.org/ns/external-identifier"; @@ -61,6 +65,7 @@ public class OrcidParser { ap.declareXPathNameSpace(NS_WORK, NS_WORK_URL); ap.declareXPathNameSpace(NS_EXTERNAL, NS_EXTERNAL_URL); ap.declareXPathNameSpace(NS_ACTIVITIES, NS_ACTIVITIES_URL); + ap.declareXPathNameSpace(NS_EMPLOYMENT, NS_EMPLOYMENT_URL); } public Author parseSummary(final String xml) { @@ -70,13 +75,15 @@ public class OrcidParser { generateParsedDocument(xml); List recordNodes = VtdUtilityParser .getTextValuesWithAttributes( - ap, vn, "//record:record", Arrays.asList("path")); + ap, vn, "//record:record", Collections.singletonList("path")); if (!recordNodes.isEmpty()) { final String oid = (recordNodes.get(0).getAttributes().get("path")).substring(1); author.setOrcid(oid); } else { return null; } + final String ltm = VtdUtilityParser.getSingleValue(ap, vn, "//common:last-modified-date"); + author.setLastModifiedDate(ltm); List personNodes = VtdUtilityParser .getTextValuesWithAttributes( ap, vn, "//person:name", Arrays.asList("visibility")); @@ -129,6 +136,64 @@ public class OrcidParser { } } + public List parseWorks(final String xml) { + + try { + String oid; + + generateParsedDocument(xml); + List workNodes = VtdUtilityParser + .getTextValuesWithAttributes(ap, vn, "//activities:works", Arrays.asList("path", "visibility")); + if (!workNodes.isEmpty()) { + oid = (workNodes.get(0).getAttributes().get("path")).split("/")[1]; + + } else { + return null; + } + final List works = new ArrayList<>(); + ap.selectXPath("//work:work-summary"); + + while (ap.evalXPath() != -1) { + final Work work = new Work(); + work.setOrcid(oid); + final AutoPilot ap1 = new AutoPilot(ap.getNav()); + ap1.selectXPath("./work:title/common:title"); + while (ap1.evalXPath() != -1) { + int it = vn.getText(); + work.setTitle(vn.toNormalizedString(it)); + } + ap1.selectXPath(".//common:external-id"); + while (ap1.evalXPath() != -1) { + final Pid pid = new Pid(); + + final AutoPilot ap2 = new AutoPilot(ap1.getNav()); + + ap2.selectXPath("./common:external-id-type"); + while (ap2.evalXPath() != -1) { + int it = vn.getText(); + pid.setSchema(vn.toNormalizedString(it)); + } + ap2.selectXPath("./common:external-id-value"); + while (ap2.evalXPath() != -1) { + int it = vn.getText(); + pid.setValue(vn.toNormalizedString(it)); + } + + work.addPid(pid); + } + + works.add(work); + } + return works; + + } catch (Throwable e) { + log.error("Error on parsing {}", xml); + log.error(e.getMessage()); + return null; + } + + } + public Work parseWork(final String xml) { try { @@ -176,11 +241,15 @@ public class OrcidParser { } private String extractEmploymentDate(final String xpath) throws Exception { + return extractEmploymentDate(xpath, ap); + } - ap.selectXPath(xpath); + private String extractEmploymentDate(final String xpath, AutoPilot pp) throws Exception { + + pp.selectXPath(xpath); StringBuilder sb = new StringBuilder(); - while (ap.evalXPath() != -1) { - final AutoPilot ap1 = new AutoPilot(ap.getNav()); + while (pp.evalXPath() != -1) { + final AutoPilot ap1 = new AutoPilot(pp.getNav()); ap1.selectXPath("./common:year"); while (ap1.evalXPath() != -1) { int it = vn.getText(); @@ -203,6 +272,104 @@ public class OrcidParser { } + public List parseEmployments(final String xml) { + try { + String oid; + Map nsContext = getNameSpaceMap(); + DocumentFactory.getInstance().setXPathNamespaceURIs(nsContext); + Document doc = DocumentHelper.parseText(xml); + oid = doc.valueOf("//activities:employments/@path"); + if (oid == null || StringUtils.isEmpty(oid)) + return null; + final String orcid = oid.split("/")[1]; + + List nodes = doc.selectNodes("//employment:employment-summary"); + return nodes.stream().map(n -> { + final Employment e = new Employment(); + e.setOrcid(orcid); + + final String depName = n.valueOf(".//common:department-name"); + if (StringUtils.isNotBlank(depName)) + e.setDepartmentName(depName); + final String roleTitle = n.valueOf(".//common:role-title"); + e.setRoleTitle(roleTitle); + final String organizationName = n.valueOf(".//common:organization/common:name"); + if (StringUtils.isEmpty(e.getDepartmentName())) + e.setDepartmentName(organizationName); + final Pid p = new Pid(); + final String pid = n + .valueOf( + "./common:organization/common:disambiguated-organization/common:disambiguated-organization-identifier"); + p.setValue(pid); + final String pidType = n + .valueOf("./common:organization/common:disambiguated-organization/common:disambiguation-source"); + p.setSchema(pidType); + e.setAffiliationId(p); + + final StringBuilder aDate = new StringBuilder(); + final String sy = n.valueOf("./common:start-date/common:year"); + if (StringUtils.isNotBlank(sy)) { + aDate.append(sy); + final String sm = n.valueOf("./common:start-date/common:month"); + final String sd = n.valueOf("./common:start-date/common:day"); + aDate.append("-"); + if (StringUtils.isNotBlank(sm)) + aDate.append(sm); + else + aDate.append("01"); + aDate.append("-"); + if (StringUtils.isNotBlank(sd)) + aDate.append(sd); + else + aDate.append("01"); + e.setEndDate(aDate.toString()); + } + + final String ey = n.valueOf("./common:end-date/common:year"); + if (StringUtils.isNotBlank(ey)) { + aDate.append(ey); + final String em = n.valueOf("./common:end-date/common:month"); + final String ed = n.valueOf("./common:end-date/common:day"); + aDate.append("-"); + if (StringUtils.isNotBlank(em)) + aDate.append(em); + else + aDate.append("01"); + aDate.append("-"); + if (StringUtils.isNotBlank(ed)) + aDate.append(ed); + else + aDate.append("01"); + e.setEndDate(aDate.toString()); + } + + return e; + + }).collect(Collectors.toList()); + } catch (Throwable e) { + log.error("Error on parsing {}", xml); + log.error(e.getMessage()); + return null; + } + } + + @NotNull + private static Map getNameSpaceMap() { + Map nsContext = new HashMap<>(); + nsContext.put(NS_COMMON, NS_COMMON_URL); + nsContext.put(NS_PERSON, NS_PERSON_URL); + nsContext.put(NS_DETAILS, NS_DETAILS_URL); + nsContext.put(NS_OTHER, NS_OTHER_URL); + nsContext.put(NS_RECORD, NS_RECORD_URL); + nsContext.put(NS_ERROR, NS_ERROR_URL); + nsContext.put(NS_HISTORY, NS_HISTORY_URL); + nsContext.put(NS_WORK, NS_WORK_URL); + nsContext.put(NS_EXTERNAL, NS_EXTERNAL_URL); + nsContext.put(NS_ACTIVITIES, NS_ACTIVITIES_URL); + nsContext.put(NS_EMPLOYMENT, NS_EMPLOYMENT_URL); + return nsContext; + } + public Employment parseEmployment(final String xml) { try { final Employment employment = new Employment(); 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 index 32c321b41..df87e4333 100644 --- 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 @@ -18,6 +18,8 @@ public class Author extends ORCIDItem { private String biography; + private String lastModifiedDate; + public String getBiography() { return biography; } @@ -74,6 +76,14 @@ public class Author extends ORCIDItem { this.otherPids = otherPids; } + public String getLastModifiedDate() { + return lastModifiedDate; + } + + public void setLastModifiedDate(String lastModifiedDate) { + this.lastModifiedDate = lastModifiedDate; + } + public void addOtherPid(final Pid pid) { if (otherPids == null) diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java index a19ca5c68..97d2d2585 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/CollectorPlugin.java @@ -10,7 +10,8 @@ import eu.dnetlib.dhp.common.collection.CollectorException; public interface CollectorPlugin { enum NAME { - oai, other, rest_json2xml, file, fileGzip; + + oai, other, rest_json2xml, file, fileGzip, baseDump; public enum OTHER_NAME { mdstore_mongodb_dump, mdstore_mongodb diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/base/BaseCollectorIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/base/BaseCollectorIterator.java new file mode 100644 index 000000000..c41cc8f0f --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/base/BaseCollectorIterator.java @@ -0,0 +1,171 @@ + +package eu.dnetlib.dhp.collection.plugin.base; + +import java.io.BufferedInputStream; +import java.io.ByteArrayInputStream; +import java.io.InputStream; +import java.io.StringWriter; +import java.util.Iterator; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; + +import javax.xml.stream.XMLEventReader; +import javax.xml.stream.XMLEventWriter; +import javax.xml.stream.XMLInputFactory; +import javax.xml.stream.XMLOutputFactory; +import javax.xml.stream.events.EndElement; +import javax.xml.stream.events.StartElement; +import javax.xml.stream.events.XMLEvent; + +import org.apache.commons.compress.archivers.tar.TarArchiveEntry; +import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; +import org.apache.commons.compress.compressors.CompressorInputStream; +import org.apache.commons.compress.compressors.CompressorStreamFactory; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.common.aggregation.AggregatorReport; + +public class BaseCollectorIterator implements Iterator { + + private String nextElement; + + private final BlockingQueue queue = new LinkedBlockingQueue<>(100); + + private static final Logger log = LoggerFactory.getLogger(BaseCollectorIterator.class); + + private static final String END_ELEM = "__END__"; + + public BaseCollectorIterator(final FileSystem fs, final Path filePath, final AggregatorReport report) { + new Thread(() -> importHadoopFile(fs, filePath, report)).start(); + try { + this.nextElement = this.queue.take(); + } catch (final InterruptedException e) { + throw new RuntimeException(e); + } + } + + protected BaseCollectorIterator(final String resourcePath, final AggregatorReport report) { + new Thread(() -> importTestFile(resourcePath, report)).start(); + try { + this.nextElement = this.queue.take(); + } catch (final InterruptedException e) { + throw new RuntimeException(e); + } + } + + @Override + public synchronized boolean hasNext() { + return (this.nextElement != null) & !END_ELEM.equals(this.nextElement); + } + + @Override + public synchronized String next() { + try { + return END_ELEM.equals(this.nextElement) ? null : this.nextElement; + } finally { + try { + this.nextElement = this.queue.take(); + } catch (final InterruptedException e) { + throw new RuntimeException(e); + } + } + + } + + private void importHadoopFile(final FileSystem fs, final Path filePath, final AggregatorReport report) { + log.info("I start to read the TAR stream"); + + try (InputStream origInputStream = fs.open(filePath); + final TarArchiveInputStream tarInputStream = new TarArchiveInputStream(origInputStream)) { + importTarStream(tarInputStream, report); + } catch (final Throwable e) { + throw new RuntimeException("Error processing BASE records", e); + } + } + + private void importTestFile(final String resourcePath, final AggregatorReport report) { + try (final InputStream origInputStream = BaseCollectorIterator.class.getResourceAsStream(resourcePath); + final TarArchiveInputStream tarInputStream = new TarArchiveInputStream(origInputStream)) { + importTarStream(tarInputStream, report); + } catch (final Throwable e) { + throw new RuntimeException("Error processing BASE records", e); + } + } + + private void importTarStream(final TarArchiveInputStream tarInputStream, final AggregatorReport report) { + long count = 0; + + final XMLInputFactory xmlInputFactory = XMLInputFactory.newInstance(); + final XMLOutputFactory xmlOutputFactory = XMLOutputFactory.newInstance(); + + try { + TarArchiveEntry entry; + while ((entry = (TarArchiveEntry) tarInputStream.getNextEntry()) != null) { + final String name = entry.getName(); + + if (!entry.isDirectory() && name.contains("ListRecords") && name.endsWith(".bz2")) { + + log.info("Processing file (BZIP): " + name); + + final byte[] bzipData = new byte[(int) entry.getSize()]; + IOUtils.readFully(tarInputStream, bzipData); + + try (InputStream bzipIs = new ByteArrayInputStream(bzipData); + final BufferedInputStream bzipBis = new BufferedInputStream(bzipIs); + final CompressorInputStream bzipInput = new CompressorStreamFactory() + .createCompressorInputStream(bzipBis)) { + + final XMLEventReader reader = xmlInputFactory.createXMLEventReader(bzipInput); + + XMLEventWriter eventWriter = null; + StringWriter xmlWriter = null; + + while (reader.hasNext()) { + final XMLEvent nextEvent = reader.nextEvent(); + + if (nextEvent.isStartElement()) { + final StartElement startElement = nextEvent.asStartElement(); + if ("record".equals(startElement.getName().getLocalPart())) { + xmlWriter = new StringWriter(); + eventWriter = xmlOutputFactory.createXMLEventWriter(xmlWriter); + } + } + + if (eventWriter != null) { + eventWriter.add(nextEvent); + } + + if (nextEvent.isEndElement()) { + final EndElement endElement = nextEvent.asEndElement(); + if ("record".equals(endElement.getName().getLocalPart())) { + eventWriter.flush(); + eventWriter.close(); + + this.queue.put(xmlWriter.toString()); + + eventWriter = null; + xmlWriter = null; + count++; + } + } + + } + } + } + } + + this.queue.put(END_ELEM); // TO INDICATE THE END OF THE QUEUE + } catch (final Throwable e) { + log.error("Error processing BASE records", e); + report.put(e.getClass().getName(), e.getMessage()); + throw new RuntimeException("Error processing BASE records", e); + } finally { + log.info("Total records (written in queue): " + count); + } + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/base/BaseCollectorPlugin.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/base/BaseCollectorPlugin.java new file mode 100644 index 000000000..42341b406 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/base/BaseCollectorPlugin.java @@ -0,0 +1,159 @@ + +package eu.dnetlib.dhp.collection.plugin.base; + +import java.io.IOException; +import java.sql.SQLException; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Optional; +import java.util.Set; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.dom4j.Document; +import org.dom4j.DocumentException; +import org.dom4j.DocumentHelper; +import org.dom4j.Node; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.collection.ApiDescriptor; +import eu.dnetlib.dhp.collection.plugin.CollectorPlugin; +import eu.dnetlib.dhp.collection.plugin.file.AbstractSplittedRecordPlugin; +import eu.dnetlib.dhp.common.DbClient; +import eu.dnetlib.dhp.common.aggregation.AggregatorReport; +import eu.dnetlib.dhp.common.collection.CollectorException; + +public class BaseCollectorPlugin implements CollectorPlugin { + + private final FileSystem fs; + + private static final Logger log = LoggerFactory.getLogger(AbstractSplittedRecordPlugin.class); + + // MAPPING AND FILTERING ARE DEFINED HERE: + // https://docs.google.com/document/d/1Aj-ZAV11b44MCrAAUCPiS2TUlXb6PnJEu1utCMAcCOU/edit + + public BaseCollectorPlugin(final FileSystem fs) { + this.fs = fs; + } + + @Override + public Stream collect(final ApiDescriptor api, final AggregatorReport report) throws CollectorException { + // the path of the dump file on HDFS + // http://oai.base-search.net/initial_load/base_oaipmh_dump-current.tar + // it could be downloaded from iis-cdh5-test-gw.ocean.icm.edu.pl and then copied on HDFS + final Path filePath = Optional + .ofNullable(api.getBaseUrl()) + .map(Path::new) + .orElseThrow(() -> new CollectorException("missing baseUrl")); + + // get the parameters for the connection to the OpenAIRE database. + // the database is used to obtain the list of the datasources that the plugin will collect + final String dbUrl = api.getParams().get("dbUrl"); + final String dbUser = api.getParams().get("dbUser"); + final String dbPassword = api.getParams().get("dbPassword"); + + // the types(comma separated, empty value for all) that the plugin will collect, + // the types should be expressed in the format of the normalized types of BASE (for example 1,121,...) + final String acceptedNormTypesString = api.getParams().get("acceptedNormTypes"); + + log.info("baseUrl: {}", filePath); + log.info("dbUrl: {}", dbUrl); + log.info("dbUser: {}", dbUser); + log.info("dbPassword: {}", "***"); + log.info("acceptedNormTypes: {}", acceptedNormTypesString); + + try { + if (!this.fs.exists(filePath)) { + throw new CollectorException("path does not exist: " + filePath); + } + } catch (final Throwable e) { + throw new CollectorException(e); + } + + final Set acceptedOpendoarIds = findAcceptedOpendoarIds(dbUrl, dbUser, dbPassword); + + final Set acceptedNormTypes = new HashSet<>(); + if (StringUtils.isNotBlank(acceptedNormTypesString)) { + for (final String s : StringUtils.split(acceptedNormTypesString, ",")) { + if (StringUtils.isNotBlank(s)) { + acceptedNormTypes.add(s.trim()); + } + } + } + + final Iterator iterator = new BaseCollectorIterator(this.fs, filePath, report); + final Spliterator spliterator = Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED); + return StreamSupport + .stream(spliterator, false) + .filter(doc -> filterXml(doc, acceptedOpendoarIds, acceptedNormTypes)); + } + + private Set findAcceptedOpendoarIds(final String dbUrl, final String dbUser, final String dbPassword) + throws CollectorException { + final Set accepted = new HashSet<>(); + + try (final DbClient dbClient = new DbClient(dbUrl, dbUser, dbPassword)) { + + final String sql = IOUtils + .toString( + getClass().getResourceAsStream("/eu/dnetlib/dhp/collection/plugin/base/sql/opendoar-accepted.sql")); + + dbClient.processResults(sql, row -> { + try { + final String dsId = row.getString("id"); + log.info("Accepted Datasource: " + dsId); + accepted.add(dsId); + } catch (final SQLException e) { + log.error("Error in SQL", e); + throw new RuntimeException("Error in SQL", e); + } + }); + + } catch (final IOException e) { + log.error("Error accessong SQL", e); + throw new CollectorException("Error accessong SQL", e); + } + + log.info("Accepted Datasources (TOTAL): " + accepted.size()); + + return accepted; + } + + protected static boolean filterXml(final String xml, + final Set acceptedOpendoarIds, + final Set acceptedNormTypes) { + try { + + final Document doc = DocumentHelper.parseText(xml); + + final String id = doc.valueOf("//*[local-name()='collection']/@opendoar_id").trim(); + + if (StringUtils.isBlank(id) || !acceptedOpendoarIds.contains("opendoar____::" + id)) { + return false; + } + + if (acceptedNormTypes.isEmpty()) { + return true; + } + + for (final Object s : doc.selectNodes("//*[local-name()='typenorm']")) { + if (acceptedNormTypes.contains(((Node) s).getText().trim())) { + return true; + } + } + + return false; + } catch (final DocumentException e) { + log.error("Error parsing document", e); + throw new RuntimeException("Error parsing document", e); + } + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/rest/RestIterator.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/rest/RestIterator.java index e4bad2f8d..1107bcf46 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/rest/RestIterator.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/rest/RestIterator.java @@ -52,8 +52,6 @@ public class RestIterator implements Iterator { private final String BASIC = "basic"; - private final JsonUtils jsonUtils; - private final String baseUrl; private final String resumptionType; private final String resumptionParam; @@ -106,7 +104,6 @@ public class RestIterator implements Iterator { final String resultOutputFormat) { this.clientParams = clientParams; - this.jsonUtils = new JsonUtils(); this.baseUrl = baseUrl; this.resumptionType = resumptionType; this.resumptionParam = resumptionParam; @@ -126,6 +123,7 @@ public class RestIterator implements Iterator { } catch (Exception e) { throw new IllegalStateException("xml transformation init failed: " + e.getMessage()); } + initQueue(); } @@ -190,7 +188,7 @@ public class RestIterator implements Iterator { String resultJson; String resultXml = ""; String nextQuery = ""; - String emptyXml = resultXml + "<" + JsonUtils.wrapName + ">"; + String emptyXml = resultXml + "<" + JsonUtils.XML_WRAP_TAG + ">"; Node resultNode = null; NodeList nodeList = null; String qUrlArgument = ""; @@ -231,7 +229,7 @@ public class RestIterator implements Iterator { resultStream = theHttpInputStream; if ("json".equals(resultOutputFormat)) { resultJson = IOUtils.toString(resultStream, StandardCharsets.UTF_8); - resultXml = jsonUtils.convertToXML(resultJson); + resultXml = JsonUtils.convertToXML(resultJson); resultStream = IOUtils.toInputStream(resultXml, UTF_8); } diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/utils/JsonUtils.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/utils/JsonUtils.java index 15401e223..1bdc8b138 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/utils/JsonUtils.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/collection/plugin/utils/JsonUtils.java @@ -3,82 +3,142 @@ package eu.dnetlib.dhp.collection.plugin.utils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.json.JSONArray; +import org.json.JSONObject; public class JsonUtils { + public static final String XML_WRAP_TAG = "recordWrap"; + private static final String XML_HEADER = ""; + private static final String INVALID_XMLTAG_CHARS = "!\"#$%&'()*+,/;<=>?@[\\]^`{|}~,"; private static final Log log = LogFactory.getLog(JsonUtils.class); - public static final String wrapName = "recordWrap"; - /** - * convert in JSON-KeyName 'whitespace(s)' to '_' and '/' to '_', '(' and ')' to '' + * cleanup in JSON-KeyName * check W3C XML syntax: https://www.w3.org/TR/2006/REC-xml11-20060816/#sec-starttags for valid tag names * and work-around for the JSON to XML converting of org.json.XML-package. * - * known bugs: doesn't prevent "key name":" ["sexy name",": penari","erotic dance"], - * - * @param jsonInput - * @return convertedJsonKeynameOutput + * @param input + * @return converted json object */ - public String syntaxConvertJsonKeyNames(String jsonInput) { - - log.trace("before convertJsonKeyNames: " + jsonInput); - // pre-clean json - rid spaces of element names (misinterpreted as elements with attributes in xml) - // replace ' 's in JSON Namens with '_' - while (jsonInput.matches(".*\"([^\"]*)\\s+([^\"]*)\":.*")) { - jsonInput = jsonInput.replaceAll("\"([^\"]*)\\s+([^\"]*)\":", "\"$1_$2\":"); + public static JSONObject cleanJsonObject(final JSONObject input) { + if (null == input) { + return null; } - // replace forward-slash (sign '/' ) in JSON Names with '_' - while (jsonInput.matches(".*\"([^\"]*)/([^\"]*)\":.*")) { - jsonInput = jsonInput.replaceAll("\"([^\"]*)/([^\"]*)\":", "\"$1_$2\":"); + JSONObject result = new JSONObject(); + + for (String key : input.keySet()) { + Object value = input.opt(key); + if (value != null) { + result.put(cleanKey(key), cleanValue(value)); + } } - // replace '(' in JSON Names with '' - while (jsonInput.matches(".*\"([^\"]*)[(]([^\"]*)\":.*")) { - jsonInput = jsonInput.replaceAll("\"([^\"]*)[(]([^\"]*)\":", "\"$1$2\":"); - } - - // replace ')' in JSON Names with '' - while (jsonInput.matches(".*\"([^\"]*)[)]([^\"]*)\":.*")) { - jsonInput = jsonInput.replaceAll("\"([^\"]*)[)]([^\"]*)\":", "\"$1$2\":"); - } - - // add prefix of startNumbers in JSON Keynames with 'n_' - while (jsonInput.matches(".*\"([^\"][0-9])([^\"]*)\":.*")) { - jsonInput = jsonInput.replaceAll("\"([^\"][0-9])([^\"]*)\":", "\"n_$1$2\":"); - } - // add prefix of only numbers in JSON Keynames with 'm_' - while (jsonInput.matches(".*\"([0-9]+)\":.*")) { - jsonInput = jsonInput.replaceAll("\"([0-9]+)\":", "\"m_$1\":"); - } - - // replace ':' between number like '2018-08-28T11:05:00Z' in JSON keynames with '' - while (jsonInput.matches(".*\"([^\"]*[0-9]):([0-9][^\"]*)\":.*")) { - jsonInput = jsonInput.replaceAll("\"([^\"]*[0-9]):([0-9][^\"]*)\":", "\"$1$2\":"); - } - - // replace ',' in JSON Keynames with '.' to prevent , in xml tagnames. - // while (jsonInput.matches(".*\"([^\"]*),([^\"]*)\":.*")) { - // jsonInput = jsonInput.replaceAll("\"([^\"]*),([^\"]*)\":", "\"$1.$2\":"); - // } - - // replace '=' in JSON Keynames with '-' - while (jsonInput.matches(".*\"([^\"]*)=([^\"]*)\":.*")) { - jsonInput = jsonInput.replaceAll("\"([^\"]*)=([^\"]*)\":", "\"$1-$2\":"); - } - - log.trace("after syntaxConvertJsonKeyNames: " + jsonInput); - return jsonInput; + return result; } - public String convertToXML(final String jsonRecord) { - String resultXml = ""; - org.json.JSONObject jsonObject = new org.json.JSONObject(syntaxConvertJsonKeyNames(jsonRecord)); - resultXml += org.json.XML.toString(jsonObject, wrapName); // wrap xml in single root element - log.trace("before inputStream: " + resultXml); - resultXml = XmlCleaner.cleanAllEntities(resultXml); - log.trace("after cleaning: " + resultXml); - return resultXml; + private static Object cleanValue(Object object) { + if (object instanceof JSONObject) { + return cleanJsonObject((JSONObject) object); + } else if (object instanceof JSONArray) { + JSONArray array = (JSONArray) object; + JSONArray res = new JSONArray(); + + for (int i = array.length() - 1; i >= 0; i--) { + res.put(i, cleanValue(array.opt(i))); + } + return res; + } else if (object instanceof String) { + String value = (String) object; + + // XML 1.0 Allowed characters + // Char ::= #x9 | #xA | #xD | [#x20-#xD7FF] | [#xE000-#xFFFD] | [#x10000-#x10FFFF] + + return value + .codePoints() + .filter( + cp -> cp == 0x9 || cp == 0xA || cp == 0xD || (cp >= 0x20 && cp <= 0xD7FF) + || (cp >= 0xE000 && cp <= 0xFFFD) + || (cp >= 0x10000 && cp <= 0x10FFFF)) + .collect( + StringBuilder::new, + StringBuilder::appendCodePoint, + StringBuilder::append) + .toString(); + } + + return object; + } + + private static String cleanKey(String key) { + if (key == null || key.isEmpty()) { + return key; + } + + // xml tag cannot begin with "-", ".", or a numeric digit. + switch (key.charAt(0)) { + case '-': + case '.': + key = "_" + key.substring(1); + break; + } + + if (Character.isDigit(key.charAt(0))) { + if (key.matches("^[0-9]+$")) { + // add prefix of only numbers in JSON Keynames with 'm_' + key = "m_" + key; + } else { + // add prefix of startNumbers in JSON Keynames with 'n_' + key = "n_" + key; + } + } + + StringBuilder res = new StringBuilder(key.length()); + for (int i = 0; i < key.length(); i++) { + char c = key.charAt(i); + + // sequence of whitespaces are rendered as a single '_' + if (Character.isWhitespace(c)) { + while (i + 1 < key.length() && Character.isWhitespace(key.charAt(i + 1))) { + i++; + } + res.append('_'); + } + // remove invalid chars for xml tags with the expception of '=' and '/' + else if (INVALID_XMLTAG_CHARS.indexOf(c) >= 0) { + switch (c) { + case '=': + res.append('-'); + break; + case '/': + res.append('_'); + break; + default: + break; + } + // nothing + } + // all other chars are kept + else { + res.append(c); + } + } + + return res.toString(); + } + + static public String convertToXML(final String jsonRecord) { + if (log.isTraceEnabled()) { + log.trace("input json: " + jsonRecord); + } + + JSONObject jsonObject = cleanJsonObject(new org.json.JSONObject(jsonRecord)); + String res = XML_HEADER + org.json.XML.toString(jsonObject, XML_WRAP_TAG); // wrap xml in single root element + + if (log.isTraceEnabled()) { + log.trace("outout xml: " + res); + } + return res; } } diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/apply_orcid_table_parameter.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/apply_orcid_table_parameter.json new file mode 100644 index 000000000..3d3374c74 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/apply_orcid_table_parameter.json @@ -0,0 +1,26 @@ +[ + { + "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": "g", + "paramLongName": "graphPath", + "paramDescription": "the PATH of the current graph path", + "paramRequired": true + }, + { + "paramName": "u", + "paramLongName": "updatePath", + "paramDescription": "the PATH of the current graph update path", + "paramRequired": true + } + ] \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/download_orcid_update_parameter.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/download_orcid_update_parameter.json new file mode 100644 index 000000000..48b37f85d --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/download_orcid_update_parameter.json @@ -0,0 +1,37 @@ +[ { + "paramName": "n", + "paramLongName": "namenode", + "paramDescription": "the Name Node URI", + "paramRequired": true + }, + { + "paramName": "m", + "paramLongName": "master", + "paramDescription": "the master name", + "paramRequired": true + }, + { + "paramName": "t", + "paramLongName": "targetPath", + "paramDescription": "the target PATH where download the files", + "paramRequired": true + }, + { + "paramName": "a", + "paramLongName": "apiURL", + "paramDescription": "the URL to download the tar file", + "paramRequired": true + }, + { + "paramName": "g", + "paramLongName": "graphPath", + "paramDescription": "the path of the input graph", + "paramRequired": true + }, + { + "paramName": "at", + "paramLongName": "accessToken", + "paramDescription": "the accessToken to contact API", + "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 index 01d81ea97..d088e8c77 100644 --- 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 @@ -16,6 +16,12 @@ "paramLongName": "sourcePath", "paramDescription": "the PATH of the ORCID sequence file", "paramRequired": true + }, + { + "paramName": "fu", + "paramLongName": "fromUpdate", + "paramDescription": "whether we have to generate table from dump or from update", + "paramRequired": false } ] \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/update/oozie_app/config-default.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/update/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/update/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/update/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/update/oozie_app/workflow.xml new file mode 100644 index 000000000..e46e303a9 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/orcid/update/oozie_app/workflow.xml @@ -0,0 +1,114 @@ + + + + graphPath + the path to store the original ORCID dump + + + targetPath + the path to store the original ORCID dump + + + apiURL + http://74804fb637bd8e2fba5b-e0a029c2f87486cddec3b416996a6057.r3.cf1.rackcdn.com/last_modified.csv.tar + The URL of the update CSV list + + + accessToken + The access token + + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + yarn + cluster + Check Latest Orcid and Download updates + eu.dnetlib.dhp.collection.orcid.OrcidGetUpdatesFile + 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} + + --masteryarn + --namenode${nameNode} + --graphPath${graphPath} + --targetPath${targetPath} + --apiURL${apiURL} + --accessToken${accessToken} + + + + + + + + + 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} + --targetPath${targetPath}/updateTable + --fromUpdatetrue + --masteryarn + + + + + + + + yarn + cluster + Update ORCID Tables + eu.dnetlib.dhp.collection.orcid.SparkApplyUpdate + 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} + + --graphPath${graphPath} + --updatePath${targetPath}/updateTable + --targetPath${targetPath}/newTable + --masteryarn + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/plugin/base/sql/base.sql b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/plugin/base/sql/base.sql new file mode 100644 index 000000000..9a3012949 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/plugin/base/sql/base.sql @@ -0,0 +1,114 @@ +BEGIN; + +INSERT INTO dsm_services( + _dnet_resource_identifier_, + id, + officialname, + englishname, + namespaceprefix, + websiteurl, + logourl, + platform, + contactemail, + collectedfrom, + provenanceaction, + _typology_to_remove_, + eosc_type, + eosc_datasource_type, + research_entity_types, + thematic +) VALUES ( + 'openaire____::base_search', + 'openaire____::base_search', + 'Bielefeld Academic Search Engine (BASE)', + 'Bielefeld Academic Search Engine (BASE)', + 'base_search_', + 'https://www.base-search.net', + 'https://www.base-search.net/about/download/logo_224x57_white.gif', + 'BASE', + 'openaire-helpdesk@uni-bielefeld.de', + 'infrastruct_::openaire', + 'user:insert', + 'aggregator::pubsrepository::unknown', + 'Data Source', + 'Aggregator', + ARRAY['Research Products'], + false +); + +INSERT INTO dsm_service_organization( + _dnet_resource_identifier_, + organization, + service +) VALUES ( + 'fairsharing_::org::214@@openaire____::base_search', + 'fairsharing_::org::214', + 'openaire____::base_search' +); + +INSERT INTO dsm_api( + _dnet_resource_identifier_, + id, + service, + protocol, + baseurl, + metadata_identifier_path +) VALUES ( + 'api_________::openaire____::base_search::dump', + 'api_________::openaire____::base_search::dump', + 'openaire____::base_search', + 'baseDump', + '/user/michele.artini/base-import/base_oaipmh_dump-current.tar', + '//*[local-name()=''header'']/*[local-name()=''identifier'']' +); + + +INSERT INTO dsm_apiparams( + _dnet_resource_identifier_, + api, + param, + value +) VALUES ( + 'api_________::openaire____::base_search::dump@@dbUrl', + 'api_________::openaire____::base_search::dump', + 'dbUrl', + 'jdbc:postgresql://postgresql.services.openaire.eu:5432/dnet_openaireplus' +); + +INSERT INTO dsm_apiparams( + _dnet_resource_identifier_, + api, + param, + value +) VALUES ( + 'api_________::openaire____::base_search::dump@@dbUser', + 'api_________::openaire____::base_search::dump', + 'dbUser', + 'dnet' +); + +INSERT INTO dsm_apiparams( + _dnet_resource_identifier_, + api, + param, + value +) VALUES ( + 'api_________::openaire____::base_search::dump@@dbPassword', + 'api_________::openaire____::base_search::dump', + 'dbPassword', + '***' +); + +INSERT INTO dsm_apiparams( + _dnet_resource_identifier_, + api, + param, + value +) VALUES ( + 'api_________::openaire____::base_search::dump@@acceptedNormTypes', + 'api_________::openaire____::base_search::dump', + 'acceptedNormTypes', + '1,11,111,121,14,15,18,181,182,183,1A,6,7' +); + +COMMIT; \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/plugin/base/sql/opendoar-accepted.sql b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/plugin/base/sql/opendoar-accepted.sql new file mode 100644 index 000000000..7b3cb9610 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/plugin/base/sql/opendoar-accepted.sql @@ -0,0 +1,9 @@ +select s.id as id +from dsm_services s +where collectedfrom = 'openaire____::opendoar' +and jurisdiction = 'Institutional' +and s.id in ( + select service from dsm_api where coalesce(compatibility_override, compatibility) = 'driver' or coalesce(compatibility_override, compatibility) = 'UNKNOWN' +) and s.id not in ( + select service from dsm_api where coalesce(compatibility_override, compatibility) like '%openaire%' +); diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/plugin/base/sql/opendoar-aggregation-status.sql b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/plugin/base/sql/opendoar-aggregation-status.sql new file mode 100644 index 000000000..7aeead8f3 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/plugin/base/sql/opendoar-aggregation-status.sql @@ -0,0 +1,11 @@ +select + s.id as id, + s.jurisdiction as jurisdiction, + array_remove(array_agg(a.id || ' (compliance: ' || coalesce(a.compatibility_override, a.compatibility, 'UNKNOWN') || ')@@@' || coalesce(a.last_collection_total, 0)), NULL) as aggregations +from + dsm_services s + join dsm_api a on (s.id = a.service) +where + collectedfrom = 'openaire____::opendoar' +group by + s.id; diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/plugin/base/xml/base-types.vocabulary.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/plugin/base/xml/base-types.vocabulary.xml new file mode 100644 index 000000000..51f4ee6ca --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/plugin/base/xml/base-types.vocabulary.xml @@ -0,0 +1,180 @@ + +

+ + + + + +
+ + + base:normalized_types + base:normalized_types + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + String + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/plugin/base/xml/base2oaf.transformationRule.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/plugin/base/xml/base2oaf.transformationRule.xml new file mode 100644 index 000000000..a49687e3a --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/plugin/base/xml/base2oaf.transformationRule.xml @@ -0,0 +1,432 @@ + +
+ + + + + +
+ + + + + + + + + + +
\ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/plugin/base/xml/base2odf.transformationRule.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/plugin/base/xml/base2odf.transformationRule.xml new file mode 100644 index 000000000..0e0e41f12 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/collection/plugin/base/xml/base2odf.transformationRule.xml @@ -0,0 +1,461 @@ + +
+ + + + + +
+ + + + + + + + + + +
\ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/orcid/SparkApplyUpdate.scala b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/orcid/SparkApplyUpdate.scala new file mode 100644 index 000000000..57cb0e2e4 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/scala/eu/dnetlib/dhp/collection/orcid/SparkApplyUpdate.scala @@ -0,0 +1,120 @@ +package eu.dnetlib.dhp.collection.orcid + +import eu.dnetlib.dhp.application.AbstractScalaApplication +import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession} +import org.slf4j.{Logger, LoggerFactory} + +class SparkApplyUpdate(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: String = parser.get("graphPath") + log.info("found parameters graphPath: {}", graphPath) + val updatePath: String = parser.get("updatePath") + log.info("found parameters updatePath: {}", updatePath) + val targetPath: String = parser.get("targetPath") + log.info("found parameters targetPath: {}", targetPath) + applyTableUpdate(spark, graphPath, updatePath, targetPath) + checkUpdate(spark, graphPath, targetPath) + moveTable(spark, graphPath, targetPath) + + } + + private def moveTable(spark: SparkSession, graphPath: String, updatePath: String): Unit = { + spark.read + .load(s"$updatePath/Authors") + .repartition(1000) + .write + .mode(SaveMode.Overwrite) + .save(s"$graphPath/Authors") + spark.read + .load(s"$updatePath/Works") + .repartition(1000) + .write + .mode(SaveMode.Overwrite) + .save(s"$graphPath/Works") + spark.read + .load(s"$updatePath/Employments") + .repartition(1000) + .write + .mode(SaveMode.Overwrite) + .save(s"$graphPath/Employments") + + } + + private def updateDataset( + inputDataset: DataFrame, + idUpdate: DataFrame, + updateDataframe: DataFrame, + targetPath: String + ): Unit = { + inputDataset + .join(idUpdate, inputDataset("orcid").equalTo(idUpdate("orcid")), "leftanti") + .select(inputDataset("*")) + .unionByName(updateDataframe) + .write + .mode(SaveMode.Overwrite) + .save(targetPath) + } + + private def checkUpdate(spark: SparkSession, graphPath: String, updatePath: String): Unit = { + val totalOriginalAuthors = spark.read.load(s"$graphPath/Authors").count + val totalOriginalWorks = spark.read.load(s"$graphPath/Works").count + val totalOriginalEmployments = spark.read.load(s"$graphPath/Employments").count + val totalUpdateAuthors = spark.read.load(s"$updatePath/Authors").count + val totalUpdateWorks = spark.read.load(s"$updatePath/Works").count + val totalUpdateEmployments = spark.read.load(s"$updatePath/Employments").count + + log.info("totalOriginalAuthors: {}", totalOriginalAuthors) + log.info("totalOriginalWorks: {}", totalOriginalWorks) + log.info("totalOriginalEmployments: {}", totalOriginalEmployments) + log.info("totalUpdateAuthors: {}", totalUpdateAuthors) + log.info("totalUpdateWorks: {}", totalUpdateWorks) + log.info("totalUpdateEmployments: {}", totalUpdateEmployments) + if ( + totalUpdateAuthors < totalOriginalAuthors || totalUpdateEmployments < totalOriginalEmployments || totalUpdateWorks < totalOriginalWorks + ) + throw new RuntimeException("The updated Graph contains less elements of the original one") + + } + + private def applyTableUpdate(spark: SparkSession, graphPath: String, updatePath: String, targetPath: String): Unit = { + val orcidIDUpdate = spark.read.load(s"$updatePath/Authors").select("orcid") + updateDataset( + spark.read.load(s"$graphPath/Authors"), + orcidIDUpdate, + spark.read.load(s"$updatePath/Authors"), + s"$targetPath/Authors" + ) + updateDataset( + spark.read.load(s"$graphPath/Employments"), + orcidIDUpdate, + spark.read.load(s"$updatePath/Employments"), + s"$targetPath/Employments" + ) + updateDataset( + spark.read.load(s"$graphPath/Works"), + orcidIDUpdate, + spark.read.load(s"$updatePath/Works"), + s"$targetPath/Works" + ) + } + +} + +object SparkApplyUpdate { + + val log: Logger = LoggerFactory.getLogger(SparkGenerateORCIDTable.getClass) + + def main(args: Array[String]): Unit = { + + new SparkApplyUpdate("/eu/dnetlib/dhp/collection/orcid/apply_orcid_table_parameter.json", args, log) + .initialize() + .run() + + } +} 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 index f0c4cd214..4f617f15c 100644 --- 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 @@ -6,6 +6,7 @@ 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} +import scala.collection.JavaConverters._ class SparkGenerateORCIDTable(propertyPath: String, args: Array[String], log: Logger) extends AbstractScalaApplication(propertyPath, args, log: Logger) { @@ -18,12 +19,16 @@ class SparkGenerateORCIDTable(propertyPath: String, args: Array[String], log: Lo 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) + val fromUpdate = "true".equals(parser.get("fromUpdate")) + val sourceSummaryPath = if (fromUpdate) s"$sourcePath/summary*" else sourcePath + val sourceEmploymentsPath = if (fromUpdate) s"$sourcePath/employments*" else sourcePath + val sourceWorksPath = if (fromUpdate) s"$sourcePath/works*" else sourcePath + extractORCIDTable(spark, sourceSummaryPath, targetPath, fromUpdate) + extractORCIDEmploymentsTable(spark, sourceEmploymentsPath, targetPath, fromUpdate) + extractORCIDWorksTable(spark, sourceWorksPath, targetPath, fromUpdate) } - def extractORCIDTable(spark: SparkSession, sourcePath: String, targetPath: String): Unit = { + def extractORCIDTable(spark: SparkSession, sourcePath: String, targetPath: String, skipFilterByKey: Boolean): Unit = { val sc: SparkContext = spark.sparkContext import spark.implicits._ val df = sc @@ -32,8 +37,8 @@ class SparkGenerateORCIDTable(propertyPath: String, args: Array[String], log: Lo .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")) + val newDf = if (!skipFilterByKey) df.filter(r => r._1.contains("summaries")) else df + newDf .map { r => val p = new OrcidParser p.parseSummary(r._2) @@ -44,7 +49,12 @@ class SparkGenerateORCIDTable(propertyPath: String, args: Array[String], log: Lo .save(s"$targetPath/Authors") } - def extractORCIDWorksTable(spark: SparkSession, sourcePath: String, targetPath: String): Unit = { + def extractORCIDWorksTable( + spark: SparkSession, + sourcePath: String, + targetPath: String, + skipFilterByKey: Boolean + ): Unit = { val sc: SparkContext = spark.sparkContext import spark.implicits._ val df = sc @@ -53,19 +63,37 @@ class SparkGenerateORCIDTable(propertyPath: String, args: Array[String], log: Lo .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 => + + //We are in the case of parsing ORCID UPDATE + if (skipFilterByKey) { + df.flatMap { r => val p = new OrcidParser - p.parseWork(r._2) - } - .filter(p => p != null) - .write - .mode(SaveMode.Overwrite) - .save(s"$targetPath/Works") + p.parseWorks(r._2).asScala + }.filter(p => p != null) + .write + .mode(SaveMode.Overwrite) + .save(s"$targetPath/Works") + } + //We are in the case of parsing ORCID DUMP + else { + 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 = { + def extractORCIDEmploymentsTable( + spark: SparkSession, + sourcePath: String, + targetPath: String, + skipFilterByKey: Boolean + ): Unit = { val sc: SparkContext = spark.sparkContext import spark.implicits._ val df = sc @@ -74,16 +102,27 @@ class SparkGenerateORCIDTable(propertyPath: String, args: Array[String], log: Lo .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 => + if (skipFilterByKey) { + df.flatMap { r => val p = new OrcidParser - p.parseEmployment(r._2) - } - .filter(p => p != null) - .write - .mode(SaveMode.Overwrite) - .save(s"$targetPath/Employments") + p.parseEmployments(r._2).asScala + }.filter(p => p != null) + .write + .mode(SaveMode.Overwrite) + .save(s"$targetPath/Employments") + } + //We are in the case of parsing ORCID DUMP + else { + 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") + } } } 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 868f4e92d..f42d1d875 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 @@ -2,6 +2,7 @@ package eu.dnetlib.dhp.collection.orcid; import java.io.IOException; +import java.net.URI; import java.util.Arrays; import java.util.List; import java.util.Objects; @@ -9,7 +10,12 @@ 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.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +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.apache.spark.SparkContext; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Encoders; @@ -27,6 +33,7 @@ import com.ximpleware.XPathParseException; import eu.dnetlib.dhp.collection.orcid.model.Author; import eu.dnetlib.dhp.collection.orcid.model.ORCIDItem; +import eu.dnetlib.dhp.collection.orcid.model.Work; import eu.dnetlib.dhp.parser.utility.VtdException; public class DownloadORCIDTest { @@ -82,6 +89,34 @@ public class DownloadORCIDTest { }); } + @Test + public void testParsingOrcidUpdateEmployments() throws Exception { + final String xml = IOUtils + .toString( + Objects + .requireNonNull( + getClass().getResourceAsStream("/eu/dnetlib/dhp/collection/orcid/update_employments.xml"))); + + final OrcidParser parser = new OrcidParser(); + final ObjectMapper mapper = new ObjectMapper(); + System.out.println(mapper.writeValueAsString(parser.parseEmployments(xml))); + } + + @Test + public void testParsingOrcidUpdateWorks() throws Exception { + final String xml = IOUtils + .toString( + Objects + .requireNonNull( + getClass().getResourceAsStream("/eu/dnetlib/dhp/collection/orcid/update_work.xml"))); + + final OrcidParser parser = new OrcidParser(); + final List works = parser.parseWorks(xml); + + final ObjectMapper mapper = new ObjectMapper(); + System.out.println(mapper.writeValueAsString(works)); + } + @Test public void testParsingEmployments() throws Exception { diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/base/BaseCollectionInfo.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/base/BaseCollectionInfo.java new file mode 100644 index 000000000..06dfe45e2 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/base/BaseCollectionInfo.java @@ -0,0 +1,38 @@ + +package eu.dnetlib.dhp.collection.plugin.base; + +import java.io.Serializable; + +public class BaseCollectionInfo implements Serializable { + + private static final long serialVersionUID = 5766333937429419647L; + + private String id; + private String opendoarId; + private String rorId; + + public String getId() { + return this.id; + } + + public void setId(final String id) { + this.id = id; + } + + public String getOpendoarId() { + return this.opendoarId; + } + + public void setOpendoarId(final String opendoarId) { + this.opendoarId = opendoarId; + } + + public String getRorId() { + return this.rorId; + } + + public void setRorId(final String rorId) { + this.rorId = rorId; + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/base/BaseCollectorIteratorTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/base/BaseCollectorIteratorTest.java new file mode 100644 index 000000000..7ffe4bb87 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/base/BaseCollectorIteratorTest.java @@ -0,0 +1,184 @@ + +package eu.dnetlib.dhp.collection.plugin.base; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.dom4j.Attribute; +import org.dom4j.Document; +import org.dom4j.DocumentException; +import org.dom4j.DocumentHelper; +import org.dom4j.Element; +import org.dom4j.Node; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.common.aggregation.AggregatorReport; + +@Disabled +public class BaseCollectorIteratorTest { + + @Test + void testImportFile() throws Exception { + + long count = 0; + + final BaseCollectorIterator iterator = new BaseCollectorIterator("base-sample.tar", new AggregatorReport()); + + final Map> collections = new HashMap<>(); + final Map fields = new HashMap<>(); + final Set types = new HashSet<>(); + + while (iterator.hasNext()) { + + final Document record = DocumentHelper.parseText(iterator.next()); + + count++; + + if ((count % 1000) == 0) { + System.out.println("# Read records: " + count); + } + + // System.out.println(record.asXML()); + + for (final Object o : record.selectNodes("//*|//@*")) { + final String path = ((Node) o).getPath(); + + if (fields.containsKey(path)) { + fields.get(path).incrementAndGet(); + } else { + fields.put(path, new AtomicInteger(1)); + } + + if (o instanceof Element) { + final Element n = (Element) o; + + if ("collection".equals(n.getName())) { + final String collName = n.getText().trim(); + if (StringUtils.isNotBlank(collName) && !collections.containsKey(collName)) { + final Map collAttrs = new HashMap<>(); + for (final Object ao : n.attributes()) { + collAttrs.put(((Attribute) ao).getName(), ((Attribute) ao).getValue()); + } + collections.put(collName, collAttrs); + } + } else if ("type".equals(n.getName())) { + types.add(n.getText().trim()); + } + + } + } + + } + + final ObjectMapper mapper = new ObjectMapper(); + for (final Entry> e : collections.entrySet()) { + System.out.println(e.getKey() + ": " + mapper.writeValueAsString(e.getValue())); + + } + + for (final Entry e : fields.entrySet()) { + System.out.println(e.getKey() + ": " + e.getValue().get()); + + } + + System.out.println("TYPES: "); + for (final String s : types) { + System.out.println(s); + + } + + assertEquals(30000, count); + } + + @Test + public void testParquet() throws Exception { + + final String xml = IOUtils.toString(getClass().getResourceAsStream("record.xml")); + + final SparkSession spark = SparkSession.builder().master("local[*]").getOrCreate(); + + final List ls = new ArrayList<>(); + + for (int i = 0; i < 10; i++) { + ls.add(extractInfo(xml)); + } + + final JavaRDD rdd = JavaSparkContext + .fromSparkContext(spark.sparkContext()) + .parallelize(ls); + + final Dataset df = spark + .createDataset(rdd.rdd(), Encoders.bean(BaseRecordInfo.class)); + + df.printSchema(); + + df.show(false); + } + + private BaseRecordInfo extractInfo(final String s) { + try { + final Document record = DocumentHelper.parseText(s); + + final BaseRecordInfo info = new BaseRecordInfo(); + + final Set paths = new LinkedHashSet<>(); + final Set types = new LinkedHashSet<>(); + final List colls = new ArrayList<>(); + + for (final Object o : record.selectNodes("//*|//@*")) { + paths.add(((Node) o).getPath()); + + if (o instanceof Element) { + final Element n = (Element) o; + + final String nodeName = n.getName(); + + if ("collection".equals(nodeName)) { + final String collName = n.getText().trim(); + + if (StringUtils.isNotBlank(collName)) { + final BaseCollectionInfo coll = new BaseCollectionInfo(); + coll.setId(collName); + coll.setOpendoarId(n.valueOf("@opendoar_id").trim()); + coll.setRorId(n.valueOf("@ror_id").trim()); + colls.add(coll); + } + } else if ("type".equals(nodeName)) { + types.add("TYPE: " + n.getText().trim()); + } else if ("typenorm".equals(nodeName)) { + types.add("TYPE_NORM: " + n.getText().trim()); + } + } + } + + info.setId(record.valueOf("//*[local-name() = 'header']/*[local-name() = 'identifier']").trim()); + info.getTypes().addAll(types); + info.getPaths().addAll(paths); + info.setCollections(colls); + + return info; + } catch (final DocumentException e) { + throw new RuntimeException(e); + } + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/base/BaseCollectorPluginTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/base/BaseCollectorPluginTest.java new file mode 100644 index 000000000..d3b08f212 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/base/BaseCollectorPluginTest.java @@ -0,0 +1,32 @@ + +package eu.dnetlib.dhp.collection.plugin.base; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; + +import org.apache.commons.io.IOUtils; +import org.junit.jupiter.api.Test; + +class BaseCollectorPluginTest { + + @Test + void testFilterXml() throws Exception { + final String xml = IOUtils.toString(getClass().getResourceAsStream("record.xml")); + + final Set validIds = new HashSet<>(Arrays.asList("opendoar____::1234", "opendoar____::4567")); + final Set validTypes = new HashSet<>(Arrays.asList("1", "121")); + final Set validTypes2 = new HashSet<>(Arrays.asList("1", "11")); + + assertTrue(BaseCollectorPlugin.filterXml(xml, validIds, validTypes)); + assertTrue(BaseCollectorPlugin.filterXml(xml, validIds, new HashSet<>())); + + assertFalse(BaseCollectorPlugin.filterXml(xml, new HashSet<>(), validTypes)); + assertFalse(BaseCollectorPlugin.filterXml(xml, validIds, validTypes2)); + + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/base/BaseRecordInfo.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/base/BaseRecordInfo.java new file mode 100644 index 000000000..0fe6175a7 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/base/BaseRecordInfo.java @@ -0,0 +1,49 @@ + +package eu.dnetlib.dhp.collection.plugin.base; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +public class BaseRecordInfo implements Serializable { + + private static final long serialVersionUID = -8848232018350074593L; + + private String id; + private List collections = new ArrayList<>(); + private List paths = new ArrayList<>(); + private List types = new ArrayList<>(); + + public String getId() { + return this.id; + } + + public void setId(final String id) { + this.id = id; + } + + public List getPaths() { + return this.paths; + } + + public void setPaths(final List paths) { + this.paths = paths; + } + + public List getTypes() { + return this.types; + } + + public void setTypes(final List types) { + this.types = types; + } + + public List getCollections() { + return this.collections; + } + + public void setCollections(final List collections) { + this.collections = collections; + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/base/BaseTransfomationTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/base/BaseTransfomationTest.java new file mode 100644 index 000000000..6c0c23922 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/base/BaseTransfomationTest.java @@ -0,0 +1,94 @@ + +package eu.dnetlib.dhp.collection.plugin.base; + +import static org.junit.jupiter.api.Assertions.assertThrows; + +import java.io.IOException; + +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.util.LongAccumulator; +import org.dom4j.io.SAXReader; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.junit.jupiter.MockitoExtension; + +import eu.dnetlib.dhp.aggregation.AbstractVocabularyTest; +import eu.dnetlib.dhp.aggregation.common.AggregationCounter; +import eu.dnetlib.dhp.schema.mdstore.MetadataRecord; +import eu.dnetlib.dhp.schema.mdstore.Provenance; +import eu.dnetlib.dhp.transformation.xslt.XSLTTransformationFunction; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; + +// @Disabled +@ExtendWith(MockitoExtension.class) +public class BaseTransfomationTest extends AbstractVocabularyTest { + + private SparkConf sparkConf; + + @BeforeEach + public void setUp() throws IOException, ISLookUpException { + setUpVocabulary(); + + this.sparkConf = new SparkConf(); + this.sparkConf.setMaster("local[*]"); + this.sparkConf.set("spark.driver.host", "localhost"); + this.sparkConf.set("spark.ui.enabled", "false"); + } + + @Test + void testBase2ODF() throws Exception { + + final MetadataRecord mr = new MetadataRecord(); + mr.setProvenance(new Provenance("DSID", "DSNAME", "PREFIX")); + mr.setBody(IOUtils.toString(getClass().getResourceAsStream("record.xml"))); + + final XSLTTransformationFunction tr = loadTransformationRule("xml/base2odf.transformationRule.xml"); + + final MetadataRecord result = tr.call(mr); + + System.out.println(result.getBody()); + } + + @Test + void testBase2OAF() throws Exception { + + final MetadataRecord mr = new MetadataRecord(); + mr.setProvenance(new Provenance("DSID", "DSNAME", "PREFIX")); + mr.setBody(IOUtils.toString(getClass().getResourceAsStream("record.xml"))); + + final XSLTTransformationFunction tr = loadTransformationRule("xml/base2oaf.transformationRule.xml"); + + final MetadataRecord result = tr.call(mr); + + System.out.println(result.getBody()); + } + + @Test + void testBase2ODF_wrong_date() throws Exception { + + final MetadataRecord mr = new MetadataRecord(); + mr.setProvenance(new Provenance("DSID", "DSNAME", "PREFIX")); + mr.setBody(IOUtils.toString(getClass().getResourceAsStream("record_wrong_1.xml"))); + + final XSLTTransformationFunction tr = loadTransformationRule("xml/base2oaf.transformationRule.xml"); + + assertThrows(NullPointerException.class, () -> { + final MetadataRecord result = tr.call(mr); + System.out.println(result.getBody()); + }); + } + + private XSLTTransformationFunction loadTransformationRule(final String path) throws Exception { + final String xslt = new SAXReader() + .read(this.getClass().getResourceAsStream(path)) + .selectSingleNode("//CODE/*") + .asXML(); + + final LongAccumulator la = new LongAccumulator(); + + return new XSLTTransformationFunction(new AggregationCounter(la, la, la), xslt, 0, this.vocabularies); + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/rest/RestIteratorTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/rest/RestIteratorTest.java index 906f69dc9..e2d6ad3e7 100644 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/rest/RestIteratorTest.java +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/rest/RestIteratorTest.java @@ -9,6 +9,7 @@ import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import eu.dnetlib.dhp.common.collection.CollectorException; import eu.dnetlib.dhp.common.collection.HttpClientParams; /** @@ -37,7 +38,7 @@ public class RestIteratorTest { @Disabled @Test - public void test() { + public void test() throws CollectorException { HttpClientParams clientParams = new HttpClientParams(); diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/utils/JsonUtilsTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/utils/JsonUtilsTest.java new file mode 100644 index 000000000..59d24fdc2 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/collection/plugin/utils/JsonUtilsTest.java @@ -0,0 +1,48 @@ + +package eu.dnetlib.dhp.collection.plugin.utils; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import org.junit.jupiter.api.Test; + +class JsonUtilsTest { + + static private String wrapped(String xml) { + return "" + xml + ""; + } + + @Test + void keyStartWithDigit() { + assertEquals( + wrapped("null"), + JsonUtils.convertToXML("{\"100\" : {\"200v\" : null}}")); + } + + @Test + void keyStartWithSpecialchars() { + assertEquals( + wrapped("<_parent><_nest1><_nest2>null"), + JsonUtils.convertToXML("{\" parent\" : {\"-nest1\" : {\".nest2\" : null}}}")); + } + + @Test + void encodeArray() { + assertEquals( + wrapped("<_parent.child>1<_parent.child>2"), + JsonUtils.convertToXML("{\" parent.child\":[1, 2]}")); + } + + @Test + void arrayOfObjects() { + assertEquals( + wrapped("12"), + JsonUtils.convertToXML("{\"parent\": [{\"id\": 1}, {\"id\": 2}]}")); + } + + @Test + void removeControlCharacters() { + assertEquals( + wrapped("Test"), + JsonUtils.convertToXML("{\"100\" : {\"200v\" : \"\\u0000\\u000cTest\"}}")); + } +} diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/summary2.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/summary2.xml new file mode 100644 index 000000000..49aa117aa --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/summary2.xml @@ -0,0 +1,966 @@ + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + en + + + Direct + 2016-01-06T05:08:45.720Z + 2024-01-02T20:07:05.186Z + true + true + true + + + 2023-12-02T13:32:05.269Z + + + 2016-02-09T09:18:18.417Z + + 2016-02-09T09:18:18.416Z + 2016-02-09T09:18:18.417Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + Dr Michael Muchiri + http://www.rmit.edu.au/contact/staff-contacts/academic-staff/m/muchiri-dr-michael + + + + + 2023-12-02T13:32:05.269Z + + 2018-02-13T02:32:04.094Z + 2023-12-02T13:32:05.269Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + AU + + + 2023-12-02T13:32:05.260Z + 2023-12-02T13:32:05.260Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + SA + + + + 2023-12-02T13:31:16.269Z + + 2016-02-09T09:16:44.001Z + 2023-12-02T13:31:16.269Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + Organizational Behavior + + + 2016-02-09T09:16:27.374Z + 2023-12-02T13:31:16.269Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + Organizational Leadership + + + 2016-02-09T09:17:08.998Z + 2023-12-02T13:31:16.269Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + Organizational performance + + + 2023-12-02T13:31:16.259Z + 2023-12-02T13:31:16.259Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + Thriving at work + + + + 2018-04-10T00:49:55.386Z + + 2018-04-10T00:49:55.385Z + 2018-04-10T00:49:55.386Z + + + https://orcid.org/client/0000-0003-1377-5676 + 0000-0003-1377-5676 + orcid.org + + ResearcherID + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + ResearcherID + D-1929-2018 + http://www.researcherid.com/rid/D-1929-2018 + self + + + + + 2023-12-02T13:28:26.051Z + + + 2018-02-13T02:33:38.225Z + + 2016-02-09T06:55:21.838Z + + + 2016-02-09T06:54:39.199Z + 2016-02-09T06:55:21.838Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + Management + PhD + + 2021 + + + 2022 + 02 + 12 + + + 2003 + 03 + + + 2007 + 03 + + + University of New England + + Armidale + NSW + AU + + + 1319 + RINGGOLD + + + + + + 2018-02-13T02:33:38.225Z + + + 2016-02-09T06:57:04.181Z + 2018-02-13T02:33:38.225Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + Psychology + Master of Science (Industrial and Organizational) Psychology + + 1998 + 01 + + + 2000 + 01 + + + Universitas Gadjah Mada + + Yogyakarta + Daerah Istimewa Yogyakart + ID + + + 59166 + RINGGOLD + + + + + + 2018-02-13T02:33:35.821Z + + + 2016-02-09T06:58:59.869Z + 2018-02-13T02:33:35.821Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + Education + Bachelor of Education (Honors) + + 1988 + 03 + + + 1991 + 03 + + + Kenyatta University + + Nairobi + Nairobi + KE + + + 107864 + RINGGOLD + + + + + + + 2023-12-02T13:28:26.051Z + + 2023-12-02T13:28:26.051Z + + + 2023-12-02T13:28:26.051Z + 2023-12-02T13:28:26.051Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + Management + Associate Professor in Management + + 2023 + 08 + 20 + + + Alfaisal University + + Riyadh + SA + + + https://ror.org/00cdrtq48 + ROR + + + https://faculty.alfaisal.edu/user/mmuchiri + + + + 2016-02-09T07:00:06.052Z + + + 2016-02-09T07:00:06.052Z + 2016-02-09T07:00:06.052Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + Management + Senior Lecturer + + 2014 + 02 + + + RMIT University + + Melbourne + VIC + AU + + + 5376 + RINGGOLD + + + + + + 2016-02-09T07:01:08.398Z + + + 2016-02-09T07:01:08.398Z + 2016-02-09T07:01:08.398Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + Management + Senior Lecturer in Human Resource Management + + 2010 + 01 + + + 2014 + 02 + + + Central Queensland University + + Rockhampton + QLD + AU + + + 273488 + RINGGOLD + + + + + + 2016-02-09T07:01:47.814Z + + + 2016-02-09T07:01:47.814Z + 2016-02-09T07:01:47.814Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + Management + Lecturer in Management + + 2007 + 01 + + + 2010 + 01 + + + Central Queensland University + + Rockhampton + QLD + AU + + + 273488 + RINGGOLD + + + + + + 2018-02-13T02:33:13.213Z + + + 2016-02-09T07:03:42.180Z + 2018-02-13T02:33:13.213Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + Human Resource Development Division + Chief Human Resource Development Officer + + 2005 + 01 + + + 2007 + 01 + + + Government of Kenya Directorate of Personnel Management + + Nairobi + Nairobi + KE + + + 360256 + RINGGOLD + + + + + + 2016-02-09T07:05:02.300Z + + + 2016-02-09T07:05:02.300Z + 2016-02-09T07:05:02.300Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + Human Resource Development Division + Human Resource Development Officer + + 2001 + 01 + + + 2005 + 01 + + + Government of Kenya Directorate of Personnel Management + + Nairobi + Nairobi + KE + + + 360256 + RINGGOLD + + + + + + 2016-02-09T07:36:52.398Z + + + 2016-02-09T07:36:52.398Z + 2016-02-09T07:36:52.398Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + Public Sector Management Technical Assistance Project + Project Coordinator for Development Learning Centre + + 2002 + 08 + + + 2003 + 03 + + + Government of Kenya Directorate of Personnel Management + + Nairobi + Nairobi + KE + + + 360256 + RINGGOLD + + + + + + + 2016-02-09T09:05:27.100Z + + 2016-02-09T09:05:27.100Z + + + 2016-02-09T09:05:27.100Z + 2016-02-09T09:05:27.100Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + + A cross-country examination of Employee Wellbeing, Leadership, High Performance Work Systems and Innovative Behaviours + + grant + + 2016 + 01 + + + 2016 + 12 + + + RMIT University + + VIC + VIC + AU + + + http://dx.doi.org/10.13039/501100001780 + FUNDREF + + + + + + 2016-02-09T09:03:51.641Z + + + 2016-02-09T09:03:51.641Z + 2016-02-09T09:03:51.641Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + + Leading Safe and Thriving Organisations: An Investigation of the Relationships between Leadership, Thriving Behaviour, Authentic Followership and Safety Climate in an Australian Multinational Enterprise + + grant + + 2015 + 01 + + + 2015 + 12 + + + RMIT University + + VIC + VIC + AU + + + http://dx.doi.org/10.13039/501100001780 + FUNDREF + + + + + + 2016-02-09T09:02:28.297Z + + + 2016-02-09T09:02:28.297Z + 2016-02-09T09:02:28.297Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + + A multilevel, cross-country examination of leadership, followership and innovative behaviours in Australia and Indonesia. + + grant + + 2015 + 01 + + + 2015 + 12 + + + RMIT University + + VIC + VIC + AU + + + http://dx.doi.org/10.13039/501100001780 + FUNDREF + + + + + + 2016-02-09T09:00:51.749Z + + + 2016-02-09T09:00:51.749Z + 2016-02-09T09:00:51.749Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + + Workplace safety and positive leadership: Exploring relationships between leader behaviours, organisational climate, safety climate, safety citizenship behaviours and innovative behaviours within city councils in Victoria + + grant + + 2014 + 01 + + + 2014 + 12 + + + RMIT University + + VIC + VIC + AU + + + http://dx.doi.org/10.13039/501100001780 + FUNDREF + + + + + + 2016-02-09T07:46:44.919Z + + + 2016-02-09T07:46:44.919Z + 2016-02-09T07:46:44.919Z + + + https://orcid.org/0000-0001-6816-8350 + 0000-0001-6816-8350 + orcid.org + + + + Sustainable Business Model for Central Queensland Regional Information Systems. + + grant + + 2008 + 01 + + + 2008 + 12 + + + Department of Local Government, Planning, Sport and Recreation, Queensland, Australia + + Rockhampton + Central Queensland + AU + + + + + + + + + 2023-05-31T05:53:44.542Z + + 2023-05-31T05:53:44.542Z + + + peer-review + issn:0167-4544 + + + + 2023-02-28T06:51:52.426Z + + + source-work-id + c9bdf086-cfee-4cd9-bcfb-268cc5423248 + c9bdf086-cfee-4cd9-bcfb-268cc5423248 + + self + + + + 2023-02-28T06:51:52.426Z + 2023-02-28T06:51:52.426Z + + + https://orcid.org/client/APP-945VYTN20C7BZXYT + APP-945VYTN20C7BZXYT + orcid.org + + Springer Nature @ Editorial Manager + + reviewer + + + source-work-id + c9bdf086-cfee-4cd9-bcfb-268cc5423248 + c9bdf086-cfee-4cd9-bcfb-268cc5423248 + + self + + + review + + 2023 + + issn:0167-4544 + + Springer Nature + + New York + US + + + grid.467660.5 + GRID + + + + + + 2023-05-31T05:53:44.542Z + + + source-work-id + c442840b-5807-459d-802a-303d8ba4e25e + c442840b-5807-459d-802a-303d8ba4e25e + + self + + + + 2023-05-31T05:53:44.542Z + 2023-05-31T05:53:44.542Z + + + https://orcid.org/client/APP-945VYTN20C7BZXYT + APP-945VYTN20C7BZXYT + orcid.org + + Springer Nature @ Editorial Manager + + reviewer + + + source-work-id + c442840b-5807-459d-802a-303d8ba4e25e + c442840b-5807-459d-802a-303d8ba4e25e + + self + + + review + + 2023 + + issn:0167-4544 + + Springer Nature + + New York + US + + + grid.467660.5 + GRID + + + + + + + + + + + 2023-06-02T20:12:00.338Z + + 2023-06-02T20:12:00.338Z + + + doi + 10.4337/9781800881945.00020 + 10.4337/9781800881945.00020 + https://doi.org/10.4337/9781800881945.00020 + self + + + + 2023-05-11T21:05:54.188Z + 2023-06-02T20:12:00.338Z + + + https://orcid.org/client/0000-0001-9884-1913 + 0000-0001-9884-1913 + orcid.org + + Crossref + + + Ethical leadership as workplace innovation and enabler for employee commitment and innovative work behaviours in Vietnam + + + + doi + 10.4337/9781800881945.00020 + 10.4337/9781800881945.00020 + https://doi.org/10.4337/9781800881945.00020 + self + + + https://doi.org/10.4337/9781800881945.00020 + book-chapter + + 2023 + 05 + 26 + + + + + 2023-03-01T11:30:31.972Z + + + doi + 10.1007/s10551-022-05081-6 + 10.1007/s10551-022-05081-6 + https://doi.org/10.1007/s10551-022-05081-6 + self + + + + 2022-03-18T03:36:55.927Z + 2023-03-01T11:30:31.972Z + + + https://orcid.org/client/0000-0001-9884-1913 + 0000-0001-9884-1913 + orcid.org + + Crossref + + + Unethical Leadership: Review, Synthesis and Directions for Future Research + + + + doi + 10.1007/s10551-022-05081-6 + 10.1007/s10551-022-05081-6 + https://doi.org/10.1007/s10551-022-05081-6 + self + + + https://doi.org/10.1007/s10551-022-05081-6 + journal-article + + 2023 + 03 + + Journal of Business Ethics + + + + 2022-05-28T18:16:16.575Z + + + doi + 10.1017/jmo.2019.33 + 10.1017/jmo.2019.33 + https://doi.org/10.1017/jmo.2019.33 + self + + + + 2019-05-10T07:23:14.608Z + 2022-05-28T18:16:16.575Z + + + https://orcid.org/client/0000-0001-9884-1913 + 0000-0001-9884-1913 + orcid.org + + Crossref + + + And now for something completely different: Reframing social processes of leadership theory using positive organisational behaviour + + + + doi + 10.1017/jmo.2019.33 + 10.1017/jmo.2019.33 + https://doi.org/10.1017/jmo.2019.33 + self + + + https://doi.org/10.1017/jmo.2019.33 + journal-article + + 2019 + 05 + 09 + + Journal of Management & Organization + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/update_employments.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/update_employments.xml new file mode 100644 index 000000000..1aa8072ef --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/update_employments.xml @@ -0,0 +1,60 @@ + + + 2024-01-07T23:59:38.869Z + + 2024-01-07T23:59:38.869Z + + + 2024-01-07T23:59:38.869Z + 2024-01-07T23:59:38.869Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Tampere University + + Tampere + FI + + + https://ror.org/033003e23 + ROR + + + + + + 2019-01-03T17:00:05.658Z + + + 2017-02-26T04:46:20.917Z + 2019-01-03T17:00:05.658Z + + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + Faculty of Medicine and Health Technology + Academy Research Fellow + + Tampere University + + Tampere + FI + + + 7839 + RINGGOLD + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/update_work.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/update_work.xml new file mode 100644 index 000000000..054b8b832 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/orcid/update_work.xml @@ -0,0 +1,6234 @@ + + + 2024-01-12T23:59:50.769Z + + 2023-09-27T23:12:39.432Z + + + source-work-id + 4c0ac3a6-5a96-4fff-8655-cb80580c4680 + 4c0ac3a6-5a96-4fff-8655-cb80580c4680 + self + + + doi + 10.1007/s12021-023-09627-5 + 10.1007/s12021-023-09627-5 + self + + + eid + 2-s2.0-85158107722 + 2-s2.0-85158107722 + self + + + + 2023-05-11T23:04:41.983Z + 2023-09-27T23:12:39.432Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + CellRemorph + + + + source-work-id + 4c0ac3a6-5a96-4fff-8655-cb80580c4680 + 4c0ac3a6-5a96-4fff-8655-cb80580c4680 + self + + + eid + 2-s2.0-85158107722 + 2-s2.0-85158107722 + self + + + doi + 10.1007/s12021-023-09627-5 + 10.1007/s12021-023-09627-5 + self + + + https://researchportal.tuni.fi/en/publications/4c0ac3a6-5a96-4fff-8655-cb80580c4680 + journal-article + + 2023 + 05 + + Neuroinformatics + + + + 2023-09-26T23:08:14.737Z + + + doi + 10.1007/s12021-023-09622-w + 10.1007/s12021-023-09622-w + https://doi.org/10.1007/s12021-023-09622-w + self + + + source-work-id + 0add7569-3394-4418-84af-ba70066e9efb + 0add7569-3394-4418-84af-ba70066e9efb + self + + + eid + 2-s2.0-85150654537 + 2-s2.0-85150654537 + self + + + + 2023-03-29T02:22:33.697Z + 2023-03-29T02:22:33.697Z + + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Analysis of Network Models with Neuron-Astrocyte Interactions + + + + doi + 10.1007/s12021-023-09622-w + 10.1007/s12021-023-09622-w + https://doi.org/10.1007/s12021-023-09622-w + self + + + issn + 1539-2791 + 1539-2791 + https://portal.issn.org/resource/ISSN/1539-2791 + part-of + + + issn + 1559-0089 + 1559-0089 + https://portal.issn.org/resource/ISSN/1559-0089 + part-of + + + http://dx.doi.org/10.1007/s12021-023-09622-w + journal-article + + 2023 + 03 + 23 + + Neuroinformatics + + + 2023-03-29T23:04:09.044Z + 2023-09-26T23:08:14.737Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Analysis of network models with neuron-astrocyte interactions + + + + source-work-id + 0add7569-3394-4418-84af-ba70066e9efb + 0add7569-3394-4418-84af-ba70066e9efb + self + + + eid + 2-s2.0-85150654537 + 2-s2.0-85150654537 + self + + + doi + 10.1007/s12021-023-09622-w + 10.1007/s12021-023-09622-w + self + + + https://researchportal.tuni.fi/en/publications/0add7569-3394-4418-84af-ba70066e9efb + journal-article + + 2023 + 03 + + Neuroinformatics + + + + 2023-04-22T23:03:33.804Z + + + source-work-id + 93d91fd0-2f67-4d75-b642-90dead70b4f7 + 93d91fd0-2f67-4d75-b642-90dead70b4f7 + self + + + eid + 2-s2.0-85148306997 + 2-s2.0-85148306997 + self + + + doi + 10.1007/s10827-022-00841-9 + 10.1007/s10827-022-00841-9 + self + + + + 2023-04-13T23:02:32.786Z + 2023-04-22T23:03:33.804Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational modeling of neuron-astrocyte interactions in large neural populations using the NEST simulator + + + + source-work-id + 93d91fd0-2f67-4d75-b642-90dead70b4f7 + 93d91fd0-2f67-4d75-b642-90dead70b4f7 + self + + + eid + 2-s2.0-85148306997 + 2-s2.0-85148306997 + self + + + doi + 10.1007/s10827-022-00841-9 + 10.1007/s10827-022-00841-9 + self + + + https://researchportal.tuni.fi/en/publications/93d91fd0-2f67-4d75-b642-90dead70b4f7 + other + + 2023 + + Journal of Computational Neuroscience + + + + 2023-11-10T00:08:55.440Z + + + doi + 10.1002/glia.24419 + 10.1002/glia.24419 + self + + + source-work-id + 30765311-57a1-4ac1-a6cf-061e3c645842 + 30765311-57a1-4ac1-a6cf-061e3c645842 + self + + + eid + 2-s2.0-85164267874 + 2-s2.0-85164267874 + self + + + source-work-id + ea9acca1-3a14-41ea-9dc1-7bd942b28c74 + ea9acca1-3a14-41ea-9dc1-7bd942b28c74 + self + + + + 2023-11-10T00:08:55.434Z + 2023-11-10T00:08:55.434Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational modeling of neuron-astrocyte interactions in networks: Experiments, theory, and models + + + + source-work-id + ea9acca1-3a14-41ea-9dc1-7bd942b28c74 + ea9acca1-3a14-41ea-9dc1-7bd942b28c74 + self + + + doi + 10.1002/glia.24419 + 10.1002/glia.24419 + self + + + https://researchportal.tuni.fi/en/publications/ea9acca1-3a14-41ea-9dc1-7bd942b28c74 + other + + 2023 + + Glia + + + 2023-11-10T00:08:55.440Z + 2023-11-10T00:08:55.440Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Reconstruction of Bergmann glial morphology for whole-cell calcium simulations + + + + source-work-id + 30765311-57a1-4ac1-a6cf-061e3c645842 + 30765311-57a1-4ac1-a6cf-061e3c645842 + self + + + eid + 2-s2.0-85164267874 + 2-s2.0-85164267874 + self + + + doi + 10.1002/glia.24419 + 10.1002/glia.24419 + version-of + + + https://researchportal.tuni.fi/en/publications/30765311-57a1-4ac1-a6cf-061e3c645842 + other + + 2023 + + Glia + + + + 2023-11-10T00:08:55.426Z + + + source-work-id + 6092190f-1187-4499-bca0-44dfdb9c41b6 + 6092190f-1187-4499-bca0-44dfdb9c41b6 + self + + + + 2023-11-10T00:08:55.426Z + 2023-11-10T00:08:55.426Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational modeling of neuron-astrocyte interactions using the NEST simulator (AstroNeuronNets) + + + + source-work-id + 6092190f-1187-4499-bca0-44dfdb9c41b6 + 6092190f-1187-4499-bca0-44dfdb9c41b6 + self + + + https://researchportal.tuni.fi/en/publications/6092190f-1187-4499-bca0-44dfdb9c41b6 + conference-abstract + + 2023 + + Human Brain Project Summit 2023, Marseille, France, 28/03/23 + + + + 2023-11-10T00:08:55.423Z + + + source-work-id + 75b79d75-0f37-4e48-a46d-3680d0f9df5a + 75b79d75-0f37-4e48-a46d-3680d0f9df5a + self + + + + 2023-11-10T00:08:55.423Z + 2023-11-10T00:08:55.423Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational modelling of neuron-astrocyte interactions in the NEST simulator + + + + source-work-id + 75b79d75-0f37-4e48-a46d-3680d0f9df5a + 75b79d75-0f37-4e48-a46d-3680d0f9df5a + self + + + https://researchportal.tuni.fi/en/publications/75b79d75-0f37-4e48-a46d-3680d0f9df5a + conference-abstract + + 2023 + + 15th Göttingen Meeting of the German Neuroscience Society, Göttingen, Germany, 22/03/23 + + + + 2024-01-12T23:59:50.769Z + + + source-work-id + e22ca8b2-c61a-42bc-b434-a9f7eca4e685 + e22ca8b2-c61a-42bc-b434-a9f7eca4e685 + self + + + + 2024-01-12T23:59:50.769Z + 2024-01-12T23:59:50.769Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Integrating wet-lab data, computational models and tools in EBRAINS digital neuroscience platform for better understanding of the human brain + + + + source-work-id + e22ca8b2-c61a-42bc-b434-a9f7eca4e685 + e22ca8b2-c61a-42bc-b434-a9f7eca4e685 + self + + + https://researchportal.tuni.fi/en/publications/e22ca8b2-c61a-42bc-b434-a9f7eca4e685 + conference-abstract + + 2023 + + MET Research Day 2023, Tampere, Finland, 30/11/23 + + + + 2023-11-10T00:08:55.429Z + + + source-work-id + e5c8172f-32dc-4f99-bfec-67c620f69f71 + e5c8172f-32dc-4f99-bfec-67c620f69f71 + self + + + + 2023-11-10T00:08:55.429Z + 2023-11-10T00:08:55.429Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Interaction schemes and spatial organization of cells in computational models of neuron-astrocyte networks + + + + source-work-id + e5c8172f-32dc-4f99-bfec-67c620f69f71 + e5c8172f-32dc-4f99-bfec-67c620f69f71 + self + + + https://researchportal.tuni.fi/en/publications/e5c8172f-32dc-4f99-bfec-67c620f69f71 + conference-abstract + + 2023 + + Human Brain Project Summit 2023, Marseille, France, 28/03/23 + + + + 2022-11-03T00:14:12.019Z + + + eid + 2-s2.0-85129140475 + 2-s2.0-85129140475 + self + + + pmid + 35471536 + 35471536 + self + + + source-work-id + b540f47e-b252-46dd-82ea-ef3094e63572 + b540f47e-b252-46dd-82ea-ef3094e63572 + self + + + doi + 10.1007/978-3-030-89439-9_4 + 10.1007/978-3-030-89439-9_4 + self + + + + 2022-06-29T22:29:50.771Z + 2022-11-03T00:14:12.019Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Neuron–Glia Interactions and Brain Circuits + + + + source-work-id + b540f47e-b252-46dd-82ea-ef3094e63572 + b540f47e-b252-46dd-82ea-ef3094e63572 + self + + + eid + 2-s2.0-85129140475 + 2-s2.0-85129140475 + self + + + pmid + 35471536 + 35471536 + self + + + doi + 10.1007/978-3-030-89439-9_4 + 10.1007/978-3-030-89439-9_4 + self + + + isbn + 978-3-030-89439-9 + 9783030894399 + part-of + + + isbn + 978-3-030-89438-2 + 9783030894382 + part-of + + + https://researchportal.tuni.fi/en/publications/b540f47e-b252-46dd-82ea-ef3094e63572 + book-chapter + + 2022 + 04 + 27 + + Computational Modelling of the Brain + + + + 2022-12-14T23:34:35.217Z + + + source-work-id + 26df09ce-b3a1-4f6f-9dec-1fb4f57ab39a + 26df09ce-b3a1-4f6f-9dec-1fb4f57ab39a + self + + + + 2022-12-14T23:34:35.217Z + 2022-12-14T23:34:35.217Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Astrocytic modulation of synaptic plasticity: How to integrate biological knowledge, computational modeling, and model sensitivity analysis + + + + source-work-id + 26df09ce-b3a1-4f6f-9dec-1fb4f57ab39a + 26df09ce-b3a1-4f6f-9dec-1fb4f57ab39a + self + + + https://researchportal.tuni.fi/en/publications/26df09ce-b3a1-4f6f-9dec-1fb4f57ab39a + conference-abstract + + 2022 + + 51st Annual Meeting of the Society for Neuroscience (SFN 2022), San Diego, United States, 12/11/22 + + + + 2022-12-14T23:34:35.214Z + + + source-work-id + 0ce907d8-1018-4a44-bd54-c116807642ea + 0ce907d8-1018-4a44-bd54-c116807642ea + self + + + + 2022-12-14T23:34:35.214Z + 2022-12-14T23:34:35.214Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Calcium dynamics simulations with morphologically detailed reconstruction of cerebellar astroglial cell model + + + + source-work-id + 0ce907d8-1018-4a44-bd54-c116807642ea + 0ce907d8-1018-4a44-bd54-c116807642ea + self + + + https://researchportal.tuni.fi/en/publications/0ce907d8-1018-4a44-bd54-c116807642ea + conference-abstract + + 2022 + + MET Research Day 2022, Tampere, Finland, 24/11/22 + + + + 2022-11-23T23:32:51.637Z + + + source-work-id + ca9165b9-107a-4e3a-a6c4-0e05fe2093d7 + ca9165b9-107a-4e3a-a6c4-0e05fe2093d7 + self + + + + 2022-11-23T23:32:51.637Z + 2022-11-23T23:32:51.637Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Calcium dynamics simulations with morphologically-detailed reconstruction of Bergmann glial cell model + + + + source-work-id + ca9165b9-107a-4e3a-a6c4-0e05fe2093d7 + ca9165b9-107a-4e3a-a6c4-0e05fe2093d7 + self + + + https://researchportal.tuni.fi/en/publications/ca9165b9-107a-4e3a-a6c4-0e05fe2093d7 + conference-abstract + + 2022 + + FENS Forum of Neuroscience 2022, Paris, France, 9/07/22 + + + + 2022-12-14T23:34:35.220Z + + + source-work-id + 0b317082-8c2e-474d-870b-cd6719c222fd + 0b317082-8c2e-474d-870b-cd6719c222fd + self + + + + 2022-12-14T23:34:35.220Z + 2022-12-14T23:34:35.220Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational modeling of neuron-astrocyte interactions in the NEST simulator + + + + source-work-id + 0b317082-8c2e-474d-870b-cd6719c222fd + 0b317082-8c2e-474d-870b-cd6719c222fd + self + + + https://researchportal.tuni.fi/en/publications/0b317082-8c2e-474d-870b-cd6719c222fd + conference-abstract + + 2022 + + INM IBI Retreat, Jülich, Germany, 18/10/22 + + + + 2023-02-16T00:00:45.671Z + + + source-work-id + 2d3269f5-f51d-42fb-822d-7a98d800f221 + 2d3269f5-f51d-42fb-822d-7a98d800f221 + self + + + + 2023-02-16T00:00:45.671Z + 2023-02-16T00:00:45.671Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Modeling spiking networks with neuron-glia interactions in NEST + + + + source-work-id + 2d3269f5-f51d-42fb-822d-7a98d800f221 + 2d3269f5-f51d-42fb-822d-7a98d800f221 + self + + + https://researchportal.tuni.fi/en/publications/2d3269f5-f51d-42fb-822d-7a98d800f221 + conference-abstract + + 2022 + + NEST Conference 2022, 23/06/22 + + + + 2023-07-20T23:04:42.594Z + + + source-work-id + a908c30c-2bad-4ed0-ad0f-65ba16a10b23 + a908c30c-2bad-4ed0-ad0f-65ba16a10b23 + self + + + doi + 10.1016/j.ifacol.2023.01.025 + 10.1016/j.ifacol.2023.01.025 + self + + + eid + 2-s2.0-85161971058 + 2-s2.0-85161971058 + self + + + + 2023-02-16T00:00:45.675Z + 2023-07-20T23:04:42.594Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Sensitivity analysis of biophysically-detailed tripartite synapse model + + + + source-work-id + a908c30c-2bad-4ed0-ad0f-65ba16a10b23 + a908c30c-2bad-4ed0-ad0f-65ba16a10b23 + self + + + eid + 2-s2.0-85161971058 + 2-s2.0-85161971058 + self + + + doi + 10.1016/j.ifacol.2023.01.025 + 10.1016/j.ifacol.2023.01.025 + self + + + https://researchportal.tuni.fi/en/publications/a908c30c-2bad-4ed0-ad0f-65ba16a10b23 + conference-paper + + 2022 + + 9th Conference on Foundations of Systems Biology in Engineering (FOSBE 2022) + + + + 2023-01-11T23:43:24.149Z + + + source-work-id + 72ac821c-bdfb-4b78-8b57-ee75cf9dc1e0 + 72ac821c-bdfb-4b78-8b57-ee75cf9dc1e0 + self + + + + 2023-01-11T23:43:24.149Z + 2023-01-11T23:43:24.149Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Sensitivity analysis of cortical synapse model with Uncertainpy + + + + source-work-id + 72ac821c-bdfb-4b78-8b57-ee75cf9dc1e0 + 72ac821c-bdfb-4b78-8b57-ee75cf9dc1e0 + self + + + https://researchportal.tuni.fi/en/publications/72ac821c-bdfb-4b78-8b57-ee75cf9dc1e0 + conference-abstract + + 2022 + + MET Research Day 2022, Tampere, Finland, 24/11/22 + + + + 2022-10-26T22:35:57.768Z + + + source-work-id + 26f8033e-0918-4a1c-ad12-9dd8c9184cf4 + 26f8033e-0918-4a1c-ad12-9dd8c9184cf4 + self + + + + 2022-10-26T22:35:57.768Z + 2022-10-26T22:35:57.768Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + The role of astrocytes in somatosensory synaptic plasticity during development: Integration of data and computational modeling + + + + source-work-id + 26f8033e-0918-4a1c-ad12-9dd8c9184cf4 + 26f8033e-0918-4a1c-ad12-9dd8c9184cf4 + self + + + https://researchportal.tuni.fi/en/publications/26f8033e-0918-4a1c-ad12-9dd8c9184cf4 + conference-abstract + + 2022 + + FENS Forum of Neuroscience 2022, Paris, France, 9/07/22 + + + + 2022-06-02T17:42:49.229Z + + + source-work-id + 7b60d5d5-aa11-4ef9-afe1-cea4d5f75747 + 7b60d5d5-aa11-4ef9-afe1-cea4d5f75747 + self + + + + 2022-01-28T23:28:42.551Z + 2022-06-02T17:42:49.229Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + AstroNeuronNets: Computational modeling of neuron-glia interaction in large-scale brain systems + + + + source-work-id + 7b60d5d5-aa11-4ef9-afe1-cea4d5f75747 + 7b60d5d5-aa11-4ef9-afe1-cea4d5f75747 + self + + + https://researchportal.tuni.fi/en/publications/7b60d5d5-aa11-4ef9-afe1-cea4d5f75747 + conference-poster + + 2021 + + Human Brain Project Summit 2021 - Pushing the Boundaries of Brain Research, 12/10/21 + + + + 2022-06-02T11:34:19.078Z + + + doi + 10.1002/glia.24036 + 10.1002/glia.24036 + self + + + source-work-id + e01f91ee-6b0c-4760-b761-6a5a229237a0 + e01f91ee-6b0c-4760-b761-6a5a229237a0 + self + + + + 2021-12-14T23:27:13.491Z + 2022-06-02T11:34:19.078Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Astrocytic modulation of cortical synaptic plasticity + + + + source-work-id + e01f91ee-6b0c-4760-b761-6a5a229237a0 + e01f91ee-6b0c-4760-b761-6a5a229237a0 + self + + + doi + 10.1002/glia.24036 + 10.1002/glia.24036 + self + + + https://researchportal.tuni.fi/en/publications/e01f91ee-6b0c-4760-b761-6a5a229237a0 + other + + 2021 + + Glia + + + + 2022-06-02T16:43:15.636Z + + + eid + 2-s2.0-85122908421 + 2-s2.0-85122908421 + self + + + source-work-id + 63869731-d3a2-4c6e-b068-dd93a05d20de + 63869731-d3a2-4c6e-b068-dd93a05d20de + self + + + doi + 10.1007/s10827-021-00801-9 + 10.1007/s10827-021-00801-9 + self + + + + 2022-01-21T23:27:54.805Z + 2022-06-02T16:43:15.636Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Astrocytic modulation of synaptic transmission and plasticity in developing somatosensory cortex + + + + source-work-id + 63869731-d3a2-4c6e-b068-dd93a05d20de + 63869731-d3a2-4c6e-b068-dd93a05d20de + self + + + eid + 2-s2.0-85122908421 + 2-s2.0-85122908421 + self + + + doi + 10.1007/s10827-021-00801-9 + 10.1007/s10827-021-00801-9 + self + + + https://researchportal.tuni.fi/en/publications/63869731-d3a2-4c6e-b068-dd93a05d20de + conference-abstract + + 2021 + + 30th Annual Computational Neuroscience Meeting CNS*2021, Online, 3/07/21 + + + + 2022-06-02T11:34:19.066Z + + + source-work-id + e604bb25-6e7c-4ff4-856a-54dbc79309e3 + e604bb25-6e7c-4ff4-856a-54dbc79309e3 + self + + + + 2021-12-14T23:27:13.486Z + 2022-06-02T11:34:19.066Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational neuroscience research group + + + + source-work-id + e604bb25-6e7c-4ff4-856a-54dbc79309e3 + e604bb25-6e7c-4ff4-856a-54dbc79309e3 + self + + + https://researchportal.tuni.fi/en/publications/e604bb25-6e7c-4ff4-856a-54dbc79309e3 + conference-abstract + + 2021 + + MET Research Day 2021, Tampere, Finland, 25/11/21 + + + + 2022-06-02T11:34:19.083Z + + + source-work-id + e26171e3-0b1f-49ca-96b7-0343dfdaa637 + e26171e3-0b1f-49ca-96b7-0343dfdaa637 + self + + + + 2021-12-14T23:27:13.493Z + 2022-06-02T11:34:19.083Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + In silico analysis of biochemical mechanisms responsible for astrocyte modulation of synaptic plasticity in developing somatosensory cortex + + + + source-work-id + e26171e3-0b1f-49ca-96b7-0343dfdaa637 + e26171e3-0b1f-49ca-96b7-0343dfdaa637 + self + + + https://researchportal.tuni.fi/en/publications/e26171e3-0b1f-49ca-96b7-0343dfdaa637 + conference-abstract + + 2021 + + SfN Global Connectome 2021, 11/01/21 + + + + 2022-06-02T11:34:19.073Z + + + source-work-id + 4cee97fd-e11b-4d64-8eee-92b9d672cbe2 + 4cee97fd-e11b-4d64-8eee-92b9d672cbe2 + self + + + + 2021-12-14T23:27:13.489Z + 2022-06-02T11:34:19.073Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Integrating neuron-glia knowledge and computational modeling to understand cortical synaptic plasticity + + + + source-work-id + 4cee97fd-e11b-4d64-8eee-92b9d672cbe2 + 4cee97fd-e11b-4d64-8eee-92b9d672cbe2 + self + + + https://researchportal.tuni.fi/en/publications/4cee97fd-e11b-4d64-8eee-92b9d672cbe2 + conference-abstract + + 2021 + + 50th Annual Meeting of the Society for Neuroscience (SFN 2021), 8/11/21 + + + + 2022-06-02T16:56:18.467Z + + + source-work-id + 2b9fecf1-7c32-40c1-85bd-a1b81fc82ee8 + 2b9fecf1-7c32-40c1-85bd-a1b81fc82ee8 + self + + + + 2022-01-24T23:28:41.097Z + 2022-06-02T16:56:18.467Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Morphologically-detailed reconstruction and simulation of cerebellar glial cell model + + + + source-work-id + 2b9fecf1-7c32-40c1-85bd-a1b81fc82ee8 + 2b9fecf1-7c32-40c1-85bd-a1b81fc82ee8 + self + + + https://researchportal.tuni.fi/en/publications/2b9fecf1-7c32-40c1-85bd-a1b81fc82ee8 + conference-abstract + + 2021 + + MET Research Day 2021, Tampere, Finland, 25/11/21 + + + + 2023-06-22T00:08:00.175Z + + + eid + 2-s2.0-85096029284 + 2-s2.0-85096029284 + self + + + pmid + 33170856 + 33170856 + self + + + doi + 10.1371/journal.pcbi.1008360 + 10.1371/journal.pcbi.1008360 + self + + + source-work-id + 03768df1-7f92-471f-bafd-b308970e8090 + 03768df1-7f92-471f-bafd-b308970e8090 + self + + + + 2021-09-01T00:39:10.271Z + 2023-06-22T00:08:00.175Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Astrocyte-mediated spike-timing-dependent long-term depression modulates synaptic properties in the developing cortex + + + + source-work-id + 03768df1-7f92-471f-bafd-b308970e8090 + 03768df1-7f92-471f-bafd-b308970e8090 + self + + + eid + 2-s2.0-85096029284 + 2-s2.0-85096029284 + self + + + pmid + 33170856 + 33170856 + self + + + doi + 10.1371/journal.pcbi.1008360 + 10.1371/journal.pcbi.1008360 + self + + + https://researchportal.tuni.fi/en/publications/03768df1-7f92-471f-bafd-b308970e8090 + journal-article + + 2020 + 11 + 10 + + PLoS Computational Biology + + + + 2022-09-27T22:32:28.415Z + + + source-work-id + 0055c66f-9fd0-4d93-bea1-07863a28ccd3 + 0055c66f-9fd0-4d93-bea1-07863a28ccd3 + self + + + + 2022-09-07T22:31:20.978Z + 2022-09-27T22:32:28.415Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Astrocyte simulations with realistic morphologies reveal diverse calcium transients in soma and processes + + + + source-work-id + 0055c66f-9fd0-4d93-bea1-07863a28ccd3 + 0055c66f-9fd0-4d93-bea1-07863a28ccd3 + self + + + doi + 10.1186/s12868-020-00593-1 + 10.1186/s12868-020-00593-1 + part-of + + + https://researchportal.tuni.fi/en/publications/0055c66f-9fd0-4d93-bea1-07863a28ccd3 + other + + 2020 + + BMC Neuroscience + + + + 2022-06-02T08:43:51.760Z + + + source-work-id + 1dc0564c-d38d-476b-8ec7-6e1f2cff323d + 1dc0564c-d38d-476b-8ec7-6e1f2cff323d + self + + + + 2021-11-12T23:28:17.067Z + 2022-06-02T08:43:51.760Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Biophysical modeling of astrocytes and astrocyte-neuron interactions + + + + source-work-id + 1dc0564c-d38d-476b-8ec7-6e1f2cff323d + 1dc0564c-d38d-476b-8ec7-6e1f2cff323d + self + + + https://researchportal.tuni.fi/en/publications/1dc0564c-d38d-476b-8ec7-6e1f2cff323d + conference-abstract + + 2020 + + FENS 2020 Virtual Forum, 11/07/20 + + + + 2022-06-02T08:43:51.766Z + + + source-work-id + 2ba73cb2-7e40-4f07-9f20-030b374cbe81 + 2ba73cb2-7e40-4f07-9f20-030b374cbe81 + self + + + + 2021-11-12T23:28:17.069Z + 2022-06-02T08:43:51.766Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational modeling of neural network dynamic in vitro: the role of neuronal, synaptic and non-neuronal mechanisms in spontaneous network bursts in rodent cortical cultures + + + + source-work-id + 2ba73cb2-7e40-4f07-9f20-030b374cbe81 + 2ba73cb2-7e40-4f07-9f20-030b374cbe81 + self + + + https://researchportal.tuni.fi/en/publications/2ba73cb2-7e40-4f07-9f20-030b374cbe81 + conference-abstract + + 2020 + + FENS 2020 Virtual Forum, 11/07/20 + + + + 2022-06-02T08:43:51.772Z + + + source-work-id + aff0da30-ff82-4cb2-86e9-1918c4b1802b + aff0da30-ff82-4cb2-86e9-1918c4b1802b + self + + + + 2021-11-12T23:28:17.070Z + 2022-06-02T08:43:51.772Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Morphologically detailed modeling of astrocytes to study calcium signals in soma and processes + + + + source-work-id + aff0da30-ff82-4cb2-86e9-1918c4b1802b + aff0da30-ff82-4cb2-86e9-1918c4b1802b + self + + + https://researchportal.tuni.fi/en/publications/aff0da30-ff82-4cb2-86e9-1918c4b1802b + conference-poster + + 2020 + + FENS 2020 Virtual Forum, 11/07/20 + + + + 2022-09-07T22:31:20.983Z + + + eid + 2-s2.0-85098745160 + 2-s2.0-85098745160 + self + + + source-work-id + 17c3a876-eb07-4ec0-b27d-7be578831323 + 17c3a876-eb07-4ec0-b27d-7be578831323 + self + + + + 2022-09-07T22:31:20.983Z + 2022-09-07T22:31:20.983Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Morphologically‐detailed reconstruction of cerebellar glial cells + + + + source-work-id + 17c3a876-eb07-4ec0-b27d-7be578831323 + 17c3a876-eb07-4ec0-b27d-7be578831323 + self + + + eid + 2-s2.0-85098745160 + 2-s2.0-85098745160 + self + + + doi + 10.1186/s12868-020-00593-1 + 10.1186/s12868-020-00593-1 + part-of + + + https://researchportal.tuni.fi/en/publications/17c3a876-eb07-4ec0-b27d-7be578831323 + other + + 2020 + + BMC Neuroscience + + + + 2022-06-02T08:43:51.777Z + + + source-work-id + 100b221b-9831-4432-888d-3b2d7423f8a9 + 100b221b-9831-4432-888d-3b2d7423f8a9 + self + + + + 2021-11-12T23:28:17.072Z + 2022-06-02T08:43:51.777Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + NEST component for modeling spiking neuron-astrocyte networks + + + + source-work-id + 100b221b-9831-4432-888d-3b2d7423f8a9 + 100b221b-9831-4432-888d-3b2d7423f8a9 + self + + + https://researchportal.tuni.fi/en/publications/100b221b-9831-4432-888d-3b2d7423f8a9 + conference-abstract + + 2020 + + NEST Conference 2020, 29/06/20 + + + + 2023-02-24T13:20:59.897Z + + + doi + 10.1186/s12868-020-00593-1 + 10.1186/s12868-020-00593-1 + self + + + source-work-id + ce72ab98-c1f4-43bf-8eb2-7deddb2672ab + ce72ab98-c1f4-43bf-8eb2-7deddb2672ab + self + + + + 2022-09-07T22:31:20.972Z + 2023-02-24T13:20:59.897Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + The interplay of neural mechanisms regulates spontaneous cortical network activity: inferring the role of key mechanisms using data-driven modeling + + + + source-work-id + ce72ab98-c1f4-43bf-8eb2-7deddb2672ab + ce72ab98-c1f4-43bf-8eb2-7deddb2672ab + self + + + doi + 10.1186/s12868-020-00593-1 + 10.1186/s12868-020-00593-1 + self + + + https://researchportal.tuni.fi/en/publications/ce72ab98-c1f4-43bf-8eb2-7deddb2672ab + other + + 2020 + + BMC Neuroscience + + + + 2022-05-28T18:18:37.970Z + + + eid + 2-s2.0-85065589802 + 2-s2.0-85065589802 + self + + + doi + 10.7554/elife.37102 + 10.7554/elife.37102 + https://doi.org/10.7554/elife.37102 + self + + + source-work-id + 0b43a97e-df2f-4fde-877c-fd34fc4263eb + 0b43a97e-df2f-4fde-877c-fd34fc4263eb + self + + + + 2019-05-11T21:49:44.462Z + 2019-05-11T21:49:44.462Z + + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Cerebellar Purkinje cells control eye movements with a rapid rate code that is invariant to spike irregularity + + + + doi + 10.7554/elife.37102 + 10.7554/elife.37102 + https://doi.org/10.7554/elife.37102 + self + + + issn + 2050-084X + 2050-084X + part-of + + + http://dx.doi.org/10.7554/elife.37102 + journal-article + + 2019 + 05 + 03 + + eLife + + + 2019-05-10T22:51:32.790Z + 2022-05-28T18:18:37.970Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Cerebellar Purkinje cells control eye movements with a rapid rate code that is invariant to spike irregularity + + + + source-work-id + 0b43a97e-df2f-4fde-877c-fd34fc4263eb + 0b43a97e-df2f-4fde-877c-fd34fc4263eb + self + + + eid + 2-s2.0-85065589802 + 2-s2.0-85065589802 + self + + + doi + 10.7554/eLife.37102 + 10.7554/elife.37102 + self + + + https://researchportal.tuni.fi/en/publications/0b43a97e-df2f-4fde-877c-fd34fc4263eb + journal-article + + 2019 + + eLIFE + + + + 2022-05-27T19:37:31.239Z + + + doi + https://doi.org/10.1007/978-3-030-00817-8_16 + 10.1007/978-3-030-00817-8_16 + self + + + source-work-id + 9ddbf82d-a80e-4a6c-8306-9f152853dcac + 9ddbf82d-a80e-4a6c-8306-9f152853dcac + self + + + + 2019-01-29T23:52:29.515Z + 2022-05-27T19:37:31.239Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational models of astrocytes and astrocyte-neuron interactions: Characterization, reproducibility, and future perspectives + + + + source-work-id + 9ddbf82d-a80e-4a6c-8306-9f152853dcac + 9ddbf82d-a80e-4a6c-8306-9f152853dcac + self + + + doi + https://doi.org/10.1007/978-3-030-00817-8_16 + 10.1007/978-3-030-00817-8_16 + self + + + isbn + 978-3-030-00817-8 + 9783030008178 + part-of + + + isbn + 978-3-030-00815-4 + 9783030008154 + part-of + + + https://researchportal.tuni.fi/en/publications/9ddbf82d-a80e-4a6c-8306-9f152853dcac + book-chapter + + 2019 + 03 + 13 + + Computational Glioscience + + + + 2022-06-02T08:43:51.749Z + + + source-work-id + 1923fb6e-6216-4311-b73c-238a3e38cdd3 + 1923fb6e-6216-4311-b73c-238a3e38cdd3 + self + + + + 2021-11-12T23:28:17.064Z + 2022-06-02T08:43:51.749Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational and theoretical tools for advancing neuroscience and neuropsychiatry + + + + source-work-id + 1923fb6e-6216-4311-b73c-238a3e38cdd3 + 1923fb6e-6216-4311-b73c-238a3e38cdd3 + self + + + https://researchportal.tuni.fi/en/publications/1923fb6e-6216-4311-b73c-238a3e38cdd3 + conference-abstract + + 2019 + + MET Research Day, Tampere, Finland, 29/11/19 + + + + 2022-06-02T08:43:51.744Z + + + source-work-id + 926809b2-aad2-4ae4-824a-86d02130a51d + 926809b2-aad2-4ae4-824a-86d02130a51d + self + + + + 2021-11-12T23:28:17.062Z + 2022-06-02T08:43:51.744Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational modeling of cerebellar functions + + + + source-work-id + 926809b2-aad2-4ae4-824a-86d02130a51d + 926809b2-aad2-4ae4-824a-86d02130a51d + self + + + https://researchportal.tuni.fi/en/publications/926809b2-aad2-4ae4-824a-86d02130a51d + conference-abstract + + 2019 + + MET Research Day, Tampere, Finland, 29/11/19 + + + + 2022-06-02T08:43:51.727Z + + + source-work-id + 4c0502b3-0406-4bbd-a4f8-6135e27955fd + 4c0502b3-0406-4bbd-a4f8-6135e27955fd + self + + + + 2021-11-12T23:28:17.058Z + 2022-06-02T08:43:51.727Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational models of astrocyte calcium dynamics and astrocyte-neuron interactions + + + + source-work-id + 4c0502b3-0406-4bbd-a4f8-6135e27955fd + 4c0502b3-0406-4bbd-a4f8-6135e27955fd + self + + + https://researchportal.tuni.fi/en/publications/4c0502b3-0406-4bbd-a4f8-6135e27955fd + conference-abstract + + 2019 + + 28th Annual Computational Neuroscience Meeting (CNS*2019), Barcelona, Spain, 13/07/19 + + + + 2022-06-02T08:43:51.738Z + + + source-work-id + d1a1626a-1f1a-4af8-942b-17509e7cc749 + d1a1626a-1f1a-4af8-942b-17509e7cc749 + self + + + + 2021-11-12T23:28:17.061Z + 2022-06-02T08:43:51.738Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + In silico modeling of astrocytes and neuron-astrocyte interactions + + + + source-work-id + d1a1626a-1f1a-4af8-942b-17509e7cc749 + d1a1626a-1f1a-4af8-942b-17509e7cc749 + self + + + https://researchportal.tuni.fi/en/publications/d1a1626a-1f1a-4af8-942b-17509e7cc749 + conference-abstract + + 2019 + + International Astrocyte School (IAS 2019), Bertinoro, Italy, 7/04/19 + + + + 2022-05-29T18:19:07.128Z + + + source-work-id + a2e58c08-3364-47c0-ac69-e1dd29188007 + a2e58c08-3364-47c0-ac69-e1dd29188007 + self + + + doi + 10.1186/s12868-019-0538-0 + 10.1186/s12868-019-0538-0 + self + + + + 2020-01-08T23:56:34.016Z + 2022-05-29T18:19:07.128Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Modeling the influence of neuron-astrocyte interactions on signal transmission in neuronal networks + + + + source-work-id + a2e58c08-3364-47c0-ac69-e1dd29188007 + a2e58c08-3364-47c0-ac69-e1dd29188007 + self + + + doi + 10.1186/s12868-019-0538-0 + 10.1186/s12868-019-0538-0 + self + + + https://researchportal.tuni.fi/en/publications/a2e58c08-3364-47c0-ac69-e1dd29188007 + other + + 2019 + + BMC Neuroscience + + + + 2022-05-26T19:22:40.645Z + + + eid + 2-s2.0-85049072275 + 2-s2.0-85049072275 + self + + + doi + 10.3389/fncom.2018.00014 + 10.3389/fncom.2018.00014 + https://doi.org/10.3389/fncom.2018.00014 + self + + + source-work-id + 068ea89d-98ee-4b72-846a-9c4d0834350b + 068ea89d-98ee-4b72-846a-9c4d0834350b + self + + + + 2018-04-04T16:16:06.222Z + 2018-04-04T16:16:06.222Z + + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Computational models for calcium-mediated astrocyte functions + + + + doi + 10.3389/fncom.2018.00014 + 10.3389/fncom.2018.00014 + https://doi.org/10.3389/fncom.2018.00014 + self + + + journal-article + + 2018 + 04 + 04 + + Frontiers in Computational Neuroscience + + + 2018-04-05T22:52:00.152Z + 2022-05-26T19:22:40.645Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational models for calcium-mediated astrocyte functions + + + + source-work-id + 068ea89d-98ee-4b72-846a-9c4d0834350b + 068ea89d-98ee-4b72-846a-9c4d0834350b + self + + + eid + 2-s2.0-85049072275 + 2-s2.0-85049072275 + self + + + doi + 10.3389/fncom.2018.00014 + 10.3389/fncom.2018.00014 + self + + + https://researchportal.tuni.fi/en/publications/068ea89d-98ee-4b72-846a-9c4d0834350b + journal-article + + 2018 + + Frontiers in Computational Neuroscience + + + + 2022-05-25T22:51:07.717Z + + + doi + 10.1007/978-981-10-5122-7_170 + 10.1007/978-981-10-5122-7_170 + self + + + eid + 2-s2.0-85021708806 + 2-s2.0-85021708806 + self + + + source-work-id + eec86343-5f78-4f86-8dee-cf1d222d33d3 + eec86343-5f78-4f86-8dee-cf1d222d33d3 + self + + + + 2017-08-28T12:54:33.008Z + 2022-05-25T22:51:07.717Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Altered synaptic signaling due to β-amyloid interference in astrocytes + + + + source-work-id + eec86343-5f78-4f86-8dee-cf1d222d33d3 + eec86343-5f78-4f86-8dee-cf1d222d33d3 + self + + + eid + 2-s2.0-85021708806 + 2-s2.0-85021708806 + self + + + doi + 10.1007/978-981-10-5122-7_170 + 10.1007/978-981-10-5122-7_170 + self + + + isbn + 9789811051210 + 9789811051210 + part-of + + + https://researchportal.tuni.fi/en/publications/eec86343-5f78-4f86-8dee-cf1d222d33d3 + conference-paper + + 2018 + + EMBEC and NBC 2017 - Joint Conference of the European Medical and Biological Engineering Conference EMBEC 2017 and the Nordic-Baltic Conference on Biomedical Engineering and Medical Physics, NBC 2017 + + + + 2022-05-27T01:27:19.180Z + + + source-work-id + 99ec6501-61db-461f-b81c-861cdc4639ff + 99ec6501-61db-461f-b81c-861cdc4639ff + self + + + doi + 10.3389/fninf.2018.00020 + 10.3389/fninf.2018.00020 + self + + + eid + 2-s2.0-85049113764 + 2-s2.0-85049113764 + self + + + + 2018-06-05T22:52:00.480Z + 2022-05-27T01:27:19.180Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Challenges in reproducibility, replicability, and comparability of computational models and tools for neuronal and glial networks, cells, and subcellular structures + + + + source-work-id + 99ec6501-61db-461f-b81c-861cdc4639ff + 99ec6501-61db-461f-b81c-861cdc4639ff + self + + + eid + 2-s2.0-85049113764 + 2-s2.0-85049113764 + self + + + doi + 10.3389/fninf.2018.00020 + 10.3389/fninf.2018.00020 + self + + + https://researchportal.tuni.fi/en/publications/99ec6501-61db-461f-b81c-861cdc4639ff + journal-article + + 2018 + + Frontiers in Neuroinformatics + + + + 2022-06-02T08:43:51.732Z + + + source-work-id + c6dc5e37-dcba-476a-8361-d4dd61e34052 + c6dc5e37-dcba-476a-8361-d4dd61e34052 + self + + + + 2021-11-12T23:28:17.059Z + 2022-06-02T08:43:51.732Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational and theoretical tools for advancing neuroscience: from molecules to networks + + + + source-work-id + c6dc5e37-dcba-476a-8361-d4dd61e34052 + c6dc5e37-dcba-476a-8361-d4dd61e34052 + self + + + https://researchportal.tuni.fi/en/publications/c6dc5e37-dcba-476a-8361-d4dd61e34052 + conference-abstract + + 2018 + + Biomeditech &amp; med research day, Tampere, Finland, 23/11/18 + + + + 2022-05-27T19:45:47.872Z + + + source-work-id + 2d7453b6-b2e5-46a6-a6a8-c7ba96b4ee6e + 2d7453b6-b2e5-46a6-a6a8-c7ba96b4ee6e + self + + + doi + 10.1186/s12868-018-0452-x + 10.1186/s12868-018-0452-x + self + + + + 2019-01-30T23:52:23.773Z + 2022-05-27T19:45:47.872Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational modeling of neuron-astrocyte interactions: Evolution, reproducibility, comparability and future development of models + + + + source-work-id + 2d7453b6-b2e5-46a6-a6a8-c7ba96b4ee6e + 2d7453b6-b2e5-46a6-a6a8-c7ba96b4ee6e + self + + + doi + 10.1186/s12868-018-0452-x + 10.1186/s12868-018-0452-x + self + + + https://researchportal.tuni.fi/en/publications/2d7453b6-b2e5-46a6-a6a8-c7ba96b4ee6e + other + + 2018 + + BMC Neuroscience + + + + 2022-06-02T08:43:51.721Z + + + source-work-id + 3c40e3f3-83a4-4443-b8e1-e9d3caba497a + 3c40e3f3-83a4-4443-b8e1-e9d3caba497a + self + + + + 2021-11-12T23:28:17.056Z + 2022-06-02T08:43:51.721Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational models of astrocytes and neuron-astrocyte interactions to promote understanding of synaptic function and dysfunction + + + + source-work-id + 3c40e3f3-83a4-4443-b8e1-e9d3caba497a + 3c40e3f3-83a4-4443-b8e1-e9d3caba497a + self + + + https://researchportal.tuni.fi/en/publications/3c40e3f3-83a4-4443-b8e1-e9d3caba497a + conference-abstract + + 2018 + + Neuroplasticity: From Bench to Machine Learning, Guildford, United Kingdom, 13/07/18 + + + + 2022-06-02T08:43:51.755Z + + + source-work-id + f6766530-0fb8-49bb-b28d-e1d0f0e6c7ca + f6766530-0fb8-49bb-b28d-e1d0f0e6c7ca + self + + + + 2021-11-12T23:28:17.065Z + 2022-06-02T08:43:51.755Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + In silico modeling of astrocytes and neuron-astrocyte interactions + + + + source-work-id + f6766530-0fb8-49bb-b28d-e1d0f0e6c7ca + f6766530-0fb8-49bb-b28d-e1d0f0e6c7ca + self + + + https://researchportal.tuni.fi/en/publications/f6766530-0fb8-49bb-b28d-e1d0f0e6c7ca + conference-abstract + + 2018 + + Neuroscience 2013, Nov 9-13, San Diego, California, 1/01/13 + + + + 2022-06-02T08:43:51.716Z + + + source-work-id + 60e4cea2-2843-49e0-8c9b-60af98739255 + 60e4cea2-2843-49e0-8c9b-60af98739255 + self + + + + 2021-11-12T23:28:17.055Z + 2022-06-02T08:43:51.716Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Astrocyte-modulated synaptic plasticity in sensory cortex in health and pathology: A computational study + + + + source-work-id + 60e4cea2-2843-49e0-8c9b-60af98739255 + 60e4cea2-2843-49e0-8c9b-60af98739255 + self + + + https://researchportal.tuni.fi/en/publications/60e4cea2-2843-49e0-8c9b-60af98739255 + conference-abstract + + 2017 + + 47th Annual Meeting of the Society for Neuroscience (SFN 2017), Washington, United States, 11/11/17 + + + + 2022-05-26T09:51:33.186Z + + + source-work-id + 32648b4a-bca0-4d2b-8af9-d53f58793afc + 32648b4a-bca0-4d2b-8af9-d53f58793afc + self + + + + 2017-12-15T23:58:46.198Z + 2022-05-26T09:51:33.186Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Astrocyte-modulated synaptic plasticity in sensory cortex: A computational study + + + + source-work-id + 32648b4a-bca0-4d2b-8af9-d53f58793afc + 32648b4a-bca0-4d2b-8af9-d53f58793afc + self + + + https://researchportal.tuni.fi/en/publications/32648b4a-bca0-4d2b-8af9-d53f58793afc + other + + 2017 + + BMC Neuroscience + + + + 2022-05-26T08:52:19.369Z + + + source-work-id + ea94e870-8e7a-4604-99ec-7691f6b64b08 + ea94e870-8e7a-4604-99ec-7691f6b64b08 + self + + + doi + 10.1002/glia.v65.S1 + 10.1002/glia.v65.s1 + self + + + + 2017-11-30T00:13:49.706Z + 2022-05-26T08:52:19.369Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational modeling of glial-neuronal interactions + + + + source-work-id + ea94e870-8e7a-4604-99ec-7691f6b64b08 + ea94e870-8e7a-4604-99ec-7691f6b64b08 + self + + + doi + 10.1002/glia.v65.S1 + 10.1002/glia.v65.s1 + self + + + https://researchportal.tuni.fi/en/publications/ea94e870-8e7a-4604-99ec-7691f6b64b08 + other + + 2017 + + Glia + + + + 2022-05-26T08:52:19.377Z + + + source-work-id + 6a974f0a-cb8f-4191-a005-049510314685 + 6a974f0a-cb8f-4191-a005-049510314685 + self + + + eid + 2-s2.0-85027678828 + 2-s2.0-85027678828 + self + + + doi + 10.1007/978-3-319-63312-1_14 + 10.1007/978-3-319-63312-1_14 + self + + + + 2017-11-30T00:13:49.712Z + 2022-05-26T08:52:19.377Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Modeling neuron-astrocyte interactions: towards understanding synaptic plasticity and learning in the brain + + + + source-work-id + 6a974f0a-cb8f-4191-a005-049510314685 + 6a974f0a-cb8f-4191-a005-049510314685 + self + + + eid + 2-s2.0-85027678828 + 2-s2.0-85027678828 + self + + + doi + 10.1007/978-3-319-63312-1_14 + 10.1007/978-3-319-63312-1_14 + self + + + isbn + 978-3-319-63312-1 + 9783319633121 + part-of + + + isbn + 978-3-319-63311-4 + 9783319633114 + part-of + + + https://researchportal.tuni.fi/en/publications/6a974f0a-cb8f-4191-a005-049510314685 + conference-paper + + 2017 + + Intelligent Computing Theories and Application + + + + 2022-05-25T22:51:07.705Z + + + eid + 2-s2.0-85015371791 + 2-s2.0-85015371791 + self + + + source-work-id + 32d5fcbc-20ef-410c-984e-ce81b0c4c5c8 + 32d5fcbc-20ef-410c-984e-ce81b0c4c5c8 + self + + + doi + 10.3389/fninf.2017.00011 + 10.3389/fninf.2017.00011 + self + + + + 2017-08-28T12:54:33.008Z + 2022-05-25T22:51:07.705Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Reproducibility and Comparability of Computational Models for Astrocyte Calcium Excitability + + + + source-work-id + 32d5fcbc-20ef-410c-984e-ce81b0c4c5c8 + 32d5fcbc-20ef-410c-984e-ce81b0c4c5c8 + self + + + eid + 2-s2.0-85015371791 + 2-s2.0-85015371791 + self + + + doi + 10.3389/fninf.2017.00011 + 10.3389/fninf.2017.00011 + self + + + https://researchportal.tuni.fi/en/publications/32d5fcbc-20ef-410c-984e-ce81b0c4c5c8 + journal-article + + 2017 + + Frontiers in Neuroinformatics + + + + 2022-05-26T10:06:48.771Z + + + source-work-id + 0cb20c15-92dd-4b34-b724-36f8fa0cecde + 0cb20c15-92dd-4b34-b724-36f8fa0cecde + self + + + doi + 10.7717/peerj-cs.142 + 10.7717/peerj-cs.142 + self + + + eid + 2-s2.0-85040188790 + 2-s2.0-85040188790 + self + + + + 2017-12-20T23:58:50.709Z + 2022-05-26T10:06:48.771Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Sustainable computational science: the ReScience initiative + + + + source-work-id + 0cb20c15-92dd-4b34-b724-36f8fa0cecde + 0cb20c15-92dd-4b34-b724-36f8fa0cecde + self + + + eid + 2-s2.0-85040188790 + 2-s2.0-85040188790 + self + + + doi + 10.7717/peerj-cs.142 + 10.7717/peerj-cs.142 + self + + + https://researchportal.tuni.fi/en/publications/0cb20c15-92dd-4b34-b724-36f8fa0cecde + journal-article + + 2017 + + PeerJ Computer Science + + + + 2022-06-02T08:43:51.709Z + + + source-work-id + 2f0c0a22-2895-49bb-a814-85e55d5cb4ab + 2f0c0a22-2895-49bb-a814-85e55d5cb4ab + self + + + + 2021-11-12T23:28:17.053Z + 2022-06-02T08:43:51.709Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Astrocyte-neuron interactions in long-term plasticity: a computational study + + + + source-work-id + 2f0c0a22-2895-49bb-a814-85e55d5cb4ab + 2f0c0a22-2895-49bb-a814-85e55d5cb4ab + self + + + https://researchportal.tuni.fi/en/publications/2f0c0a22-2895-49bb-a814-85e55d5cb4ab + conference-abstract + + 2016 + + The 46th Annual Meeting of the Society for Neuroscience, SFN 2016, San Diego, CA, USA, 12-16 November 2016, San Diego, United States, 12/11/16 + + + + 2022-06-02T08:43:51.701Z + + + source-work-id + 33b2c0a1-30b5-4aaf-a444-3ed4ba0c5c29 + 33b2c0a1-30b5-4aaf-a444-3ed4ba0c5c29 + self + + + + 2021-11-12T23:28:17.050Z + 2022-06-02T08:43:51.701Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Astrocyte-neuron interactions in vitro and in vivo: Evaluation of computational models + + + + source-work-id + 33b2c0a1-30b5-4aaf-a444-3ed4ba0c5c29 + 33b2c0a1-30b5-4aaf-a444-3ed4ba0c5c29 + self + + + https://researchportal.tuni.fi/en/publications/33b2c0a1-30b5-4aaf-a444-3ed4ba0c5c29 + conference-abstract + + 2016 + + 10th FENS Forum of Neuroscience, Copenhagen, Denmark, 2/07/16 + + + + 2022-05-25T22:51:07.752Z + + + source-work-id + 70f8cd58-7f1f-4ac4-bdb3-d3bc88260910 + 70f8cd58-7f1f-4ac4-bdb3-d3bc88260910 + self + + + + 2017-08-28T12:54:33.010Z + 2022-05-25T22:51:07.752Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Extending computational models of astrocyte-neuron interactions with biochemical mechanisms on the postsynaptic terminal + + + + source-work-id + 70f8cd58-7f1f-4ac4-bdb3-d3bc88260910 + 70f8cd58-7f1f-4ac4-bdb3-d3bc88260910 + self + + + https://researchportal.tuni.fi/en/publications/70f8cd58-7f1f-4ac4-bdb3-d3bc88260910 + other + + 2015 + + BMC Neuroscience + + + + 2022-05-25T22:51:07.741Z + + + source-work-id + 0216cbb3-5483-4ee1-bc94-811fbbeeeb7d + 0216cbb3-5483-4ee1-bc94-811fbbeeeb7d + self + + + + 2017-08-28T12:54:33.010Z + 2022-05-25T22:51:07.741Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + How temporal variation affects downstream neurons through deep cerebellar nuclei model + + + + source-work-id + 0216cbb3-5483-4ee1-bc94-811fbbeeeb7d + 0216cbb3-5483-4ee1-bc94-811fbbeeeb7d + self + + + https://researchportal.tuni.fi/en/publications/0216cbb3-5483-4ee1-bc94-811fbbeeeb7d + conference-paper + + 2015 + + Annual Biomedical Research Conference for Minority Student, ABRCMS 2015, Seattle, WA, USA, 11-14 November, 2015 + + + + 2022-05-25T22:51:07.736Z + + + source-work-id + 3319ec06-7170-46ae-9578-ee9b173b7a34 + 3319ec06-7170-46ae-9578-ee9b173b7a34 + self + + + + 2017-08-28T12:54:33.009Z + 2022-05-25T22:51:07.736Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Modeling cortical neuroplasticity and synaptic functions: Implications for future neuroscience, medicine, and computing + + + + source-work-id + 3319ec06-7170-46ae-9578-ee9b173b7a34 + 3319ec06-7170-46ae-9578-ee9b173b7a34 + self + + + https://researchportal.tuni.fi/en/publications/3319ec06-7170-46ae-9578-ee9b173b7a34 + conference-paper + + 2015 + + BioMediTech Research Day, Tampere, Finland, 4 Dec 2015 + + + + 2022-05-25T22:51:07.730Z + + + source-work-id + cf74b814-b098-4fbd-80cf-efea52849585 + cf74b814-b098-4fbd-80cf-efea52849585 + self + + + doi + 10.1186/1471-2202-16-S1-P147 + 10.1186/1471-2202-16-s1-p147 + self + + + + 2017-08-28T12:54:33.009Z + 2022-05-25T22:51:07.730Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Numerical characterization of noisy fluctuations in two different types of stochastic differential equation models of neural signaling + + + + source-work-id + cf74b814-b098-4fbd-80cf-efea52849585 + cf74b814-b098-4fbd-80cf-efea52849585 + self + + + doi + 10.1186/1471-2202-16-S1-P147 + 10.1186/1471-2202-16-s1-p147 + self + + + https://researchportal.tuni.fi/en/publications/cf74b814-b098-4fbd-80cf-efea52849585 + other + + 2015 + + BMC Neuroscience + + + + 2022-05-25T22:51:07.723Z + + + source-work-id + d85c2e1c-a68e-41c2-bee9-85a1fe43d918 + d85c2e1c-a68e-41c2-bee9-85a1fe43d918 + self + + + doi + 10.1186/1471-2202-16-S1-P202 + 10.1186/1471-2202-16-s1-p202 + self + + + + 2017-08-28T12:54:33.009Z + 2022-05-25T22:51:07.723Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Regular and irregular stimuli result in changes in mice eye movement and cerebellar nuclei neuron model behavior + + + + source-work-id + d85c2e1c-a68e-41c2-bee9-85a1fe43d918 + d85c2e1c-a68e-41c2-bee9-85a1fe43d918 + self + + + doi + 10.1186/1471-2202-16-S1-P202 + 10.1186/1471-2202-16-s1-p202 + self + + + https://researchportal.tuni.fi/en/publications/d85c2e1c-a68e-41c2-bee9-85a1fe43d918 + other + + 2015 + + BMC Neuroscience + + + + 2022-05-25T22:51:07.758Z + + + source-work-id + 0aff2728-3b22-4069-bacd-c28b54b19781 + 0aff2728-3b22-4069-bacd-c28b54b19781 + self + + + + 2017-08-28T12:54:33.010Z + 2022-05-25T22:51:07.758Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Contribution of amyloid beta to intracellular calcium signals in astrocytes: a computational study + + + + source-work-id + 0aff2728-3b22-4069-bacd-c28b54b19781 + 0aff2728-3b22-4069-bacd-c28b54b19781 + self + + + https://researchportal.tuni.fi/en/publications/0aff2728-3b22-4069-bacd-c28b54b19781 + conference-paper + + 2012 + + The 42nd Annual Meeting of the Society for Neuroscience, SFN 2012, New Orleans, LA, USA, 13-17 October 2012 + + + + 2022-05-25T22:51:07.790Z + + + source-work-id + 7177dc31-7fb6-4db6-970f-d93cf01ed582 + 7177dc31-7fb6-4db6-970f-d93cf01ed582 + self + + + pmid + 21693049 + 21693049 + self + + + doi + 10.1186/1471-2105-12-252 + 10.1186/1471-2105-12-252 + self + + + wosuid + 000293000900001 + 000293000900001 + self + + + eid + 2-s2.0-79959247643 + 2-s2.0-79959247643 + self + + + pmc + PMC3142227 + 3142227 + self + + + + 2016-09-21T17:41:34.823Z + 2022-05-25T04:26:20.957Z + + + https://orcid.org/client/0000-0002-9157-3431 + 0000-0002-9157-3431 + orcid.org + + Europe PubMed Central + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Computational study of noise in a large signal transduction network. + + + + pmid + 21693049 + 21693049 + self + + + pmc + PMC3142227 + 3142227 + self + + + doi + 10.1186/1471-2105-12-252 + 10.1186/1471-2105-12-252 + self + + + http://europepmc.org/abstract/med/21693049 + journal-article + + 2011 + + + + 2016-09-21T17:59:59.984Z + 2022-05-25T04:26:23.093Z + + + https://orcid.org/client/0000-0002-5982-8983 + 0000-0002-5982-8983 + orcid.org + + Scopus - Elsevier + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Computational study of noise in a large signal transduction network + + + + doi + 10.1186/1471-2105-12-252 + 10.1186/1471-2105-12-252 + self + + + eid + 2-s2.0-79959247643 + 2-s2.0-79959247643 + self + + + http://www.scopus.com/inward/record.url?eid=2-s2.0-79959247643&partnerID=MN8TOARS + journal-article + + 2011 + + BMC Bioinformatics + + + 2017-08-28T12:54:33.012Z + 2022-05-25T22:51:07.790Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational study of noise in a large signal transduction network + + + + source-work-id + 7177dc31-7fb6-4db6-970f-d93cf01ed582 + 7177dc31-7fb6-4db6-970f-d93cf01ed582 + self + + + wosuid + 000293000900001 + 000293000900001 + self + + + eid + 2-s2.0-79959247643 + 2-s2.0-79959247643 + self + + + doi + 10.1186/1471-2105-12-252 + 10.1186/1471-2105-12-252 + self + + + https://researchportal.tuni.fi/en/publications/7177dc31-7fb6-4db6-970f-d93cf01ed582 + journal-article + + 2011 + + BMC Bioinformatics + + + + 2022-05-25T22:51:07.783Z + + + doi + 10.1186/1471-2202-12-S1-P201 + 10.1186/1471-2202-12-s1-p201 + self + + + source-work-id + 5fde6fbe-e1f1-4794-88d3-f3935bba346b + 5fde6fbe-e1f1-4794-88d3-f3935bba346b + self + + + + 2017-08-28T12:54:33.012Z + 2022-05-25T22:51:07.783Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Contribution Of SERCA And IP3 Sensitivity to Calcium Signaling in Astrocytes: A Computational Study + + + + source-work-id + 5fde6fbe-e1f1-4794-88d3-f3935bba346b + 5fde6fbe-e1f1-4794-88d3-f3935bba346b + self + + + doi + 10.1186/1471-2202-12-S1-P201 + 10.1186/1471-2202-12-s1-p201 + self + + + https://researchportal.tuni.fi/en/publications/5fde6fbe-e1f1-4794-88d3-f3935bba346b + other + + 2011 + + BMC Neuroscience + + + + 2022-05-25T22:51:07.777Z + + + source-work-id + 289a5eae-86c1-4317-83e2-24cc045277f9 + 289a5eae-86c1-4317-83e2-24cc045277f9 + self + + + doi + 10.1371/journal.pone.0017914 + 10.1371/journal.pone.0017914 + self + + + wosuid + 000289054600020 + 000289054600020 + self + + + eid + 2-s2.0-79953199022 + 2-s2.0-79953199022 + self + + + pmc + PMC3066169 + 3066169 + self + + + pmid + 21483471 + 21483471 + self + + + + 2016-09-21T17:41:34.837Z + 2022-05-25T04:26:20.969Z + + + https://orcid.org/client/0000-0002-9157-3431 + 0000-0002-9157-3431 + orcid.org + + Europe PubMed Central + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Effects of transmitters and amyloid-beta peptide on calcium signals in rat cortical astrocytes: Fura-2AM measurements and stochastic model simulations. + + + + pmid + 21483471 + 21483471 + self + + + pmc + PMC3066169 + 3066169 + self + + + doi + 10.1371/journal.pone.0017914 + 10.1371/journal.pone.0017914 + self + + + http://europepmc.org/abstract/med/21483471 + journal-article + + 2011 + + + + 2016-09-21T17:59:59.951Z + 2022-05-25T04:26:23.076Z + + + https://orcid.org/client/0000-0002-5982-8983 + 0000-0002-5982-8983 + orcid.org + + Scopus - Elsevier + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Effects of transmitters and amyloid-beta peptide on calcium signals in rat cortical astrocytes: Fura-2AM measurements and stochastic model simulations + + + + doi + 10.1371/journal.pone.0017914 + 10.1371/journal.pone.0017914 + self + + + eid + 2-s2.0-79953199022 + 2-s2.0-79953199022 + self + + + http://www.scopus.com/inward/record.url?eid=2-s2.0-79953199022&partnerID=MN8TOARS + journal-article + + 2011 + + PLoS ONE + + + 2017-08-28T12:54:33.011Z + 2022-05-25T22:51:07.777Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Effects of Transmitters and Amyloid-Beta Peptide on Calcium Signals in Rat Cortical Astrocytes: Fura-2AM Measurements and Stochastic Model Simulations + + + + source-work-id + 289a5eae-86c1-4317-83e2-24cc045277f9 + 289a5eae-86c1-4317-83e2-24cc045277f9 + self + + + wosuid + 000289054600020 + 000289054600020 + self + + + eid + 2-s2.0-79953199022 + 2-s2.0-79953199022 + self + + + doi + 10.1371/journal.pone.0017914 + 10.1371/journal.pone.0017914 + self + + + https://researchportal.tuni.fi/en/publications/289a5eae-86c1-4317-83e2-24cc045277f9 + journal-article + + 2011 + + PLoS ONE + + + + 2022-05-25T22:51:07.770Z + + + doi + 10.1155/2011/797250 + 10.1155/2011/797250 + self + + + pmc + PMC3171304 + 3171304 + self + + + pmid + 21559300 + 21559300 + self + + + source-work-id + 473ed2f3-c41e-49df-9bc1-4ca4417b2fdf + 473ed2f3-c41e-49df-9bc1-4ca4417b2fdf + self + + + eid + 2-s2.0-79959244197 + 2-s2.0-79959244197 + self + + + + 2016-09-21T17:41:34.830Z + 2022-05-25T04:26:20.963Z + + + https://orcid.org/client/0000-0002-9157-3431 + 0000-0002-9157-3431 + orcid.org + + Europe PubMed Central + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Modeling signal transduction leading to synaptic plasticity: evaluation and comparison of five models. + + + + pmid + 21559300 + 21559300 + self + + + pmc + PMC3171304 + 3171304 + self + + + doi + 10.1155/2011/797250 + 10.1155/2011/797250 + self + + + http://europepmc.org/abstract/med/21559300 + journal-article + + 2011 + + + + 2016-09-21T17:59:59.967Z + 2022-05-25T04:26:23.081Z + + + https://orcid.org/client/0000-0002-5982-8983 + 0000-0002-5982-8983 + orcid.org + + Scopus - Elsevier + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Modeling signal transduction leading to synaptic plasticity: Evaluation and comparison of five models + + + + doi + 10.1155/2011/797250 + 10.1155/2011/797250 + self + + + eid + 2-s2.0-79959244197 + 2-s2.0-79959244197 + self + + + http://www.scopus.com/inward/record.url?eid=2-s2.0-79959244197&partnerID=MN8TOARS + journal-article + + 2011 + + Eurasip Journal on Bioinformatics and Systems Biology + + + 2017-08-28T12:54:33.011Z + 2022-05-25T22:51:07.770Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Modeling Signal Transduction Leading to Synaptic Plasticity: Evaluation and Comparison of Five Models + + + + source-work-id + 473ed2f3-c41e-49df-9bc1-4ca4417b2fdf + 473ed2f3-c41e-49df-9bc1-4ca4417b2fdf + self + + + eid + 2-s2.0-79959244197 + 2-s2.0-79959244197 + self + + + doi + 10.1155/2011/797250 + 10.1155/2011/797250 + self + + + https://researchportal.tuni.fi/en/publications/473ed2f3-c41e-49df-9bc1-4ca4417b2fdf + journal-article + + 2011 + + Eurasip Journal on Bioinformatics and Systems Biology + + + + 2022-05-25T22:51:07.764Z + + + source-work-id + 0b24f981-76d6-426b-9f98-87519e44d59d + 0b24f981-76d6-426b-9f98-87519e44d59d + self + + + + 2017-08-28T12:54:33.011Z + 2022-05-25T22:51:07.764Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Studying the quality of noise in a large biochemical reaction network as a function of the system volume + + + + source-work-id + 0b24f981-76d6-426b-9f98-87519e44d59d + 0b24f981-76d6-426b-9f98-87519e44d59d + self + + + https://researchportal.tuni.fi/en/publications/0b24f981-76d6-426b-9f98-87519e44d59d + conference-paper + + 2011 + + The International Conference on Applied Mathematics, Modeling and Computational Science, AMMCS 2011, 25 - 29 July, Waterloo, Ontario, Canada + + + + 2022-05-25T22:51:07.846Z + + + doi + 10.3389/conf.fnins.2010.13.00061 + 10.3389/conf.fnins.2010.13.00061 + self + + + source-work-id + 54df0b2a-9c00-4bd4-8106-9316b22947e8 + 54df0b2a-9c00-4bd4-8106-9316b22947e8 + self + + + + 2017-08-28T12:54:33.014Z + 2022-05-25T22:51:07.846Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Calcium signaling in astrocytes: modeling Fura-2AM measurements + + + + source-work-id + 54df0b2a-9c00-4bd4-8106-9316b22947e8 + 54df0b2a-9c00-4bd4-8106-9316b22947e8 + self + + + doi + 10.3389/conf.fnins.2010.13.00061 + 10.3389/conf.fnins.2010.13.00061 + self + + + https://researchportal.tuni.fi/en/publications/54df0b2a-9c00-4bd4-8106-9316b22947e8 + other + + 2010 + + Frontiers in Neuroscience + + + + 2022-05-25T22:51:07.839Z + + + source-work-id + 5a0c7c83-1a0b-4700-9898-93f37ad53bda + 5a0c7c83-1a0b-4700-9898-93f37ad53bda + self + + + + 2017-08-28T12:54:33.014Z + 2022-05-25T22:51:07.839Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Comparing two stochastic differential equation models for protein kinase C activation pathway + + + + source-work-id + 5a0c7c83-1a0b-4700-9898-93f37ad53bda + 5a0c7c83-1a0b-4700-9898-93f37ad53bda + self + + + https://researchportal.tuni.fi/en/publications/5a0c7c83-1a0b-4700-9898-93f37ad53bda + conference-paper + + 2010 + + Abstracts of Papers, Posters and Talks presented at the 2010 Joint RECOMB Satellite Conference on Systems Biology - Regulatory Genomics - DREAM5, New York, USA, 16-20 November 2010 + + + + 2022-05-25T22:51:07.833Z + + + doi + 10.1145/1854776.1854838 + 10.1145/1854776.1854838 + self + + + source-work-id + 11469981-5081-4337-9f58-416c93658b54 + 11469981-5081-4337-9f58-416c93658b54 + self + + + eid + 2-s2.0-77958042740 + 2-s2.0-77958042740 + self + + + + 2016-09-21T18:00:00.521Z + 2022-05-25T04:26:23.127Z + + + https://orcid.org/client/0000-0002-5982-8983 + 0000-0002-5982-8983 + orcid.org + + Scopus - Elsevier + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Comparison of discrete- and continuous-state stochastic methods to model neuronal signal transduction + + + + doi + 10.1145/1854776.1854838 + 10.1145/1854776.1854838 + self + + + eid + 2-s2.0-77958042740 + 2-s2.0-77958042740 + self + + + http://www.scopus.com/inward/record.url?eid=2-s2.0-77958042740&partnerID=MN8TOARS + conference-paper + + 2010 + + 2010 ACM International Conference on Bioinformatics and Computational Biology, ACM-BCB 2010 + + + 2017-08-28T12:54:33.014Z + 2022-05-25T22:51:07.833Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Comparison of discrete- and continuous-state stochastic methods to model neuronal signal transduction + + + + source-work-id + 11469981-5081-4337-9f58-416c93658b54 + 11469981-5081-4337-9f58-416c93658b54 + self + + + eid + 2-s2.0-77958042740 + 2-s2.0-77958042740 + self + + + doi + 10.1145/1854776.1854838 + 10.1145/1854776.1854838 + self + + + isbn + 978-1-4603-0192-3 + 9781460301923 + part-of + + + https://researchportal.tuni.fi/en/publications/11469981-5081-4337-9f58-416c93658b54 + conference-paper + + 2010 + + Proceedings of ACM International Conference on Bioinformatics and Computational Biology BCB10, Niagara Falls, New York, USA, August 2-4, 2010 + + + + 2022-05-25T22:51:07.819Z + + + source-work-id + 827ba6ea-0c57-487f-8a87-4923fcb57359 + 827ba6ea-0c57-487f-8a87-4923fcb57359 + self + + + + 2017-08-28T12:54:33.013Z + 2022-05-25T22:51:07.819Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational models for synaptic plasticity + + + + source-work-id + 827ba6ea-0c57-487f-8a87-4923fcb57359 + 827ba6ea-0c57-487f-8a87-4923fcb57359 + self + + + https://researchportal.tuni.fi/en/publications/827ba6ea-0c57-487f-8a87-4923fcb57359 + conference-paper + + 2010 + + 3rd INCF National Node of Finland Workshop on Neuroinformatics, Tampere, Finland, 21 September 2010 + + + + 2022-05-25T22:51:07.812Z + + + source-work-id + 1757d3d6-dc97-4f4f-bc1b-17d683c04655 + 1757d3d6-dc97-4f4f-bc1b-17d683c04655 + self + + + doi + 10.1186/1471-2202-11-S1-P190 + 10.1186/1471-2202-11-s1-p190 + self + + + + 2017-08-28T12:54:33.013Z + 2022-05-25T22:51:07.812Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Modeling signal transduction in synaptic plasticity: comparison of models and methods + + + + source-work-id + 1757d3d6-dc97-4f4f-bc1b-17d683c04655 + 1757d3d6-dc97-4f4f-bc1b-17d683c04655 + self + + + doi + 10.1186/1471-2202-11-S1-P190 + 10.1186/1471-2202-11-s1-p190 + self + + + https://researchportal.tuni.fi/en/publications/1757d3d6-dc97-4f4f-bc1b-17d683c04655 + other + + 2010 + + BMC Neuroscience + + + + 2022-05-25T22:51:07.805Z + + + source-work-id + 037195f0-048a-454f-8e05-cf0773b36159 + 037195f0-048a-454f-8e05-cf0773b36159 + self + + + + 2017-08-28T12:54:33.013Z + 2022-05-25T22:51:07.805Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Neurotransmitter and amyloid-β peptide induced calcium signaling in astrocytes + + + + source-work-id + 037195f0-048a-454f-8e05-cf0773b36159 + 037195f0-048a-454f-8e05-cf0773b36159 + self + + + https://researchportal.tuni.fi/en/publications/037195f0-048a-454f-8e05-cf0773b36159 + conference-paper + + 2010 + + Joint Meeting on New Horizons in Calcium Signaling, Beijing, China, 10-13 October 2010 + + + + 2022-05-25T22:51:07.797Z + + + eid + 2-s2.0-79959268899 + 2-s2.0-79959268899 + self + + + wosuid + 000288500300003 + 000288500300003 + self + + + pmid + 21188161 + 21188161 + self + + + source-work-id + 25785ff8-e2f0-412d-9f29-217888aaecff + 25785ff8-e2f0-412d-9f29-217888aaecff + self + + + pmc + PMC3006457 + 3006457 + self + + + doi + 10.3389/fncom.2010.00152 + 10.3389/fncom.2010.00152 + self + + + + 2016-09-21T17:41:34.842Z + 2022-05-25T04:26:20.975Z + + + https://orcid.org/client/0000-0002-9157-3431 + 0000-0002-9157-3431 + orcid.org + + Europe PubMed Central + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Postsynaptic signal transduction models for long-term potentiation and depression. + + + + pmid + 21188161 + 21188161 + self + + + pmc + PMC3006457 + 3006457 + self + + + doi + 10.3389/fncom.2010.00152 + 10.3389/fncom.2010.00152 + self + + + http://europepmc.org/abstract/med/21188161 + journal-article + + 2010 + + + + 2016-09-21T17:59:59.974Z + 2022-05-25T04:26:23.087Z + + + https://orcid.org/client/0000-0002-5982-8983 + 0000-0002-5982-8983 + orcid.org + + Scopus - Elsevier + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Postsynaptic signal transduction models for long-term potentiation and depression + + + + doi + 10.3389/fncom.2010.00152 + 10.3389/fncom.2010.00152 + self + + + eid + 2-s2.0-79959268899 + 2-s2.0-79959268899 + self + + + http://www.scopus.com/inward/record.url?eid=2-s2.0-79959268899&partnerID=MN8TOARS + journal-article + + 2010 + + Frontiers in Computational Neuroscience + + + 2017-08-28T12:54:33.012Z + 2022-05-25T22:51:07.797Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Postsynaptic signal transduction models for long-term potentiation and depression + + + + source-work-id + 25785ff8-e2f0-412d-9f29-217888aaecff + 25785ff8-e2f0-412d-9f29-217888aaecff + self + + + wosuid + 000288500300003 + 000288500300003 + self + + + eid + 2-s2.0-79959268899 + 2-s2.0-79959268899 + self + + + doi + 10.3389/fncom.2010.00152 + 10.3389/fncom.2010.00152 + self + + + https://researchportal.tuni.fi/en/publications/25785ff8-e2f0-412d-9f29-217888aaecff + journal-article + + 2010 + + Frontiers in Computational Neuroscience + + + + 2022-05-25T22:51:07.869Z + + + source-work-id + e06e9ac4-898a-43ad-8521-3d00e22c395c + e06e9ac4-898a-43ad-8521-3d00e22c395c + self + + + + 2017-08-28T12:54:33.015Z + 2022-05-25T22:51:07.869Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Modeling biochemical systems: analysis in time and frequency domain + + + + source-work-id + e06e9ac4-898a-43ad-8521-3d00e22c395c + e06e9ac4-898a-43ad-8521-3d00e22c395c + self + + + https://researchportal.tuni.fi/en/publications/e06e9ac4-898a-43ad-8521-3d00e22c395c + conference-paper + + 2009 + + Proceedings of FOSBE 2009, Foundations of Systems Biology in Engineering, Englewood, Colorado, USA, 9-12 August 2009 + + + + 2022-05-25T22:51:07.858Z + + + source-work-id + 88c6d7e2-e240-4f25-ae0c-2b259be70332 + 88c6d7e2-e240-4f25-ae0c-2b259be70332 + self + + + + 2017-08-28T12:54:33.015Z + 2022-05-25T22:51:07.858Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Modeling protein kinase C activation in the presence of calcium ion fluctuations + + + + source-work-id + 88c6d7e2-e240-4f25-ae0c-2b259be70332 + 88c6d7e2-e240-4f25-ae0c-2b259be70332 + self + + + isbn + 978-952-15-2160-7 + 9789521521607 + part-of + + + https://researchportal.tuni.fi/en/publications/88c6d7e2-e240-4f25-ae0c-2b259be70332 + conference-paper + + 2009 + + Proceedings of the Sixth International Workshop on Computational Systems Biology, WCSB 2009, Aarhus, Denmark 10-12 June 2009 + + + + 2022-05-25T22:51:07.852Z + + + source-work-id + b49e3b55-9e0f-4417-96e8-fd2d9dd60480 + b49e3b55-9e0f-4417-96e8-fd2d9dd60480 + self + + + isbn + 978-952-15-2160-7 + 9789521521607 + self + + + + 2017-08-28T12:54:33.014Z + 2022-05-25T22:51:07.852Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Proceedings of the 6th International Workshop on Computational Systems Biology, WCSB 2009, Aarhus, Denmark, 10-12 June 2009 + + + + source-work-id + b49e3b55-9e0f-4417-96e8-fd2d9dd60480 + b49e3b55-9e0f-4417-96e8-fd2d9dd60480 + self + + + isbn + 978-952-15-2160-7 + 9789521521607 + self + + + https://researchportal.tuni.fi/en/publications/b49e3b55-9e0f-4417-96e8-fd2d9dd60480 + edited-book + + 2009 + + Tampere University of Technology, Tampere International Center for Signal Processing, TICSP Series + + + + 2023-06-22T00:07:58.547Z + + + doi + 10.3389/conf.neuro.11.2008.01.049 + 10.3389/conf.neuro.11.2008.01.049 + self + + + source-work-id + 324ea0d7-ef0b-4b33-852e-141b5facbbaf + 324ea0d7-ef0b-4b33-852e-141b5facbbaf + self + + + + 2017-08-28T12:54:33.018Z + 2023-06-22T00:07:58.547Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Computational model for astroglial cell function in Alzheimer's disease + + + + source-work-id + 324ea0d7-ef0b-4b33-852e-141b5facbbaf + 324ea0d7-ef0b-4b33-852e-141b5facbbaf + self + + + doi + 10.3389/conf.neuro.11.2008.01.049 + 10.3389/conf.neuro.11.2008.01.049 + self + + + https://researchportal.tuni.fi/en/publications/324ea0d7-ef0b-4b33-852e-141b5facbbaf + other + + 2008 + + Frontiers in Neuroinformatics + + + + 2023-06-22T00:07:59.444Z + + + doi + 10.3389/conf.neuro.11.2008.01.033 + 10.3389/conf.neuro.11.2008.01.033 + self + + + source-work-id + 68e67e7d-1df6-4644-bfb3-97d8256aff02 + 68e67e7d-1df6-4644-bfb3-97d8256aff02 + self + + + + 2017-08-28T12:54:33.018Z + 2023-06-22T00:07:59.444Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Evaluation of deterministic and stochastic simulation tools for cellular signaling + + + + source-work-id + 68e67e7d-1df6-4644-bfb3-97d8256aff02 + 68e67e7d-1df6-4644-bfb3-97d8256aff02 + self + + + doi + 10.3389/conf.neuro.11.2008.01.033 + 10.3389/conf.neuro.11.2008.01.033 + self + + + https://researchportal.tuni.fi/en/publications/68e67e7d-1df6-4644-bfb3-97d8256aff02 + other + + 2008 + + Frontiers in Neuroinformatics + + + + 2022-05-25T22:51:07.915Z + + + source-work-id + fe9b7cdb-6bf1-4bc3-94fa-f131dd371ec6 + fe9b7cdb-6bf1-4bc3-94fa-f131dd371ec6 + self + + + + 2017-08-28T12:54:33.017Z + 2022-05-25T22:51:07.915Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Kemiallisten reaktioiden mallintaminen ja tietokonesimulaatiot + + + + source-work-id + fe9b7cdb-6bf1-4bc3-94fa-f131dd371ec6 + fe9b7cdb-6bf1-4bc3-94fa-f131dd371ec6 + self + + + https://researchportal.tuni.fi/en/publications/fe9b7cdb-6bf1-4bc3-94fa-f131dd371ec6 + journal-article + + 2008 + + Solubiologi + + + + 2022-05-25T22:51:07.902Z + + + source-work-id + 58a2acb5-8b9e-4b6e-a4f4-5104e25b7790 + 58a2acb5-8b9e-4b6e-a4f4-5104e25b7790 + self + + + + 2017-08-28T12:54:33.017Z + 2022-05-25T22:51:07.902Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Modeling IP3 receptor function using stochastic approaches + + + + source-work-id + 58a2acb5-8b9e-4b6e-a4f4-5104e25b7790 + 58a2acb5-8b9e-4b6e-a4f4-5104e25b7790 + self + + + isbn + 978-952-15-1988-8 + 9789521519888 + part-of + + + https://researchportal.tuni.fi/en/publications/58a2acb5-8b9e-4b6e-a4f4-5104e25b7790 + conference-paper + + 2008 + + Proceedings of the Fifth International Workshop on Computational Systems Biology, WCSB 2008, Leipzig, Germany, 11-13 June 2008 + + + + 2022-05-25T22:51:07.908Z + + + source-work-id + c2c6e3fb-5100-4171-a887-8f7492c11936 + c2c6e3fb-5100-4171-a887-8f7492c11936 + self + + + + 2017-08-28T12:54:33.017Z + 2022-05-25T22:51:07.908Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Modeling amyloid-β and serotonin induced calcium oscillations in astrocytes + + + + source-work-id + c2c6e3fb-5100-4171-a887-8f7492c11936 + c2c6e3fb-5100-4171-a887-8f7492c11936 + self + + + https://researchportal.tuni.fi/en/publications/c2c6e3fb-5100-4171-a887-8f7492c11936 + conference-paper + + 2008 + + Abstracts of Neuroscience 2008, Washington D. C., 15-19 November 2008 + + + + 2022-05-25T22:51:07.895Z + + + source-work-id + 3a0e3c05-dcd6-40fb-8698-f684c74a655f + 3a0e3c05-dcd6-40fb-8698-f684c74a655f + self + + + doi + 10.3389/conf.neuro.11.2008.01.050 + 10.3389/conf.neuro.11.2008.01.050 + self + + + + 2017-08-28T12:54:33.016Z + 2022-05-25T22:51:07.895Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Modeling long-term potentiation: deterministic and stochastic approaches + + + + source-work-id + 3a0e3c05-dcd6-40fb-8698-f684c74a655f + 3a0e3c05-dcd6-40fb-8698-f684c74a655f + self + + + doi + 10.3389/conf.neuro.11.2008.01.050 + 10.3389/conf.neuro.11.2008.01.050 + self + + + https://researchportal.tuni.fi/en/publications/3a0e3c05-dcd6-40fb-8698-f684c74a655f + other + + 2008 + + Frontiers in Neuroinformatics + + + + 2022-05-25T22:51:07.889Z + + + source-work-id + 3492d857-b278-4bcc-825a-e3f3aa0d89f2 + 3492d857-b278-4bcc-825a-e3f3aa0d89f2 + self + + + + 2017-08-28T12:54:33.016Z + 2022-05-25T22:51:07.889Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Modeling perspective to Ca2+ oscillations in astrocytes induced by amyloid-Β peptide and serotonin + + + + source-work-id + 3492d857-b278-4bcc-825a-e3f3aa0d89f2 + 3492d857-b278-4bcc-825a-e3f3aa0d89f2 + self + + + https://researchportal.tuni.fi/en/publications/3492d857-b278-4bcc-825a-e3f3aa0d89f2 + conference-paper + + 2008 + + Workshop on Update in Alzheimer's Disease Research, Westminister, MD, USA, 12-14 November 2008 + + + + 2022-05-25T22:51:07.881Z + + + source-work-id + d9dad844-5750-4a90-a5b1-da82d957b4b3 + d9dad844-5750-4a90-a5b1-da82d957b4b3 + self + + + + 2017-08-28T12:54:33.016Z + 2022-05-25T22:51:07.881Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Stochastic kinetic simulations of activity-dependent plastic modifications in neurons + + + + source-work-id + d9dad844-5750-4a90-a5b1-da82d957b4b3 + d9dad844-5750-4a90-a5b1-da82d957b4b3 + self + + + isbn + 978-952-15-1988-8 + 9789521519888 + part-of + + + https://researchportal.tuni.fi/en/publications/d9dad844-5750-4a90-a5b1-da82d957b4b3 + conference-paper + + 2008 + + Proceedings of the Fifth International Workshop on Computational Systems Biology, WCSB 2008, Leipzig, Germany, 11-13 June 2008 + + + + 2023-06-22T00:07:57.945Z + + + doi + 10.3389/conf.neuro.11.2008.01.023 + 10.3389/conf.neuro.11.2008.01.023 + self + + + source-work-id + 65383231-f662-4285-9e33-607fe2e11edf + 65383231-f662-4285-9e33-607fe2e11edf + self + + + + 2017-08-28T12:54:33.015Z + 2023-06-22T00:07:57.945Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Stochastic modeling of neuronal signaling + + + + source-work-id + 65383231-f662-4285-9e33-607fe2e11edf + 65383231-f662-4285-9e33-607fe2e11edf + self + + + doi + 10.3389/conf.neuro.11.2008.01.023 + 10.3389/conf.neuro.11.2008.01.023 + self + + + https://researchportal.tuni.fi/en/publications/65383231-f662-4285-9e33-607fe2e11edf + other + + 2008 + + Frontiers in Neuroinformatics + + + + 2022-05-25T22:51:07.952Z + + + eid + 2-s2.0-47049105095 + 2-s2.0-47049105095 + self + + + source-work-id + 5aed0483-22cf-4b36-ad12-d3ae82595e9a + 5aed0483-22cf-4b36-ad12-d3ae82595e9a + self + + + doi + 10.1109/GENSIPS.2007.4365824 + 10.1109/gensips.2007.4365824 + self + + + + 2016-09-21T18:00:00.517Z + 2022-05-25T04:26:23.121Z + + + https://orcid.org/client/0000-0002-5982-8983 + 0000-0002-5982-8983 + orcid.org + + Scopus - Elsevier + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Estimation of neuronal signaling model parameters using deterministic and stochastic in silico training data: Evaluation of four parameter estimation methods + + + + doi + 10.1109/GENSIPS.2007.4365824 + 10.1109/gensips.2007.4365824 + self + + + eid + 2-s2.0-47049105095 + 2-s2.0-47049105095 + self + + + http://www.scopus.com/inward/record.url?eid=2-s2.0-47049105095&partnerID=MN8TOARS + conference-paper + + 2007 + + GENSIPS'07 - 5th IEEE International Workshop on Genomic Signal Processing and Statistics + + + 2017-08-28T12:54:33.019Z + 2022-05-25T22:51:07.952Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Estimation of neuronal signaling model parameters using deterministic and stochastic in silico training data: evaluation of four parameter estimation methods + + + + source-work-id + 5aed0483-22cf-4b36-ad12-d3ae82595e9a + 5aed0483-22cf-4b36-ad12-d3ae82595e9a + self + + + eid + 2-s2.0-47049105095 + 2-s2.0-47049105095 + self + + + doi + 10.1109/GENSIPS.2007.4365824 + 10.1109/gensips.2007.4365824 + self + + + isbn + 1-4244-0999-3 + 1424409993 + part-of + + + https://researchportal.tuni.fi/en/publications/5aed0483-22cf-4b36-ad12-d3ae82595e9a + conference-paper + + 2007 + + Proceedings of the Fifth IEEE International Workshop on Genomic Signal Processing and Statistics, GENSIPS'07, Tuusula, Finland, 10-12 June 2007 + + + + 2022-05-25T22:51:07.946Z + + + doi + 10.1109/GENSIPS.2007.4365829 + 10.1109/gensips.2007.4365829 + self + + + eid + 2-s2.0-47149109445 + 2-s2.0-47149109445 + self + + + source-work-id + 7033b426-61c6-49e4-8eb8-b783e5e729a2 + 7033b426-61c6-49e4-8eb8-b783e5e729a2 + self + + + + 2017-08-28T12:54:33.019Z + 2022-05-25T22:51:07.946Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Parameter estimation and tuning of firefly luciferase pathway model + + + + source-work-id + 7033b426-61c6-49e4-8eb8-b783e5e729a2 + 7033b426-61c6-49e4-8eb8-b783e5e729a2 + self + + + eid + 2-s2.0-47149109445 + 2-s2.0-47149109445 + self + + + doi + 10.1109/GENSIPS.2007.4365829 + 10.1109/gensips.2007.4365829 + self + + + isbn + 1-4244-0999-3 + 1424409993 + part-of + + + https://researchportal.tuni.fi/en/publications/7033b426-61c6-49e4-8eb8-b783e5e729a2 + conference-paper + + 2007 + + Proceedings of the Fifth IEEE International Workshop on Genomic Signal Processing and Statistics, GENSIPS'07, Tuusula, Finland, 10-12 June 2007 + + + + 2022-05-25T22:51:07.940Z + + + source-work-id + 457bf799-29c4-4749-b7f8-2356bdf68716 + 457bf799-29c4-4749-b7f8-2356bdf68716 + self + + + + 2017-08-28T12:54:33.019Z + 2022-05-25T22:51:07.940Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Sequential Monte Carlo based maximum likelihood estimation for calcium binding reactions + + + + source-work-id + 457bf799-29c4-4749-b7f8-2356bdf68716 + 457bf799-29c4-4749-b7f8-2356bdf68716 + self + + + isbn + 978-3-8167-7436-5 + 9783816774365 + part-of + + + https://researchportal.tuni.fi/en/publications/457bf799-29c4-4749-b7f8-2356bdf68716 + conference-paper + + 2007 + + Proceedings of the 2nd Conference on Foundations of Systems Biology in Engineering, FOSBE 2007, Stuttgart, Germany, 9-12 September 2007 + + + + 2022-05-25T22:51:07.933Z + + + isbn + 978-952-15-1896-6 + 9789521518966 + self + + + source-work-id + 3f9fdf0c-7f7d-4717-a196-55565103963a + 3f9fdf0c-7f7d-4717-a196-55565103963a + self + + + + 2017-08-28T12:54:33.018Z + 2022-05-25T22:51:07.933Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Stochastic methods for modeling intracellular signaling + + + + source-work-id + 3f9fdf0c-7f7d-4717-a196-55565103963a + 3f9fdf0c-7f7d-4717-a196-55565103963a + self + + + isbn + 978-952-15-1896-6 + 9789521518966 + self + + + https://researchportal.tuni.fi/en/publications/3f9fdf0c-7f7d-4717-a196-55565103963a + dissertation-thesis + + 2007 + + Tampereen teknillinen yliopisto. Julkaisu + + + + 2022-05-27T18:17:13.848Z + + + source-work-id + 39227083-a66e-4cd3-a865-795792ffdd3b + 39227083-a66e-4cd3-a865-795792ffdd3b + self + + + + 2019-01-11T23:52:06.183Z + 2022-05-27T18:17:13.848Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Stochastic simulation tools for cellular signaling: survey, evaluation and quantitative analysis + + + + source-work-id + 39227083-a66e-4cd3-a865-795792ffdd3b + 39227083-a66e-4cd3-a865-795792ffdd3b + self + + + isbn + 978-3-8167-7436-5 + 9783816774365 + part-of + + + https://researchportal.tuni.fi/en/publications/39227083-a66e-4cd3-a865-795792ffdd3b + conference-paper + + 2007 + + Proceedings of the 2nd Conference on Foundations of Systems Biology in Engineering, FOSBE 2007, Stuttgart, Germany, 9-12 September 2007 + + + + 2023-06-22T00:07:56.680Z + + + doi + 10.1016/j.compbiolchem.2006.04.002 + 10.1016/j.compbiolchem.2006.04.002 + self + + + pmid + 16880117 + 16880117 + self + + + source-work-id + fe9c9f72-2eec-4d11-9277-c45bf2debb9c + fe9c9f72-2eec-4d11-9277-c45bf2debb9c + self + + + eid + 2-s2.0-33746373126 + 2-s2.0-33746373126 + self + + + wosuid + 000239876500005 + 000239876500005 + self + + + + 2016-09-21T17:41:34.857Z + 2022-05-25T04:26:20.981Z + + + https://orcid.org/client/0000-0002-9157-3431 + 0000-0002-9157-3431 + orcid.org + + Europe PubMed Central + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Developing Ito stochastic differential equation models for neuronal signal transduction pathways. + + + + pmid + 16880117 + 16880117 + self + + + doi + 10.1016/j.compbiolchem.2006.04.002 + 10.1016/j.compbiolchem.2006.04.002 + self + + + http://europepmc.org/abstract/med/16880117 + journal-article + + 2006 + 08 + + + + 2016-09-21T17:59:59.990Z + 2022-05-25T04:26:23.098Z + + + https://orcid.org/client/0000-0002-5982-8983 + 0000-0002-5982-8983 + orcid.org + + Scopus - Elsevier + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Developing Itô stochastic differential equation models for neuronal signal transduction pathways + + + + doi + 10.1016/j.compbiolchem.2006.04.002 + 10.1016/j.compbiolchem.2006.04.002 + self + + + eid + 2-s2.0-33746373126 + 2-s2.0-33746373126 + self + + + http://www.scopus.com/inward/record.url?eid=2-s2.0-33746373126&partnerID=MN8TOARS + journal-article + + 2006 + + Computational Biology and Chemistry + + + 2017-08-28T12:54:33.021Z + 2023-06-22T00:07:56.680Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Developing Ito stochastic differential equation models for neuronal signal transduction pathways + + + + source-work-id + fe9c9f72-2eec-4d11-9277-c45bf2debb9c + fe9c9f72-2eec-4d11-9277-c45bf2debb9c + self + + + wosuid + 000239876500005 + 000239876500005 + self + + + eid + 2-s2.0-33746373126 + 2-s2.0-33746373126 + self + + + doi + 10.1016/j.compbiolchem.2006.04.002 + 10.1016/j.compbiolchem.2006.04.002 + self + + + https://researchportal.tuni.fi/en/publications/fe9c9f72-2eec-4d11-9277-c45bf2debb9c + journal-article + + 2006 + + Computational Biology and Chemistry + + + + 2022-05-25T22:51:07.988Z + + + source-work-id + b0f55489-2d6b-49fa-802a-32210e0e40b3 + b0f55489-2d6b-49fa-802a-32210e0e40b3 + self + + + doi + 10.1016/j.neucom.2005.12.047 + 10.1016/j.neucom.2005.12.047 + self + + + eid + 2-s2.0-33646104196 + 2-s2.0-33646104196 + self + + + wosuid + 000237873900016 + 000237873900016 + self + + + + 2016-09-21T18:00:00.508Z + 2022-05-25T04:26:23.115Z + + + https://orcid.org/client/0000-0002-5982-8983 + 0000-0002-5982-8983 + orcid.org + + Scopus - Elsevier + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + A novel approach to model neuronal signal transduction using stochastic differential equations + + + + doi + 10.1016/j.neucom.2005.12.047 + 10.1016/j.neucom.2005.12.047 + self + + + eid + 2-s2.0-33646104196 + 2-s2.0-33646104196 + self + + + http://www.scopus.com/inward/record.url?eid=2-s2.0-33646104196&partnerID=MN8TOARS + journal-article + + 2006 + + Neurocomputing + + + 2017-08-28T12:54:33.021Z + 2022-05-25T22:51:07.988Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + A novel approach to model neuronal signal transduction using stochastic differential equations + + + + source-work-id + b0f55489-2d6b-49fa-802a-32210e0e40b3 + b0f55489-2d6b-49fa-802a-32210e0e40b3 + self + + + wosuid + 000237873900016 + 000237873900016 + self + + + eid + 2-s2.0-33646104196 + 2-s2.0-33646104196 + self + + + doi + 10.1016/j.neucom.2005.12.047 + 10.1016/j.neucom.2005.12.047 + self + + + https://researchportal.tuni.fi/en/publications/b0f55489-2d6b-49fa-802a-32210e0e40b3 + journal-article + + 2006 + + Neurocomputing + + + + 2022-05-25T22:51:07.976Z + + + source-work-id + ebfcfc56-4b0c-46ae-9f83-f751c6587fec + ebfcfc56-4b0c-46ae-9f83-f751c6587fec + self + + + doi + 10.1109/iembs.2006.260023 + 10.1109/iembs.2006.260023 + self + + + pmid + 17945691 + 17945691 + self + + + eid + 2-s2.0-34047152367 + 2-s2.0-34047152367 + self + + + + 2016-09-21T17:41:34.862Z + 2022-05-25T04:26:20.987Z + + + https://orcid.org/client/0000-0002-9157-3431 + 0000-0002-9157-3431 + orcid.org + + Europe PubMed Central + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Discrete stochastic simulation of cell signaling: comparison of computational tools. + + + + pmid + 17945691 + 17945691 + self + + + doi + 10.1109/iembs.2006.260023 + 10.1109/iembs.2006.260023 + self + + + http://europepmc.org/abstract/med/17945691 + journal-article + + 2006 + + + + 2016-09-21T18:00:00.503Z + 2022-05-25T04:26:23.110Z + + + https://orcid.org/client/0000-0002-5982-8983 + 0000-0002-5982-8983 + orcid.org + + Scopus - Elsevier + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Discrete stochastic simulation of cell signaling: Comparison of computational tools + + + + doi + 10.1109/IEMBS.2006.260023 + 10.1109/iembs.2006.260023 + self + + + eid + 2-s2.0-34047152367 + 2-s2.0-34047152367 + self + + + http://www.scopus.com/inward/record.url?eid=2-s2.0-34047152367&partnerID=MN8TOARS + conference-paper + + 2006 + + Annual International Conference of the IEEE Engineering in Medicine and Biology - Proceedings + + + 2017-08-28T12:54:33.020Z + 2022-05-25T22:51:07.976Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Discrete stochastic simulation of cell signaling: comparison of computational tools + + + + source-work-id + ebfcfc56-4b0c-46ae-9f83-f751c6587fec + ebfcfc56-4b0c-46ae-9f83-f751c6587fec + self + + + eid + 2-s2.0-34047152367 + 2-s2.0-34047152367 + self + + + doi + 10.1109/IEMBS.2006.260023 + 10.1109/iembs.2006.260023 + self + + + https://researchportal.tuni.fi/en/publications/ebfcfc56-4b0c-46ae-9f83-f751c6587fec + conference-paper + + 2006 + + Proceedings of the 28th Annual International Conference of the IEEE Engineering in Medicine and Biology Society, EMBC, New York City, USA, August 30 - September 3, 2006 + + + + 2022-05-25T22:51:07.970Z + + + source-work-id + a5255c0c-509d-4d95-ad9f-d9b6174335e1 + a5255c0c-509d-4d95-ad9f-d9b6174335e1 + self + + + + 2017-08-28T12:54:33.020Z + 2022-05-25T22:51:07.970Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Modeling neuronal signal transduction using Itô stochastic differential equations and the Gillespie stochastic simulation algorithm + + + + source-work-id + a5255c0c-509d-4d95-ad9f-d9b6174335e1 + a5255c0c-509d-4d95-ad9f-d9b6174335e1 + self + + + https://researchportal.tuni.fi/en/publications/a5255c0c-509d-4d95-ad9f-d9b6174335e1 + conference-paper + + 2006 + + Proceedings of the IASTED International Conference on Computational and Systems Biology, Dallas, Texas, USA, 13-14 November 2006 + + + + 2022-05-25T22:51:07.965Z + + + source-work-id + 459fc421-8904-41fb-bdba-6feab0050a41 + 459fc421-8904-41fb-bdba-6feab0050a41 + self + + + isbn + 952-15-1604-6 + 9521516046 + self + + + + 2017-08-28T12:54:33.020Z + 2022-05-25T22:51:07.965Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Proceedings of the 4th TICSP Workshop on Computational Systems Biology, WCSB 2006, Tampere, Finland, 12-13 June 2006 + + + + source-work-id + 459fc421-8904-41fb-bdba-6feab0050a41 + 459fc421-8904-41fb-bdba-6feab0050a41 + self + + + isbn + 952-15-1604-6 + 9521516046 + self + + + https://researchportal.tuni.fi/en/publications/459fc421-8904-41fb-bdba-6feab0050a41 + edited-book + + 2006 + + Tampere University Of Technology, Tampere International Center for Signal Processing, TICSP Series + + + + 2022-05-25T22:51:07.958Z + + + source-work-id + 6e35b2c1-6be6-4d84-a9bc-ae4dd89bbf2f + 6e35b2c1-6be6-4d84-a9bc-ae4dd89bbf2f + self + + + eid + 2-s2.0-33748782266 + 2-s2.0-33748782266 + self + + + + 2017-08-28T12:54:33.020Z + 2022-05-25T22:51:07.958Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Simulation study of deterministic differential equation model for protein kinase C signaling: sensitivity of stimuli, parameter values, and initial concentrations + + + + source-work-id + 6e35b2c1-6be6-4d84-a9bc-ae4dd89bbf2f + 6e35b2c1-6be6-4d84-a9bc-ae4dd89bbf2f + self + + + eid + 2-s2.0-33748782266 + 2-s2.0-33748782266 + self + + + https://researchportal.tuni.fi/en/publications/6e35b2c1-6be6-4d84-a9bc-ae4dd89bbf2f + conference-paper + + 2006 + + Proceedings of the 4th TICSP Workshop on Computational Systems Biology, WCSB 2006, Tampere, Finland, 12-13 June 2006. TICSP series + + + + 2022-05-25T22:51:07.995Z + + + wosuid + 000226605700011 + 000226605700011 + self + + + doi + 10.1093/bioinformatics/bti018 + 10.1093/bioinformatics/bti018 + self + + + source-work-id + 3709737f-2da0-4a4e-ad32-e7957d27620f + 3709737f-2da0-4a4e-ad32-e7957d27620f + self + + + pmid + 15358616 + 15358616 + self + + + eid + 2-s2.0-13844266102 + 2-s2.0-13844266102 + self + + + + 2016-09-21T17:41:34.868Z + 2022-05-25T04:26:20.992Z + + + https://orcid.org/client/0000-0002-9157-3431 + 0000-0002-9157-3431 + orcid.org + + Europe PubMed Central + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Simulation tools for biochemical networks: evaluation of performance and usability. + + + + pmid + 15358616 + 15358616 + self + + + doi + 10.1093/bioinformatics/bti018 + 10.1093/bioinformatics/bti018 + self + + + http://europepmc.org/abstract/med/15358616 + journal-article + + 2005 + 02 + + + + 2016-09-21T17:59:59.995Z + 2022-05-25T04:26:23.104Z + + + https://orcid.org/client/0000-0002-5982-8983 + 0000-0002-5982-8983 + orcid.org + + Scopus - Elsevier + + https://orcid.org/0000-0002-0456-1185 + 0000-0002-0456-1185 + orcid.org + + Tiina Manninen + + + Simulation tools for biochemical networks: Evaluation of performance and usability + + + + doi + 10.1093/bioinformatics/bti018 + 10.1093/bioinformatics/bti018 + self + + + eid + 2-s2.0-13844266102 + 2-s2.0-13844266102 + self + + + http://www.scopus.com/inward/record.url?eid=2-s2.0-13844266102&partnerID=MN8TOARS + journal-article + + 2005 + + Bioinformatics + + + 2017-08-28T12:54:33.021Z + 2022-05-25T22:51:07.995Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Simulation tools for biochemical networks: evaluation of performance and usability + + + + source-work-id + 3709737f-2da0-4a4e-ad32-e7957d27620f + 3709737f-2da0-4a4e-ad32-e7957d27620f + self + + + wosuid + 000226605700011 + 000226605700011 + self + + + eid + 2-s2.0-13844266102 + 2-s2.0-13844266102 + self + + + doi + 10.1093/bioinformatics/bti018 + 10.1093/bioinformatics/bti018 + self + + + https://researchportal.tuni.fi/en/publications/3709737f-2da0-4a4e-ad32-e7957d27620f + journal-article + + 2005 + + Bioinformatics + + + + 2022-05-25T22:51:08.014Z + + + source-work-id + 9b7fc5e1-5442-41d1-9319-e159defcec5a + 9b7fc5e1-5442-41d1-9319-e159defcec5a + self + + + + 2017-08-28T12:54:33.022Z + 2022-05-25T22:51:08.014Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + A program to obtain signal transduction models from a database + + + + source-work-id + 9b7fc5e1-5442-41d1-9319-e159defcec5a + 9b7fc5e1-5442-41d1-9319-e159defcec5a + self + + + https://researchportal.tuni.fi/en/publications/9b7fc5e1-5442-41d1-9319-e159defcec5a + conference-paper + + 2005 + + Proceedings of The 3rd TICSP Workshop on Computational Systems Biology, WCSB 2005, Tampere, Finland, 13-14 June 2005 + + + + 2022-05-25T22:51:08.008Z + + + eid + 2-s2.0-19644387496 + 2-s2.0-19644387496 + self + + + source-work-id + 301a2145-5b51-46dc-bd0e-143568c0f95f + 301a2145-5b51-46dc-bd0e-143568c0f95f + self + + + + 2017-08-28T12:54:33.022Z + 2022-05-25T22:51:08.008Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Incorporating stochasticity into deterministic differential equation models for neuronal signal transduction + + + + source-work-id + 301a2145-5b51-46dc-bd0e-143568c0f95f + 301a2145-5b51-46dc-bd0e-143568c0f95f + self + + + eid + 2-s2.0-19644387496 + 2-s2.0-19644387496 + self + + + https://researchportal.tuni.fi/en/publications/301a2145-5b51-46dc-bd0e-143568c0f95f + conference-paper + + 2005 + + Proceedings of The 3rd TICSP Workshop on Computational Systems Biology, WCSB 2005, Tampere, Finland, 13-14 June 2005 + + + + 2022-05-25T22:51:08.001Z + + + isbn + 952-15-1365-9 + 9521513659 + self + + + source-work-id + b495a6ed-c0bc-4ae0-a25f-5b8680a1d8b9 + b495a6ed-c0bc-4ae0-a25f-5b8680a1d8b9 + self + + + + 2017-08-28T12:54:33.022Z + 2022-05-25T22:51:08.001Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Proceedings of The 3rd TICSP Workshop on Computational Systems Biology, WCSB 2005 + + + + source-work-id + b495a6ed-c0bc-4ae0-a25f-5b8680a1d8b9 + b495a6ed-c0bc-4ae0-a25f-5b8680a1d8b9 + self + + + isbn + 952-15-1365-9 + 9521513659 + self + + + https://researchportal.tuni.fi/en/publications/b495a6ed-c0bc-4ae0-a25f-5b8680a1d8b9 + edited-book + + 2005 + + Tampere University of Technology, Tampere International Center for Signal Processing, TICSP Series + + + + 2022-05-25T22:51:08.037Z + + + source-work-id + 2b12d8d5-ffe1-4e5f-95a8-374fb4ccf8f3 + 2b12d8d5-ffe1-4e5f-95a8-374fb4ccf8f3 + self + + + doi + 10.1016/j.neucom.2004.01.096 + 10.1016/j.neucom.2004.01.096 + self + + + eid + 2-s2.0-2542458658 + 2-s2.0-2542458658 + self + + + + 2017-08-28T12:54:33.027Z + 2022-05-25T22:51:08.037Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + A model integrating the cerebellar granule neuron excitability and calcium signaling pathways + + + + source-work-id + 2b12d8d5-ffe1-4e5f-95a8-374fb4ccf8f3 + 2b12d8d5-ffe1-4e5f-95a8-374fb4ccf8f3 + self + + + eid + 2-s2.0-2542458658 + 2-s2.0-2542458658 + self + + + doi + 10.1016/j.neucom.2004.01.096 + 10.1016/j.neucom.2004.01.096 + self + + + https://researchportal.tuni.fi/en/publications/2b12d8d5-ffe1-4e5f-95a8-374fb4ccf8f3 + journal-article + + 2004 + + Neurocomputing + + + + 2023-06-22T00:07:57.358Z + + + source-work-id + fc7e7945-a58c-49b6-996b-ece34f727e48 + fc7e7945-a58c-49b6-996b-ece34f727e48 + self + + + + 2017-08-28T12:54:33.026Z + 2023-06-22T00:07:57.358Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Pienryhmä parantaa matematiikan taitoja + + + + source-work-id + fc7e7945-a58c-49b6-996b-ece34f727e48 + fc7e7945-a58c-49b6-996b-ece34f727e48 + self + + + https://researchportal.tuni.fi/en/publications/fc7e7945-a58c-49b6-996b-ece34f727e48 + journal-article + + 2004 + + Dimensio + + + + 2022-05-25T22:51:08.026Z + + + isbn + 952-15-1301-2 + 9521513012 + self + + + source-work-id + 6bcb454e-5fde-4aeb-b37e-557bd27c9e83 + 6bcb454e-5fde-4aeb-b37e-557bd27c9e83 + self + + + + 2017-08-28T12:54:33.026Z + 2022-05-25T22:51:08.026Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Simulation study of different equation model for protein kinase C signaling + + + + source-work-id + 6bcb454e-5fde-4aeb-b37e-557bd27c9e83 + 6bcb454e-5fde-4aeb-b37e-557bd27c9e83 + self + + + isbn + 952-15-1301-2 + 9521513012 + self + + + https://researchportal.tuni.fi/en/publications/6bcb454e-5fde-4aeb-b37e-557bd27c9e83 + report + + 2004 + + Tampere University of Technology, Institute of Signal Processing, Report + + + + 2022-05-25T22:51:08.019Z + + + source-work-id + df8ad827-d40d-495b-9173-21b49a9ae193 + df8ad827-d40d-495b-9173-21b49a9ae193 + self + + + + 2017-08-28T12:54:33.023Z + 2022-05-25T22:51:08.019Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Varying stimulus and parameter values in the PKC pathway model + + + + source-work-id + df8ad827-d40d-495b-9173-21b49a9ae193 + df8ad827-d40d-495b-9173-21b49a9ae193 + self + + + https://researchportal.tuni.fi/en/publications/df8ad827-d40d-495b-9173-21b49a9ae193 + conference-paper + + 2004 + + Proceedings of the 2nd TICSP Workshop on Computational Systems Biology, WCSB'2004, Silja Opera, Helsinki-St.Petersburg, 14-16 June 2004 + + + + 2022-05-25T22:51:08.042Z + + + source-work-id + ca0966d2-a83d-42f5-9905-b6e84115ac54 + ca0966d2-a83d-42f5-9905-b6e84115ac54 + self + + + + 2017-08-28T12:54:33.027Z + 2022-05-25T22:51:08.042Z + + + https://orcid.org/client/APP-N0TAO4G9BBK9PWHT + APP-N0TAO4G9BBK9PWHT + orcid.org + + Tampere University + + + Hermosolun solunsisäisten toimintojen matemaattinen mallinnus ja simulointi + + + + source-work-id + ca0966d2-a83d-42f5-9905-b6e84115ac54 + ca0966d2-a83d-42f5-9905-b6e84115ac54 + self + + + https://researchportal.tuni.fi/en/publications/ca0966d2-a83d-42f5-9905-b6e84115ac54 + other + + 2003 + + + + diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/plugin/base/base-sample.tar b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/plugin/base/base-sample.tar new file mode 100644 index 000000000..c575fe147 Binary files /dev/null and b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/plugin/base/base-sample.tar differ diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/plugin/base/record.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/plugin/base/record.xml new file mode 100644 index 000000000..9c58941fd --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/plugin/base/record.xml @@ -0,0 +1,58 @@ + +
+ ftdoajarticles:oai:doaj.org/article:e2d5b5126b2d4e479933cc7f9a9ae0c1 + 2022-12-31T11:48:55Z +
+ + + ftdoajarticles:oai:doaj.org/article:e2d5b5126b2d4e479933cc7f9a9ae0c1 + cww + org + ftdoajarticles + TEST REPO + Assessment of cultural heritage: the legislative and methodological framework of Russian Federation + ALBU, Svetlana + LEȘAN, Anna + architectural heritage + evaluation of architectural heritage + types of values + experience of russian federation + Social Sciences + H + Architectural heritage is the real estate inheritance by population of a country becoming an extremely valuable and specific category, preserving and capitalizing on those assets requires considerable effort. The state does not have sufficient means to maintain and preserve cultural heritage, as a result it is included in the civil circuit. The transfer of property right or of some partial rights over the architectural patrimony is accompanied by the necessity to estimate the value of goods. In this article, the authors examine the experience of Russian Federation (one of the largest countries with a huge architectural heritage) on the legislative framework of architectural and methodological heritage of architectural heritage assessment. The particularities of cultural assets valuation compared to other categories of real estate are examined, as well as the methodological aspects (types of values, methods applied in valuation, approaches according to the purpose of valuation) regarding the valuation of real estate with architectural value in Russian Federation. + Technical University of Moldova + 2020-09-01T00:00:00Z + 2020 + article + 121 + https://doi.org/10.5281/zenodo.3971988 + https://doaj.org/article/e2d5b5126b2d4e479933cc7f9a9ae0c1 + https://doi.org/10.5281/zenodo.3971988 + Journal of Social Sciences, Vol 3, Iss 3, Pp 134-143 (2020) + EN + FR + RO + http://ibn.idsi.md/sites/default/files/imag_file/JSS-3-2020_134-143.pdf + https://doaj.org/toc/2587-3490 + https://doaj.org/toc/2587-3504 + doi:10.5281/zenodo.3971988 + 2587-3490 + 2587-3504 + https://doaj.org/article/e2d5b5126b2d4e479933cc7f9a9ae0c1 + 720 + + ALBU, Svetlana + https://orcid.org/0000-0002-8648-950X + + + LEȘAN, Anna + https://orcid.org/0000-0003-3284-0525 + + https://doi.org/10.5281/zenodo.3971988 + 1 + eng + fre + rum + + +
diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/plugin/base/record_wrong_1.xml b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/plugin/base/record_wrong_1.xml new file mode 100644 index 000000000..f6dccc006 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/collection/plugin/base/record_wrong_1.xml @@ -0,0 +1,36 @@ + + +
+ ftunivminnesdc:oai:conservancy.umn.edu:11299/109914 + 2023-07-18T20:05:40Z +
+ + + ftunivminnesdc:oai:conservancy.umn.edu:11299/109914 + cna + us + ftunivminnesdc + University of Minnesota Digital Conservancy + An Experimental Investigation of the influence of an air bubble layer on radiated noise and surface pressure fluctuations in a turbulent boundary layer + Killen, John M. + Boundary layer noise + Kinetic energy dissipation + Air bubble layer + Maximum stable bubble size + bubble noise spectra + between the noise spectra of a layer of air bubbles in a turbulent flow and' a maximum stable bubble size which can exist in the same flow. An air bubbie layer with individual bubble sizes greater than a maximum stable size was introduced into the boundary layer of water flowing along a smooth flat plate. It was found that the maximum stable bubble size was related to the turbulent kinetic energy dissipation as had been previously shown by other investigators ; Office of Naval Research Arlington, Virginia 22217 + 1981-09-31T21:42:52Z + 1981 + Report + 14 + http://purl.umn.edu/109914 + http://purl.umn.edu/109914 + en_US + Project Reports + 202 + http://purl.umn.edu/109914 + 2 + eng + + +
\ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/synonyms.txt b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/synonyms.txt index 74a75e46a..6b7979a6b 100644 --- a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/synonyms.txt +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/synonyms.txt @@ -1496,4 +1496,30 @@ cnr:institutes @=@ __CDS131__ @=@ IBE - Istituto per la BioEconomia cnr:institutes @=@ https://ror.org/0263zy895 @=@ CDS132 cnr:institutes @=@ https://ror.org/0263zy895 @=@ SCITEC - Istituto di Scienze e Tecnologie Chimiche \"Giulio Natta\" cnr:institutes @=@ __CDS133__ @=@ CDS133 -cnr:institutes @=@ __CDS133__ @=@ STEMS - Istituto di Scienze e Tecnologie per l'Energia e la Mobilità Sostenibili \ No newline at end of file +cnr:institutes @=@ __CDS133__ @=@ STEMS - Istituto di Scienze e Tecnologie per l'Energia e la Mobilità Sostenibili +base:normalized_types @=@ Text @=@ 1 +base:normalized_types @=@ Book @=@ 11 +base:normalized_types @=@ Book part @=@ 111 +base:normalized_types @=@ Journal/Newspaper @=@ 12 +base:normalized_types @=@ Article contribution @=@ 121 +base:normalized_types @=@ Other non-article @=@ 122 +base:normalized_types @=@ Conference object @=@ 13 +base:normalized_types @=@ Report @=@ 14 +base:normalized_types @=@ Review @=@ 15 +base:normalized_types @=@ Course material @=@ 16 +base:normalized_types @=@ Lecture @=@ 17 +base:normalized_types @=@ Thesis @=@ 18 +base:normalized_types @=@ Bachelor's thesis @=@ 181 +base:normalized_types @=@ Master's thesis @=@ 182 +base:normalized_types @=@ Doctoral and postdoctoral thesis @=@ 183 +base:normalized_types @=@ Manuscript @=@ 19 +base:normalized_types @=@ Patent @=@ 1A +base:normalized_types @=@ Musical notation @=@ 2 +base:normalized_types @=@ Map @=@ 3 +base:normalized_types @=@ Audio @=@ 4 +base:normalized_types @=@ Image/Video @=@ 5 +base:normalized_types @=@ Still image @=@ 51 +base:normalized_types @=@ Moving image/Video @=@ 52 +base:normalized_types @=@ Software @=@ 6 +base:normalized_types @=@ Dataset @=@ 7 +base:normalized_types @=@ Unknown @=@ F diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/terms.txt b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/terms.txt index 0f0ebaad4..29e42ffd9 100644 --- a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/terms.txt +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/transform/terms.txt @@ -1210,4 +1210,29 @@ cnr:institutes @=@ cnr:institutes @=@ __CDS130__ @=@ __CDS130__ cnr:institutes @=@ cnr:institutes @=@ __CDS131__ @=@ __CDS131__ cnr:institutes @=@ cnr:institutes @=@ https://ror.org/0263zy895 @=@ https://ror.org/0263zy895 cnr:institutes @=@ cnr:institutes @=@ __CDS133__ @=@ __CDS133__ - +base:normalized_types @=@ base:normalized_types @=@ Text @=@ Text +base:normalized_types @=@ base:normalized_types @=@ Book @=@ Book +base:normalized_types @=@ base:normalized_types @=@ Book part @=@ Book part +base:normalized_types @=@ base:normalized_types @=@ Journal/Newspaper @=@ Journal/Newspaper +base:normalized_types @=@ base:normalized_types @=@ Article contribution @=@ Article contribution +base:normalized_types @=@ base:normalized_types @=@ Other non-article @=@ Other non-article +base:normalized_types @=@ base:normalized_types @=@ Conference object @=@ Conference object +base:normalized_types @=@ base:normalized_types @=@ Report @=@ Report +base:normalized_types @=@ base:normalized_types @=@ Review @=@ Review +base:normalized_types @=@ base:normalized_types @=@ Course material @=@ Course material +base:normalized_types @=@ base:normalized_types @=@ Lecture @=@ Lecture +base:normalized_types @=@ base:normalized_types @=@ Thesis @=@ Thesis +base:normalized_types @=@ base:normalized_types @=@ Bachelor's thesis @=@ Bachelor's thesis +base:normalized_types @=@ base:normalized_types @=@ Master's thesis @=@ Master's thesis +base:normalized_types @=@ base:normalized_types @=@ Doctoral and postdoctoral thesis @=@ Doctoral and postdoctoral thesis +base:normalized_types @=@ base:normalized_types @=@ Manuscript @=@ Manuscript +base:normalized_types @=@ base:normalized_types @=@ Patent @=@ Patent +base:normalized_types @=@ base:normalized_types @=@ Musical notation @=@ Musical notation +base:normalized_types @=@ base:normalized_types @=@ Map @=@ Map +base:normalized_types @=@ base:normalized_types @=@ Audio @=@ Audio +base:normalized_types @=@ base:normalized_types @=@ Image/Video @=@ Image/Video +base:normalized_types @=@ base:normalized_types @=@ Still image @=@ Still image +base:normalized_types @=@ base:normalized_types @=@ Moving image/Video @=@ Moving image/Video +base:normalized_types @=@ base:normalized_types @=@ Software @=@ Software +base:normalized_types @=@ base:normalized_types @=@ Dataset @=@ Dataset +base:normalized_types @=@ base:normalized_types @=@ Unknown @=@ Unknown \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java index d5b106c81..35fa61b94 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java @@ -4,6 +4,7 @@ package eu.dnetlib.dhp.oa.dedup; import java.util.*; import java.util.stream.Stream; +import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils; import org.apache.commons.beanutils.BeanUtils; import org.apache.commons.lang3.StringUtils; import org.apache.spark.api.java.function.FlatMapFunction; @@ -14,6 +15,7 @@ import org.apache.spark.sql.*; import eu.dnetlib.dhp.oa.dedup.model.Identifier; import eu.dnetlib.dhp.oa.merge.AuthorMerger; import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils; import eu.dnetlib.dhp.schema.oaf.Author; import eu.dnetlib.dhp.schema.oaf.DataInfo; import eu.dnetlib.dhp.schema.oaf.OafEntity; @@ -23,190 +25,190 @@ import scala.Tuple3; import scala.collection.JavaConversions; public class DedupRecordFactory { - public static final class DedupRecordReduceState { - public final String dedupId; + public static final class DedupRecordReduceState { + public final String dedupId; - public final ArrayList aliases = new ArrayList<>(); + public final ArrayList aliases = new ArrayList<>(); - public final HashSet acceptanceDate = new HashSet<>(); + public final HashSet acceptanceDate = new HashSet<>(); - public OafEntity entity; + public OafEntity entity; - public DedupRecordReduceState(String dedupId, String id, OafEntity entity) { - this.dedupId = dedupId; - this.entity = entity; - if (entity == null) { - aliases.add(id); - } else { - if (Result.class.isAssignableFrom(entity.getClass())) { - Result result = (Result) entity; - if (result.getDateofacceptance() != null - && StringUtils.isNotBlank(result.getDateofacceptance().getValue())) { - acceptanceDate.add(result.getDateofacceptance().getValue()); - } - } - } - } + public DedupRecordReduceState(String dedupId, String id, OafEntity entity) { + this.dedupId = dedupId; + this.entity = entity; + if (entity == null) { + aliases.add(id); + } else { + if (Result.class.isAssignableFrom(entity.getClass())) { + Result result = (Result) entity; + if (result.getDateofacceptance() != null + && StringUtils.isNotBlank(result.getDateofacceptance().getValue())) { + acceptanceDate.add(result.getDateofacceptance().getValue()); + } + } + } + } - public String getDedupId() { - return dedupId; - } - } + public String getDedupId() { + return dedupId; + } + } - private static final int MAX_ACCEPTANCE_DATE = 20; + private static final int MAX_ACCEPTANCE_DATE = 20; - private DedupRecordFactory() { - } + private DedupRecordFactory() { + } - public static Dataset createDedupRecord( - final SparkSession spark, - final DataInfo dataInfo, - final String mergeRelsInputPath, - final String entitiesInputPath, - final Class clazz) { + public static Dataset createDedupRecord( + final SparkSession spark, + final DataInfo dataInfo, + final String mergeRelsInputPath, + final String entitiesInputPath, + final Class clazz) { - final long ts = System.currentTimeMillis(); - final Encoder beanEncoder = Encoders.bean(clazz); - final Encoder kryoEncoder = Encoders.kryo(clazz); + final long ts = System.currentTimeMillis(); + final Encoder beanEncoder = Encoders.bean(clazz); + final Encoder kryoEncoder = Encoders.kryo(clazz); - // - Dataset entities = spark - .read() - .schema(Encoders.bean(clazz).schema()) - .json(entitiesInputPath) - .as(beanEncoder) - .map( - (MapFunction>) entity -> { - return new Tuple2<>(entity.getId(), entity); - }, - Encoders.tuple(Encoders.STRING(), kryoEncoder)) - .selectExpr("_1 AS id", "_2 AS kryoObject"); + // + Dataset entities = spark + .read() + .schema(Encoders.bean(clazz).schema()) + .json(entitiesInputPath) + .as(beanEncoder) + .map( + (MapFunction>) entity -> { + return new Tuple2<>(entity.getId(), entity); + }, + Encoders.tuple(Encoders.STRING(), kryoEncoder)) + .selectExpr("_1 AS id", "_2 AS kryoObject"); - // : source is the dedup_id, target is the id of the mergedIn - Dataset mergeRels = spark - .read() - .load(mergeRelsInputPath) - .where("relClass == 'merges'") - .selectExpr("source as dedupId", "target as id"); + // : source is the dedup_id, target is the id of the mergedIn + Dataset mergeRels = spark + .read() + .load(mergeRelsInputPath) + .where("relClass == 'merges'") + .selectExpr("source as dedupId", "target as id"); - return mergeRels - .join(entities, JavaConversions.asScalaBuffer(Collections.singletonList("id")), "left") - .select("dedupId", "id", "kryoObject") - .as(Encoders.tuple(Encoders.STRING(), Encoders.STRING(), kryoEncoder)) - .map( - (MapFunction, DedupRecordReduceState>) t -> new DedupRecordReduceState( - t._1(), t._2(), t._3()), - Encoders.kryo(DedupRecordReduceState.class)) - .groupByKey( - (MapFunction) DedupRecordReduceState::getDedupId, Encoders.STRING()) - .reduceGroups( - (ReduceFunction) (t1, t2) -> { - if (t1.entity == null) { - t2.aliases.addAll(t1.aliases); - return t2; - } - if (t1.acceptanceDate.size() < MAX_ACCEPTANCE_DATE) { - t1.acceptanceDate.addAll(t2.acceptanceDate); - } - t1.aliases.addAll(t2.aliases); - t1.entity = reduceEntity(t1.entity, t2.entity); + return mergeRels + .join(entities, JavaConversions.asScalaBuffer(Collections.singletonList("id")), "left") + .select("dedupId", "id", "kryoObject") + .as(Encoders.tuple(Encoders.STRING(), Encoders.STRING(), kryoEncoder)) + .map( + (MapFunction, DedupRecordReduceState>) t -> new DedupRecordReduceState( + t._1(), t._2(), t._3()), + Encoders.kryo(DedupRecordReduceState.class)) + .groupByKey( + (MapFunction) DedupRecordReduceState::getDedupId, Encoders.STRING()) + .reduceGroups( + (ReduceFunction) (t1, t2) -> { + if (t1.entity == null) { + t2.aliases.addAll(t1.aliases); + return t2; + } + if (t1.acceptanceDate.size() < MAX_ACCEPTANCE_DATE) { + t1.acceptanceDate.addAll(t2.acceptanceDate); + } + t1.aliases.addAll(t2.aliases); + t1.entity = reduceEntity(t1.entity, t2.entity); - return t1; - }) - .flatMap((FlatMapFunction, OafEntity>) t -> { - String dedupId = t._1(); - DedupRecordReduceState agg = t._2(); + return t1; + }) + .flatMap((FlatMapFunction, OafEntity>) t -> { + String dedupId = t._1(); + DedupRecordReduceState agg = t._2(); - if (agg.acceptanceDate.size() >= MAX_ACCEPTANCE_DATE) { - return Collections.emptyIterator(); - } + if (agg.acceptanceDate.size() >= MAX_ACCEPTANCE_DATE) { + return Collections.emptyIterator(); + } - return Stream - .concat( - Stream - .of(agg.getDedupId()) - .map(id -> createDedupOafEntity(id, agg.entity, dataInfo, ts)), - agg.aliases - .stream() - .map(id -> createMergedDedupAliasOafEntity(id, agg.entity, dataInfo, ts))) - .iterator(); - }, beanEncoder); - } + return Stream + .concat( + Stream + .of(agg.getDedupId()) + .map(id -> createDedupOafEntity(id, agg.entity, dataInfo, ts)), + agg.aliases + .stream() + .map(id -> createMergedDedupAliasOafEntity(id, agg.entity, dataInfo, ts))) + .iterator(); + }, beanEncoder); + } - private static OafEntity createDedupOafEntity(String id, OafEntity base, DataInfo dataInfo, long ts) { - try { - OafEntity res = (OafEntity) BeanUtils.cloneBean(base); - res.setId(id); - res.setDataInfo(dataInfo); - res.setLastupdatetimestamp(ts); - return res; - } catch (Exception e) { - throw new RuntimeException(e); - } - } + private static OafEntity createDedupOafEntity(String id, OafEntity base, DataInfo dataInfo, long ts) { + try { + OafEntity res = (OafEntity) BeanUtils.cloneBean(base); + res.setId(id); + res.setDataInfo(dataInfo); + res.setLastupdatetimestamp(ts); + return res; + } catch (Exception e) { + throw new RuntimeException(e); + } + } - private static OafEntity createMergedDedupAliasOafEntity(String id, OafEntity base, DataInfo dataInfo, long ts) { - try { - OafEntity res = createDedupOafEntity(id, base, dataInfo, ts); - DataInfo ds = (DataInfo) BeanUtils.cloneBean(dataInfo); - ds.setDeletedbyinference(true); - res.setDataInfo(ds); - return res; - } catch (Exception e) { - throw new RuntimeException(e); - } - } + private static OafEntity createMergedDedupAliasOafEntity(String id, OafEntity base, DataInfo dataInfo, long ts) { + try { + OafEntity res = createDedupOafEntity(id, base, dataInfo, ts); + DataInfo ds = (DataInfo) BeanUtils.cloneBean(dataInfo); + ds.setDeletedbyinference(true); + res.setDataInfo(ds); + return res; + } catch (Exception e) { + throw new RuntimeException(e); + } + } - private static OafEntity reduceEntity(OafEntity entity, OafEntity duplicate) { + private static OafEntity reduceEntity(OafEntity entity, OafEntity duplicate) { - if (duplicate == null) { - return entity; - } + if (duplicate == null) { + return entity; + } - int compare = new IdentifierComparator<>() - .compare(Identifier.newInstance(entity), Identifier.newInstance(duplicate)); + int compare = new IdentifierComparator<>() + .compare(Identifier.newInstance(entity), Identifier.newInstance(duplicate)); - if (compare > 0) { - OafEntity swap = duplicate; - duplicate = entity; - entity = swap; - } + if (compare > 0) { + OafEntity swap = duplicate; + duplicate = entity; + entity = swap; + } - entity.mergeFrom(duplicate); + entity = MergeUtils.checkedMerge(entity, duplicate); - if (ModelSupport.isSubClass(duplicate, Result.class)) { - Result re = (Result) entity; - Result rd = (Result) duplicate; + if (ModelSupport.isSubClass(duplicate, Result.class)) { + Result re = (Result) entity; + Result rd = (Result) duplicate; - List> authors = new ArrayList<>(); - if (re.getAuthor() != null) { - authors.add(re.getAuthor()); - } - if (rd.getAuthor() != null) { - authors.add(rd.getAuthor()); - } + List> authors = new ArrayList<>(); + if (re.getAuthor() != null) { + authors.add(re.getAuthor()); + } + if (rd.getAuthor() != null) { + authors.add(rd.getAuthor()); + } - re.setAuthor(AuthorMerger.merge(authors)); - } + re.setAuthor(AuthorMerger.merge(authors)); + } - return entity; - } + return entity; + } - public static T entityMerger( - String id, Iterator> entities, long ts, DataInfo dataInfo, Class clazz) { - T base = entities.next()._2(); + public static T entityMerger( + String id, Iterator> entities, long ts, DataInfo dataInfo, Class clazz) { + T base = entities.next()._2(); - while (entities.hasNext()) { - T duplicate = entities.next()._2(); - if (duplicate != null) - base = (T) reduceEntity(base, duplicate); - } + while (entities.hasNext()) { + T duplicate = entities.next()._2(); + if (duplicate != null) + base = (T) reduceEntity(base, duplicate); + } - base.setId(id); - base.setDataInfo(dataInfo); - base.setLastupdatetimestamp(ts); + base.setId(id); + base.setDataInfo(dataInfo); + base.setLastupdatetimestamp(ts); - return base; - } + return base; + } } 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 cb1c70059..77e696501 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 @@ -3,6 +3,7 @@ package eu.dnetlib.dhp.oa.dedup; import static org.apache.spark.sql.functions.col; +import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.MapFunction; @@ -127,10 +128,8 @@ public class SparkPropagateRelation extends AbstractSparkAction { (MapFunction) r -> String .join(" ", r.getSource(), r.getTarget(), r.getRelType(), r.getSubRelType(), r.getRelClass()), Encoders.STRING()) - .reduceGroups((ReduceFunction) (b, a) -> { - b.mergeFrom(a); - return b; - }) + .reduceGroups((ReduceFunction) MergeUtils::mergeRelation + ) .map((MapFunction, Relation>) Tuple2::_2, REL_BEAN_ENC); final String outputRelationPath = graphOutputPath + "/relation"; 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 9ffaeeeef..42332b02e 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 @@ -3,17 +3,18 @@ package eu.dnetlib.doiboost import eu.dnetlib.dhp.application.ArgumentApplicationParser import eu.dnetlib.dhp.oa.merge.AuthorMerger import eu.dnetlib.dhp.schema.common.ModelConstants +import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils import eu.dnetlib.dhp.schema.oaf.{Organization, Publication, Relation, Dataset => OafDataset} import eu.dnetlib.doiboost.mag.ConversionUtil import org.apache.commons.io.IOUtils import org.apache.spark.SparkConf -import org.apache.spark.sql.expressions.Aggregator import org.apache.spark.sql.functions.col import org.apache.spark.sql._ import org.json4s.DefaultFormats import org.json4s.JsonAST.{JField, JObject, JString} import org.json4s.jackson.JsonMethods.parse import org.slf4j.{Logger, LoggerFactory} + import scala.collection.JavaConverters._ object SparkGenerateDoiBoost { @@ -78,8 +79,10 @@ object SparkGenerateDoiBoost { if (item._2 != null) { val otherPub = item._2._2 if (otherPub != null) { - crossrefPub.mergeFrom(otherPub) - crossrefPub.setAuthor(AuthorMerger.mergeAuthor(crossrefPub.getAuthor, otherPub.getAuthor)) + val mergedAuthor = AuthorMerger.mergeAuthor(crossrefPub.getAuthor, otherPub.getAuthor) + val res = MergeUtils.mergePublication(crossrefPub, otherPub) + res.setAuthor(mergedAuthor); + return res } } crossrefPub @@ -130,14 +133,13 @@ object SparkGenerateDoiBoost { // So we have to merge val b1 = left._2 val b2 = right._2 - b1.mergeFrom(b2) - b1.mergeOAFDataInfo(b2) val authors = AuthorMerger.mergeAuthor(b1.getAuthor, b2.getAuthor) - b1.setAuthor(authors) + val merged = MergeUtils.mergePublication(b1, b2); + merged.setAuthor(authors) if (b2.getId != null && b2.getId.nonEmpty) - b1.setId(b2.getId) + merged.setId(b2.getId) //Return publication Merged - (b1.getId, b1) + (merged.getId, merged) } } else { // Left is Null so we return right diff --git a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/mag/MagDataModel.scala b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/mag/MagDataModel.scala index 9a0b0d845..185381f8f 100644 --- a/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/mag/MagDataModel.scala +++ b/dhp-workflows/dhp-doiboost/src/main/scala/eu/dnetlib/doiboost/mag/MagDataModel.scala @@ -1,8 +1,8 @@ package eu.dnetlib.doiboost.mag import eu.dnetlib.dhp.schema.common.ModelConstants -import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory -import eu.dnetlib.dhp.schema.oaf.{Instance, Journal, Publication, StructuredProperty, Subject} +import eu.dnetlib.dhp.schema.oaf.utils.{IdentifierFactory, MergeUtils} +import eu.dnetlib.dhp.schema.oaf.{Instance, Journal, Publication, Subject} import eu.dnetlib.doiboost.DoiBoostMappingUtil import eu.dnetlib.doiboost.DoiBoostMappingUtil._ import org.json4s @@ -142,8 +142,7 @@ case object ConversionUtil { def mergePublication(a: Publication, b: Publication): Publication = { if ((a != null) && (b != null)) { - a.mergeFrom(b) - a + MergeUtils.mergePublication(a, b) } else { if (a == null) b else a } 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 4f755266a..f84a10d1c 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 @@ -149,7 +149,7 @@ public class SparkResultToCommunityFromOrganizationJob { } } // res.setContext(propagatedContexts); - // ret.mergeFrom(res); + // return MergeUtils.mergeResult(ret, res); } return ret; }; 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 f9c36d7ca..f791825a7 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 @@ -13,6 +13,7 @@ import java.util.List; import java.util.Optional; import java.util.stream.Collectors; +import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.MapFunction; @@ -24,8 +25,6 @@ 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; @@ -162,7 +161,7 @@ public class SparkResultToCommunityFromProject implements Serializable { } } res.setContext(propagatedContexts); - ret.mergeFrom(res); + return MergeUtils.checkedMerge(ret, res); } return ret; }; diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java index 3cf2f73c3..d2bfd75c1 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/resulttocommunityfromsemrel/SparkResultToCommunityThroughSemRelJob.java @@ -25,118 +25,118 @@ import scala.Tuple2; public class SparkResultToCommunityThroughSemRelJob { - private static final Logger log = LoggerFactory.getLogger(SparkResultToCommunityThroughSemRelJob.class); + private static final Logger log = LoggerFactory.getLogger(SparkResultToCommunityThroughSemRelJob.class); - public static void main(String[] args) throws Exception { + public static void main(String[] args) throws Exception { - String jsonConfiguration = IOUtils - .toString( - SparkResultToCommunityThroughSemRelJob.class - .getResourceAsStream( - "/eu/dnetlib/dhp/wf/subworkflows/resulttocommunityfromsemrel/input_communitytoresult_parameters.json")); + String jsonConfiguration = IOUtils + .toString( + SparkResultToCommunityThroughSemRelJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/wf/subworkflows/resulttocommunityfromsemrel/input_communitytoresult_parameters.json")); - final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); - parser.parseArgument(args); + parser.parseArgument(args); - Boolean isSparkSessionManaged = isSparkSessionManaged(parser); - log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); - String inputPath = parser.get("sourcePath"); - log.info("inputPath: {}", inputPath); + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); - final String outputPath = parser.get("outputPath"); - log.info("outputPath: {}", outputPath); + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); - final String preparedInfoPath = parser.get("preparedInfoPath"); - log.info("preparedInfoPath: {}", preparedInfoPath); + final String preparedInfoPath = parser.get("preparedInfoPath"); + log.info("preparedInfoPath: {}", preparedInfoPath); - SparkConf conf = new SparkConf(); - conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); + SparkConf conf = new SparkConf(); + conf.set("hive.metastore.uris", parser.get("hive_metastore_uris")); - final String resultClassName = parser.get("resultTableName"); - log.info("resultTableName: {}", resultClassName); + 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); + 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); + @SuppressWarnings("unchecked") + Class resultClazz = (Class) Class.forName(resultClassName); - runWithSparkHiveSession( - conf, - isSparkSessionManaged, - spark -> { - if (isTest(parser)) { - removeOutputDir(spark, outputPath); - } - if (saveGraph) { - execPropagation( - spark, inputPath, outputPath, preparedInfoPath, resultClazz); - } - }); - } + runWithSparkHiveSession( + conf, + isSparkSessionManaged, + spark -> { + if (isTest(parser)) { + removeOutputDir(spark, outputPath); + } + if (saveGraph) { + execPropagation( + spark, inputPath, outputPath, preparedInfoPath, resultClazz); + } + }); + } - private static void execPropagation( - SparkSession spark, - String inputPath, - String outputPath, - String preparedInfoPath, - Class resultClazz) { + private static void execPropagation( + SparkSession spark, + String inputPath, + String outputPath, + String preparedInfoPath, + Class resultClazz) { - Dataset possibleUpdates = readPath(spark, preparedInfoPath, ResultCommunityList.class); - Dataset result = readPath(spark, inputPath, resultClazz); + Dataset possibleUpdates = readPath(spark, preparedInfoPath, ResultCommunityList.class); + Dataset result = readPath(spark, inputPath, resultClazz); - result - .joinWith( - possibleUpdates, - result.col("id").equalTo(possibleUpdates.col("resultId")), - "left_outer") - .map(contextUpdaterFn(), Encoders.bean(resultClazz)) - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .json(outputPath); + result + .joinWith( + possibleUpdates, + result.col("id").equalTo(possibleUpdates.col("resultId")), + "left_outer") + .map(contextUpdaterFn(), Encoders.bean(resultClazz)) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath); - } + } - private static MapFunction, R> contextUpdaterFn() { - return value -> { - R ret = value._1(); - Optional rcl = Optional.ofNullable(value._2()); - if (rcl.isPresent()) { - Set contexts = new HashSet<>(); - ret.getContext().forEach(c -> contexts.add(c.getId())); - rcl - .get() - .getCommunityList() - .stream() - .forEach( - c -> { - if (!contexts.contains(c)) { - Context newContext = new Context(); - newContext.setId(c); - newContext - .setDataInfo( - Arrays - .asList( - getDataInfo( - PROPAGATION_DATA_INFO_TYPE, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, - PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, - ModelConstants.DNET_PROVENANCE_ACTIONS))); - ret.getContext().add(newContext); - } + private static MapFunction, R> contextUpdaterFn() { + return value -> { + R ret = value._1(); + Optional rcl = Optional.ofNullable(value._2()); + if (rcl.isPresent()) { + Set contexts = new HashSet<>(); + ret.getContext().forEach(c -> contexts.add(c.getId())); + rcl + .get() + .getCommunityList() + .stream() + .forEach( + c -> { + if (!contexts.contains(c)) { + Context newContext = new Context(); + newContext.setId(c); + newContext + .setDataInfo( + Arrays + .asList( + getDataInfo( + PROPAGATION_DATA_INFO_TYPE, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_ID, + PROPAGATION_RESULT_COMMUNITY_SEMREL_CLASS_NAME, + ModelConstants.DNET_PROVENANCE_ACTIONS))); + ret.getContext().add(newContext); + } - }); + }); - } + } - return ret; - }; - } + return ret; + }; + } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphTableSparkJob.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphTableSparkJob.java index d4f1d9f55..66a178c70 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphTableSparkJob.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/merge/MergeGraphTableSparkJob.java @@ -7,6 +7,7 @@ import java.util.*; import java.util.stream.Collectors; import java.util.stream.Stream; +import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.FilterFunction; @@ -24,7 +25,6 @@ import eu.dnetlib.dhp.common.HdfsSupport; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.*; -import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; import scala.Tuple2; /** @@ -248,11 +248,11 @@ public class MergeGraphTableSparkJob { private T mergeAndGet(T b, T a) { if (Objects.nonNull(a) && Objects.nonNull(b)) { if (ModelSupport.isSubClass(a, OafEntity.class) && ModelSupport.isSubClass(b, OafEntity.class)) { - return (T) OafMapperUtils.mergeEntities((OafEntity) b, (OafEntity) a); + return (T) MergeUtils.merge(b, a); } if (a instanceof Relation && b instanceof Relation) { ((Relation) a).mergeFrom(b); - return a; + return (T) MergeUtils.mergeRelation((Relation)a, (Relation) b); } } return Objects.isNull(a) ? b : a; diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java index 79ee5c5b6..9e1f3c78c 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/GenerateEntitiesApplication.java @@ -1,14 +1,16 @@ package eu.dnetlib.dhp.oa.graph.raw; -import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; - -import java.util.Arrays; -import java.util.List; -import java.util.Objects; -import java.util.Optional; -import java.util.stream.Collectors; - +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.graph.raw.common.AbstractMigrationApplication; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.io.Text; @@ -16,27 +18,18 @@ import org.apache.hadoop.io.compress.GzipCodec; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.rdd.RDD; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; -import org.dom4j.DocumentException; 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.common.vocabulary.VocabularyGroup; -import eu.dnetlib.dhp.oa.graph.raw.common.AbstractMigrationApplication; -import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.*; -import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; -import eu.dnetlib.dhp.utils.ISLookupClientFactory; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import scala.Tuple2; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + public class GenerateEntitiesApplication extends AbstractMigrationApplication { private static final Logger log = LoggerFactory.getLogger(GenerateEntitiesApplication.class); @@ -137,7 +130,7 @@ public class GenerateEntitiesApplication extends AbstractMigrationApplication { save( inputRdd .mapToPair(oaf -> new Tuple2<>(ModelSupport.idFn().apply(oaf), oaf)) - .reduceByKey(OafMapperUtils::merge) + .reduceByKey(MergeUtils::merge) .map(Tuple2::_2), targetPath); break; diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java index c69a7a6ff..da6885db3 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java @@ -317,7 +317,7 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i listKeyValues( createOpenaireId(10, rs.getString("collectedfromid"), true), rs.getString("collectedfromname"))); - p.setPid(new ArrayList<>()); + p.setPid(prepareListOfStructProps(rs.getArray("pid"), info)); p.setDateofcollection(asString(rs.getDate("dateofcollection"))); p.setDateoftransformation(asString(rs.getDate("dateoftransformation"))); p.setExtraInfo(new ArrayList<>()); // Values not present in the DB 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 3493ecb2f..72fc9e338 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 @@ -12,6 +12,16 @@ targetPath the output path of the graph enriched + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + spark 2.* extra listeners classname + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + spark 2.* sql query execution listeners classname + @@ -31,8 +41,8 @@ --executor-memory=${sparkExecutorMemory} --executor-cores=${sparkExecutorCores} --driver-memory=${sparkDriverMemory} - --conf spark.executor.memoryOverhead=2g - --conf spark.sql.shuffle.partitions=3000 + --conf spark.executor.memoryOverhead=${sparkExecutorMemory} + --conf spark.sql.shuffle.partitions=5000 --conf spark.extraListeners=${spark2ExtraListeners} --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryProjects.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryProjects.sql index af5913e75..fb584943f 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryProjects.sql +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryProjects.sql @@ -33,7 +33,7 @@ SELECT dc.officialname AS collectedfromname, p.contracttype || '@@@' || p.contracttypescheme AS contracttype, p.provenanceactionclass || '@@@' || p.provenanceactionscheme AS provenanceaction, - array_agg(DISTINCT i.pid || '###' || i.issuertype) AS pid, + array_remove(array_agg(DISTINCT i.pid || '###' || i.issuertype || '@@@' || i.issuertype), NULL) AS pid,, array_agg(DISTINCT s.name || '###' || s.semanticclass || '@@@' || s.semanticscheme) AS subjects, array_agg(DISTINCT fp.path) AS fundingtree diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryProjects_production.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryProjects_production.sql index 80cbda4b7..76a7fce23 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryProjects_production.sql +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryProjects_production.sql @@ -33,7 +33,7 @@ SELECT dc.officialname AS collectedfromname, p.contracttypeclass || '@@@' || p.contracttypescheme AS contracttype, p.provenanceactionclass || '@@@' || p.provenanceactionscheme AS provenanceaction, - array_agg(DISTINCT i.pid || '###' || i.issuertype) AS pid, + array_remove(array_agg(DISTINCT i.pid || '###' || i.issuertype || '@@@' || i.issuertype), NULL) AS pid, array_agg(DISTINCT s.name || '###' || s.semanticclass || '@@@' || s.semanticscheme) AS subjects, array_agg(DISTINCT fp.path) AS fundingtree FROM projects p 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 deleted file mode 100644 index 15513c8af..000000000 --- a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/AuthorEnricher.scala +++ /dev/null @@ -1,40 +0,0 @@ -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 - -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") - 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] = { - 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/ORCIDAuthorEnricher.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorEnricher.scala new file mode 100644 index 000000000..e2e7fada6 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorEnricher.scala @@ -0,0 +1,128 @@ +package eu.dnetlib.dhp.enrich.orcid + +import eu.dnetlib.dhp.schema.common.ModelConstants +import eu.dnetlib.dhp.schema.oaf.{Author, StructuredProperty} +import eu.dnetlib.dhp.schema.sx.OafUtils + +import java.util +import scala.beans.BeanProperty +import scala.collection.JavaConverters._ +import scala.util.control.Breaks.{break, breakable} + +case class ORCIDAuthorEnricherResult( + @BeanProperty var id: String, + @BeanProperty var enriched_author: java.util.List[Author], + @BeanProperty var author_matched: java.util.List[MatchedAuthors], + @BeanProperty var author_unmatched: java.util.List[Author], + @BeanProperty var orcid_unmatched: java.util.List[OrcidAutor] +) + +object ORCIDAuthorEnricher extends Serializable { + + def enrichOrcid( + id: String, + graph_authors: java.util.List[Author], + orcid_authors: java.util.List[OrcidAutor] + ): ORCIDAuthorEnricherResult = { + // Author enriching strategy: + // 1) create a copy of graph author list in unmatched_authors + // 2) find best match in unmatched_authors, remove it from unmatched_authors and enrich it so + // that the enrichment is reflected in graph_authors (they share author instances) + // 3) repeat (2) till the end of the list and then with different matching algorithms that have decreasing + // trust in their output + // At the end unmatched_authors will contain authors not matched with any of the matching algos + val unmatched_authors = new util.ArrayList[Author](graph_authors) + + val matches = { + // Look after exact fullname match, reconstruct ORCID fullname as givenName + familyName + extractAndEnrichMatches( + unmatched_authors, + orcid_authors, + (author, orcid) => + ORCIDAuthorMatchers.matchEqualsIgnoreCase(author.getFullname, orcid.givenName + " " + orcid.familyName), + "fullName" + ) ++ + // Look after exact reversed fullname match, reconstruct ORCID fullname as familyName + givenName + extractAndEnrichMatches( + unmatched_authors, + orcid_authors, + (author, orcid) => + ORCIDAuthorMatchers.matchEqualsIgnoreCase(author.getFullname, orcid.familyName + " " + orcid.givenName), + "reversedFullName" + ) ++ + // split author names in tokens, order the tokens, then check for matches of full tokens or abbreviations + extractAndEnrichMatches( + unmatched_authors, + orcid_authors, + (author, orcid) => + ORCIDAuthorMatchers + .matchOrderedTokenAndAbbreviations(author.getFullname, orcid.givenName + " " + orcid.familyName), + "orderedTokens" + ) ++ + // look after exact matches of ORCID creditName + extractAndEnrichMatches( + unmatched_authors, + orcid_authors, + (author, orcid) => ORCIDAuthorMatchers.matchEqualsIgnoreCase(author.getFullname, orcid.creditName), + "creditName" + ) ++ + // look after exact matches in ORCID otherNames + extractAndEnrichMatches( + unmatched_authors, + orcid_authors, + (author, orcid) => + orcid.otherNames != null && ORCIDAuthorMatchers.matchOtherNames(author.getFullname, orcid.otherNames.asScala), + "otherNames" + ) + } + + ORCIDAuthorEnricherResult(id, graph_authors, matches.asJava, unmatched_authors, orcid_authors) + } + + private def extractAndEnrichMatches( + graph_authors: java.util.List[Author], + orcid_authors: java.util.List[OrcidAutor], + matchingFunc: (Author, OrcidAutor) => Boolean, + matchName: String + ) = { + val matched = scala.collection.mutable.ArrayBuffer.empty[MatchedAuthors] + + if (graph_authors != null && !graph_authors.isEmpty) { + val ait = graph_authors.iterator + + while (ait.hasNext) { + val author = ait.next() + val oit = orcid_authors.iterator + + breakable { + while (oit.hasNext) { + val orcid = oit.next() + + if (matchingFunc(author, orcid)) { + ait.remove() + oit.remove() + matched += MatchedAuthors(author, orcid, matchName) + + if (author.getPid == null) { + author.setPid(new util.ArrayList[StructuredProperty]()) + } + + val orcidPID = OafUtils.createSP(orcid.orcid, ModelConstants.ORCID, ModelConstants.ORCID) + orcidPID.setDataInfo(OafUtils.generateDataInfo()) + orcidPID.getDataInfo.setProvenanceaction( + OafUtils.createQualifier("ORCID_ENRICHMENT", "ORCID_ENRICHMENT") + ) + + author.getPid.add(orcidPID) + + break() + } + } + } + } + } + + matched + } + +} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchers.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchers.scala new file mode 100644 index 000000000..49574fe2d --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchers.scala @@ -0,0 +1,65 @@ +package eu.dnetlib.dhp.enrich.orcid + +import java.util.Locale +import java.util.regex.Pattern + +object ORCIDAuthorMatchers { + val SPLIT_REGEX = Pattern.compile("[\\s,\\.]+") + + val WORD_DIFF = 2 + + def matchEqualsIgnoreCase(a1: String, a2: String): Boolean = { + if (a1 == null || a2 == null) + false + else + a1 == a2 || a1.toLowerCase(Locale.ROOT).equals(a2.toLowerCase(Locale.ROOT)) + } + + def matchOtherNames(fullName: String, otherNames: Seq[String]): Boolean = { + if (otherNames != null) { + otherNames.exists(matchEqualsIgnoreCase(fullName, _)) + } else { + false + } + } + + def matchOrderedTokenAndAbbreviations(a1: String, a2: String): Boolean = { + val p1: Array[String] = SPLIT_REGEX.split(a1.trim.toLowerCase(Locale.ROOT)).filter(_.nonEmpty).sorted + val p2: Array[String] = SPLIT_REGEX.split(a2.trim.toLowerCase(Locale.ROOT)).filter(_.nonEmpty).sorted + + if (p1.length < 2 || p2.length < 2) return false + if (Math.abs(p1.length - p2.length) > WORD_DIFF) return false // use alternative comparison algo + + var p1Idx: Int = 0 + var p2Idx: Int = 0 + var shortMatches: Int = 0 + var longMatches: Int = 0 + while (p1Idx < p1.length && p2Idx < p2.length) { + val e1: String = p1(p1Idx) + val c1: Char = e1.charAt(0) + val e2: String = p2(p2Idx) + val c2: Char = e2.charAt(0) + if (c1 < c2) p1Idx += 1 + else if (c1 > c2) p2Idx += 1 + else { + var res: Boolean = false + if (e1.length != 1 && e2.length != 1) { + res = e1 == e2 + longMatches += 1 + } else { + res = true + shortMatches += 1 + } + if (res) { + p1Idx += 1 + p2Idx += 1 + } else { + val diff: Int = e1.compareTo(e2) + if (diff < 0) p1Idx += 1 + else if (diff > 0) p2Idx += 1 + } + } + } + longMatches > 0 && (shortMatches + longMatches) == Math.min(p1.length, p2.length) + } +} 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 4822059c6..0824c2a71 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,14 +1,39 @@ 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 eu.dnetlib.dhp.schema.oaf._ import org.apache.spark.sql._ +import org.apache.spark.sql.functions._ import org.slf4j.{Logger, LoggerFactory} + +import scala.beans.BeanProperty import scala.collection.JavaConverters._ +case class OrcidAutor( + @BeanProperty var orcid: String, + @BeanProperty var familyName: String, + @BeanProperty var givenName: String, + @BeanProperty var creditName: String, + @BeanProperty var otherNames: java.util.List[String] +) { + def this() = this("null", "null", "null", "null", null) +} + +case class MatchData( + @BeanProperty var id: String, + @BeanProperty var graph_authors: java.util.List[Author], + @BeanProperty var orcid_authors: java.util.List[OrcidAutor] +) { + def this() = this("null", null, null) +} + +case class MatchedAuthors( + @BeanProperty var author: Author, + @BeanProperty var orcid: OrcidAutor, + @BeanProperty var `type`: String +) + class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String], log: Logger) extends AbstractScalaApplication(propertyPath, args, log: Logger) { @@ -22,107 +47,132 @@ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String] log.info(s"orcidPath is '$orcidPath'") 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", - 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]) - ) + createTemporaryData(graphPath, orcidPath, targetPath) + analisys(targetPath) + generateGraph(graphPath, targetPath) } - private def enrichResult[T <: Result]( - spark: SparkSession, - graphPath: String, - orcidPublication: Dataset[Row], - outputPath: String, - enc: Encoder[T] - ): Unit = { + private def generateGraph(graphPath: String, targetPath: String): Unit = { - val entities = spark.read - .schema(enc.schema) - .json(graphPath) - .select(col("id"), col("datainfo"), col("instance")) - .where("datainfo.deletedbyinference != true") - .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") - ) + ModelSupport.entityTypes.asScala + .filter(e => ModelSupport.isResult(e._1)) + .foreach(e => { + val resultType = e._1.name() + val enc = Encoders.bean(e._2) - 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() - orcidDnet.count() - val result = spark.read.schema(enc.schema).json(graphPath).as[T](enc) + val matched = spark.read + .schema(Encoders.bean(classOf[ORCIDAuthorEnricherResult]).schema) + .parquet(s"${targetPath}/${resultType}_matched") + .selectExpr("id", "enriched_author") + + spark.read + .schema(enc.schema) + .json(s"$graphPath/$resultType") + .join(matched, Seq("id"), "left") + .withColumn( + "author", + when(size(col("enriched_author")).gt(0), col("enriched_author")) + .otherwise(col("author")) + ) + .drop("enriched_author") + .write + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(s"${targetPath}/${resultType}") + + }) - result - .joinWith(orcidDnet, result("id").equalTo(orcidDnet("dnet_id")), "left") - .map { - 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) } - private def generateOrcidTable(spark: SparkSession, inputPath: String): Dataset[Row] = { + private def createTemporaryData(graphPath: String, orcidPath: String, targetPath: String): Unit = { val orcidAuthors = - spark.read.load(s"$inputPath/Authors").select("orcid", "familyName", "givenName", "creditName", "otherNames") + spark.read.load(s"$orcidPath/Authors").select("orcid", "familyName", "givenName", "creditName", "otherNames") + val orcidWorks = spark.read - .load(s"$inputPath/Works") + .load(s"$orcidPath/Works") .select(col("orcid"), explode(col("pids")).alias("identifier")) .where( - "identifier.schema IN('doi','pmid','pmc','arxiv','handle')" + "identifier.schema IN('doi','pmid','pmc','arxiv','handle')" // scopus eid ? ) - val orcidPublication = orcidAuthors - .join(orcidWorks, orcidAuthors("orcid").equalTo(orcidWorks("orcid"))) + + val orcidWorksWithAuthors = orcidAuthors + .join(orcidWorks, Seq("orcid")) .select( - col("identifier.schema").alias("schema"), - col("identifier.value").alias("value"), - struct(orcidAuthors("orcid").alias("orcid"), col("givenName"), col("familyName")).alias("author") + lower(col("identifier.schema")).alias("pid_schema"), + lower(col("identifier.value")).alias("pid_value"), + struct( + col("orcid"), + col("givenName"), + col("familyName"), + col("creditName"), + col("otherNames") + ).alias("author") ) - orcidPublication.cache() + .cache() + + ModelSupport.entityTypes.asScala + .filter(e => ModelSupport.isResult(e._1)) + .foreach(e => { + val resultType = e._1.name() + val enc = Encoders.bean(e._2) + + val oaEntities = spark.read + .schema(enc.schema) + .json(s"$graphPath/$resultType") + .select(col("id"), col("datainfo"), col("instance")) + .where("datainfo.deletedbyinference != true") + .drop("datainfo") + .withColumn("instances", explode(col("instance"))) + .withColumn("pids", explode(col("instances.pid"))) + .select( + lower(col("pids.qualifier.classid")).alias("pid_schema"), + lower(col("pids.value")).alias("pid_value"), + col("id") + ) + + val orcidDnet = orcidWorksWithAuthors + .join( + oaEntities, + Seq("pid_schema", "pid_value"), + "inner" + ) + .groupBy(col("id")) + .agg(collect_set(col("author")).alias("orcid_authors")) + .select("id", "orcid_authors") + + val result = + spark.read.schema(enc.schema).json(s"$graphPath/$resultType").selectExpr("id", "author as graph_authors") + + result + .join(orcidDnet, Seq("id")) + .write + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .parquet(s"$targetPath/${resultType}_unmatched") + }) + + orcidWorksWithAuthors.unpersist() + } + + private def analisys(targetPath: String): Unit = { + ModelSupport.entityTypes.asScala + .filter(e => ModelSupport.isResult(e._1)) + .foreach(e => { + val resultType = e._1.name() + + spark.read + .parquet(s"$targetPath/${resultType}_unmatched") + .where("size(graph_authors) > 0") + .as[MatchData](Encoders.bean(classOf[MatchData])) + .map(md => { + ORCIDAuthorEnricher.enrichOrcid(md.id, md.graph_authors, md.orcid_authors) + })(Encoders.bean(classOf[ORCIDAuthorEnricherResult])) + .write + .option("compression", "gzip") + .mode("overwrite") + .parquet(s"$targetPath/${resultType}_matched") + }) } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/oa/graph/raw/CopyHdfsOafSparkApplication.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/oa/graph/raw/CopyHdfsOafSparkApplication.scala index 533948289..9d7cca7dd 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/oa/graph/raw/CopyHdfsOafSparkApplication.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/oa/graph/raw/CopyHdfsOafSparkApplication.scala @@ -93,8 +93,8 @@ object CopyHdfsOafSparkApplication { hasSource != null && hasTarget != null } else { val hasId = (json \ "id").extractOrElse[String](null) - val resultType = (json \ "resulttype" \ "classid").extractOrElse[String](null) - hasId != null && oafType.equalsIgnoreCase(resultType) + val resultType = (json \ "resulttype" \ "classid").extractOrElse[String]("") + hasId != null && oafType.startsWith(resultType) } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/oa/graph/resolution/SparkResolveEntities.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/oa/graph/resolution/SparkResolveEntities.scala index f5a13e72b..0c9f0d7ba 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/oa/graph/resolution/SparkResolveEntities.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/oa/graph/resolution/SparkResolveEntities.scala @@ -3,6 +3,7 @@ package eu.dnetlib.dhp.oa.graph.resolution import com.fasterxml.jackson.databind.ObjectMapper import eu.dnetlib.dhp.application.ArgumentApplicationParser import eu.dnetlib.dhp.schema.common.EntityType +import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils import eu.dnetlib.dhp.schema.oaf.{Dataset => OafDataset, _} import org.apache.commons.io.IOUtils import org.apache.hadoop.fs.{FileSystem, Path} @@ -118,15 +119,12 @@ object SparkResolveEntities { currentEntityDataset .joinWith(re, currentEntityDataset("_1").equalTo(re("_1")), "left") .map(k => { - val a = k._1 val b = k._2 if (b == null) a._2 - else { - a._2.mergeFrom(b._2) - a._2 - } + else + MergeUtils.mergeResult(a._2, b._2) }) .map(r => mapper.writeValueAsString(r))(Encoders.STRING) .write diff --git a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkCreateInputGraph.scala b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkCreateInputGraph.scala index 704c9ab5c..ecf612cc7 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkCreateInputGraph.scala +++ b/dhp-workflows/dhp-graph-mapper/src/main/scala/eu/dnetlib/dhp/sx/graph/SparkCreateInputGraph.scala @@ -1,6 +1,7 @@ package eu.dnetlib.dhp.sx.graph import eu.dnetlib.dhp.application.ArgumentApplicationParser +import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils import eu.dnetlib.dhp.schema.oaf.{Dataset => OafDataset, _} import org.apache.commons.io.IOUtils import org.apache.spark.SparkConf @@ -130,10 +131,7 @@ object SparkCreateInputGraph { val ds: Dataset[T] = spark.read.load(sourcePath).as[T] ds.groupByKey(_.getId) - .reduceGroups { (x: T, y: T) => - x.mergeFrom(y) - x - } + .reduceGroups { (x: T, y: T) => MergeUtils.merge(x, y).asInstanceOf[T] } .map(_._2) .write .mode(SaveMode.Overwrite) diff --git a/dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorEnricherTest.java similarity index 71% rename from dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java rename to dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorEnricherTest.java index c0a8d6927..0b50312a5 100644 --- a/dhp-common/src/test/java/eu/dnetlib/oa/merge/AuthorMergerTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorEnricherTest.java @@ -1,10 +1,9 @@ -package eu.dnetlib.oa.merge; - -import static org.junit.jupiter.api.Assertions.*; +package eu.dnetlib.dhp.enrich.orcid; import java.io.BufferedReader; import java.io.InputStreamReader; +import java.util.Collections; import java.util.List; import java.util.Objects; @@ -14,10 +13,9 @@ 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 { +public class ORCIDAuthorEnricherTest { @Test public void testEnrcichAuthor() throws Exception { @@ -26,12 +24,13 @@ public class AuthorMergerTest { BufferedReader pr = new BufferedReader(new InputStreamReader( Objects .requireNonNull( - AuthorMergerTest.class - .getResourceAsStream("/eu/dnetlib/dhp/oa/merge/authors_publication_sample.json")))); + ORCIDAuthorEnricherTest.class + .getResourceAsStream("/eu/dnetlib/dhp/enrich/orcid/authors_publication_sample.json")))); BufferedReader or = new BufferedReader(new InputStreamReader( Objects .requireNonNull( - AuthorMergerTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/merge/authors_orcid_sample.json")))); + ORCIDAuthorEnricherTest.class + .getResourceAsStream("/eu/dnetlib/dhp/enrich/orcid/authors_orcid_sample.json")))); TypeReference> aclass = new TypeReference>() { }; @@ -67,7 +66,8 @@ public class AuthorMergerTest { long start = System.currentTimeMillis(); // final List enrichedList = AuthorMerger.enrichOrcid(publicationAuthors, orcidAuthors); - final List enrichedList = AuthorMerger.enrichOrcid(publicationAuthors, orcidAuthors); + final List enrichedList = Collections.emptyList(); // SparkEnrichGraphWithOrcidAuthors.enrichOrcid(publicationAuthors, + // orcidAuthors); long enrichedAuthorWithPid = enrichedList .stream() @@ -91,24 +91,4 @@ public class AuthorMergerTest { } } - @Test - public void checkSimilarityTest() { - final Author left = new Author(); - left.setName("Anand"); - left.setSurname("Rachna"); - left.setFullname("Anand, Rachna"); - - System.out.println(AuthorMerger.normalizeFullName(left.getFullname())); - - final Author right = new Author(); - right.setName("Rachna"); - right.setSurname("Anand"); - right.setFullname("Rachna, Anand"); -// System.out.println(AuthorMerger.normalize(right.getFullname())); - boolean same = AuthorMerger.checkORCIDSimilarity(left, right); - - assertTrue(same); - - } - } diff --git a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/CopyHdfsOafSparkApplicationTest.java b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/CopyHdfsOafSparkApplicationTest.java index 85cb551bc..3c6fc9c7f 100644 --- a/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/CopyHdfsOafSparkApplicationTest.java +++ b/dhp-workflows/dhp-graph-mapper/src/test/java/eu/dnetlib/dhp/oa/graph/raw/CopyHdfsOafSparkApplicationTest.java @@ -59,7 +59,19 @@ public class CopyHdfsOafSparkApplicationTest { .getResourceAsStream( "/eu/dnetlib/dhp/oa/graph/raw/publication_2_unknownProperty.json")), "publication")); + } + @Test + void isOafType_Datacite_ORP() throws IOException { + assertTrue( + CopyHdfsOafSparkApplication + .isOafType( + IOUtils + .toString( + getClass() + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/graph/raw/datacite_orp.json")), + "otherresearchproduct")); } } 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 6d6b2ffbd..bf275814c 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 @@ -1,13 +1,12 @@ package eu.dnetlib.dhp.oa.graph.raw; -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.util.List; - +import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.utils.MergeUtils; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import org.apache.commons.io.IOUtils; import org.dom4j.DocumentException; import org.junit.jupiter.api.BeforeEach; @@ -16,13 +15,12 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; -import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup; -import eu.dnetlib.dhp.oa.graph.clean.GraphCleaningFunctionsTest; -import eu.dnetlib.dhp.schema.common.ModelConstants; -import eu.dnetlib.dhp.schema.oaf.*; -import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import java.io.IOException; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.lenient; @ExtendWith(MockitoExtension.class) class GenerateEntitiesApplicationTest { @@ -72,7 +70,7 @@ class GenerateEntitiesApplicationTest { protected void verifyMerge(Result publication, Result dataset, Class clazz, String resultType) { - final Result merge = OafMapperUtils.mergeResults(publication, dataset); + final Result merge = MergeUtils.mergeResult(publication, dataset); assertTrue(clazz.isAssignableFrom(merge.getClass())); assertEquals(resultType, merge.getResulttype().getClassid()); } diff --git a/dhp-common/src/test/resources/eu/dnetlib/dhp/oa/merge/authors_orcid_sample.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/enrich/orcid/authors_orcid_sample.json similarity index 100% rename from dhp-common/src/test/resources/eu/dnetlib/dhp/oa/merge/authors_orcid_sample.json rename to dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/enrich/orcid/authors_orcid_sample.json diff --git a/dhp-common/src/test/resources/eu/dnetlib/dhp/oa/merge/authors_publication_sample.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/enrich/orcid/authors_publication_sample.json similarity index 100% rename from dhp-common/src/test/resources/eu/dnetlib/dhp/oa/merge/authors_publication_sample.json rename to dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/enrich/orcid/authors_publication_sample.json diff --git a/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/datacite_orp.json b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/datacite_orp.json new file mode 100644 index 000000000..abf44b49c --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/test/resources/eu/dnetlib/dhp/oa/graph/raw/datacite_orp.json @@ -0,0 +1 @@ +{"collectedfrom":[{"key":"10|openaire____::9e3be59865b2c1c335d32dae2fe7b254","value":"Datacite","dataInfo":null}],"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:actionset","classname":"sysimport:actionset","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":null,"id":"50|doi_________::0a2763977bb76876aff2d3b33a874552","originalId":["50|datacite____::0a2763977bb76876aff2d3b33a874552","10.25935/nhb2-wy29"],"pid":[{"value":"10.25935/nhb2-wy29","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:actionset","classname":"sysimport:actionset","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"dateofcollection":"2024-01-15T18:00:35+0000","dateoftransformation":"2024-01-15T18:00:35+0000","extraInfo":null,"oaiprovenance":null,"measures":null,"processingchargeamount":null,"processingchargecurrency":null,"author":[{"fullname":"Louis, Corentin","name":"Corentin","surname":"Louis","rank":1,"pid":[{"value":"https://orcid.org/0000-0002-9552-8822","qualifier":{"classid":"orcid","classname":"Open Researcher and Contributor ID","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:actionset","classname":"sysimport:actionset","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"affiliation":[{"value":"Dublin Institute For Advanced Studies","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:actionset","classname":"sysimport:actionset","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]},{"fullname":"Zarka, Philippe","name":"Philippe","surname":"Zarka","rank":2,"pid":[{"value":"https://orcid.org/0000-0003-1672-9878","qualifier":{"classid":"orcid","classname":"Open Researcher and Contributor ID","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:actionset","classname":"sysimport:actionset","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"affiliation":[{"value":"Laboratory of Space Studies and Instrumentation in Astrophysics","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:actionset","classname":"sysimport:actionset","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]},{"fullname":"Cecconi, Baptiste","name":"Baptiste","surname":"Cecconi","rank":3,"pid":[{"value":"https://orcid.org/0000-0001-7915-5571","qualifier":{"classid":"orcid","classname":"Open Researcher and Contributor ID","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:actionset","classname":"sysimport:actionset","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"affiliation":[{"value":"Laboratory of Space Studies and Instrumentation in Astrophysics","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:actionset","classname":"sysimport:actionset","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]},{"fullname":"Kurth, William","name":"William","surname":"Kurth","rank":4,"pid":[{"value":"https://orcid.org/0000-0002-5471-6202","qualifier":{"classid":"orcid","classname":"Open Researcher and Contributor ID","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:actionset","classname":"sysimport:actionset","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"affiliation":[{"value":"University of Iowa","dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:actionset","classname":"sysimport:actionset","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}]}],"resulttype":{"classid":"other","classname":"other","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"metaResourceType":null,"language":null,"country":null,"subject":[],"title":[{"value":"Catalogue of Jupiter radio emissions identified in the Juno/Waves observations","qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"dataInfo":null}],"relevantdate":[{"value":"2021-01-01","qualifier":{"classid":"issued","classname":"issued","schemeid":"dnet:dataCite_date","schemename":"dnet:dataCite_date"},"dataInfo":null}],"description":[],"dateofacceptance":{"value":"2021-01-01","dataInfo":null},"publisher":{"value":"PADC/MASER","dataInfo":null},"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":[{"license":{"value":"https://creativecommons.org/licenses/by/4.0/legalcode","dataInfo":null},"accessright":{"classid":"UNKNOWN","classname":"not available","schemeid":"dnet:access_modes","schemename":"dnet:access_modes","openAccessRoute":null},"instancetype":{"classid":"0022","classname":"Collection","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"instanceTypeMapping":[{"originalType":"Collection","typeCode":null,"typeLabel":null,"vocabularyName":"openaire::coar_resource_types_3_1"}],"hostedby":{"key":"10|openaire____::55045bd2a65019fd8e6741a755395c8c","value":"Unknown Repository","dataInfo":null},"url":["https://dx.doi.org/10.25935/nhb2-wy29"],"distributionlocation":null,"collectedfrom":{"key":"10|openaire____::9e3be59865b2c1c335d32dae2fe7b254","value":"Datacite","dataInfo":null},"pid":[{"value":"10.25935/nhb2-wy29","qualifier":{"classid":"doi","classname":"doi","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"dataInfo":{"invisible":false,"inferred":false,"deletedbyinference":false,"trust":"0.9","inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:actionset","classname":"sysimport:actionset","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"alternateIdentifier":null,"dateofacceptance":{"value":"2021-01-01","dataInfo":null},"processingchargeamount":null,"processingchargecurrency":null,"refereed":null,"measures":null,"fulltext":null}],"eoscifguidelines":null,"openAccessColor":null,"publiclyFunded":null,"contactperson":null,"contactgroup":null,"tool":null,"isGreen":null,"isInDiamondJournal":null} \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchersTest.scala b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchersTest.scala new file mode 100644 index 000000000..f109ebe24 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/test/scala/eu/dnetlib/dhp/enrich/orcid/ORCIDAuthorMatchersTest.scala @@ -0,0 +1,35 @@ +package eu.dnetlib.dhp.enrich.orcid + +import eu.dnetlib.dhp.enrich.orcid.ORCIDAuthorMatchers.matchOrderedTokenAndAbbreviations +import org.junit.jupiter.api.Assertions.{assertFalse, assertTrue} +import org.junit.jupiter.api.Test + +class ORCIDAuthorMatchersTest { + + @Test def testShortNames(): Unit = { + assertTrue(matchOrderedTokenAndAbbreviations("Lasagni Mariozzi Federico", "Lasagni F. Mariozzi")) + } + + @Test def testInvertedNames(): Unit = { + assertTrue(matchOrderedTokenAndAbbreviations("Andrea, Paolo Marcello", "Marcello Paolo, Andrea")) + } + + @Test def testHomonymy(): Unit = { + assertTrue(matchOrderedTokenAndAbbreviations("Jang Myung Lee", "J Lee")) + } + + @Test def testAmbiguousShortNames(): Unit = { + assertFalse(matchOrderedTokenAndAbbreviations("P. Mariozzi", "M. Paolozzi")) + } + + @Test def testNonMatches(): Unit = { + assertFalse(matchOrderedTokenAndAbbreviations("Giovanni Paolozzi", "Francesco Paolozzi")) + assertFalse(matchOrderedTokenAndAbbreviations("G. Paolozzi", "F. Paolozzi")) + } + + @Test def testChineseNames(): Unit = { + assertTrue(matchOrderedTokenAndAbbreviations("孙林 Sun Lin", "Sun Lin")) + // assertTrue(AuthorsMatchRevised.compare("孙林 Sun Lin", "孙林")); // not yet implemented + } + +}