diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/PropagationConstant.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/PropagationConstant.java index 89bdf0982..673dbf7af 100644 --- a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/PropagationConstant.java +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/PropagationConstant.java @@ -12,14 +12,13 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.common.HdfsSupport; import eu.dnetlib.dhp.schema.common.ModelConstants; -import eu.dnetlib.dhp.schema.oaf.Country; -import eu.dnetlib.dhp.schema.oaf.DataInfo; -import eu.dnetlib.dhp.schema.oaf.Qualifier; -import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; public class PropagationConstant { @@ -50,6 +49,9 @@ public class PropagationConstant { public static final String TRUE = "true"; + public static final String PROPAGATION_SUBJECT_RESULT_SEMREL_CLASS_ID = "subject:remrel"; + public static final String PROPAGATION_SUBJECT_RESULT_SEMREL_CLASS_NAME = "Propagation of subjects through semantic relation"; + public static final String PROPAGATION_COUNTRY_INSTREPO_CLASS_ID = "country:instrepos"; public static final String PROPAGATION_COUNTRY_INSTREPO_CLASS_NAME = "Propagation of country to result collected from datasources of type institutional repositories"; @@ -239,4 +241,30 @@ public class PropagationConstant { } } + public static Dataset readOafKryoPath( + SparkSession spark, String inputPath, Class clazz) { + return spark + .read() + .textFile(inputPath) + .map((MapFunction) value -> OBJECT_MAPPER.readValue(value, clazz), Encoders.kryo(clazz)); + } + + public static Class[] getModelClasses() { + List> modelClasses = Lists.newArrayList(ModelSupport.getOafModelClasses()); + modelClasses + .addAll( + Lists + .newArrayList( + Result.class, + Qualifier.class, + DataInfo.class, + Publication.class, + eu.dnetlib.dhp.schema.oaf.Dataset.class, + Software.class, + OtherResearchProduct.class, + Subject.class, + AccessRight.class)); + return modelClasses.toArray(new Class[] {}); + } + } diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/subjecttoresultfromsemrel/PrepareResultResultStep1.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/subjecttoresultfromsemrel/PrepareResultResultStep1.java new file mode 100644 index 000000000..f35ad52e1 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/subjecttoresultfromsemrel/PrepareResultResultStep1.java @@ -0,0 +1,162 @@ + +package eu.dnetlib.dhp.subjecttoresultfromsemrel; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +import java.io.Serializable; +import java.util.*; +import java.util.stream.Collectors; + +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.FilterFunction; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.MapGroupsFunction; +import org.apache.spark.sql.*; +import org.jetbrains.annotations.NotNull; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.Result; +import eu.dnetlib.dhp.schema.oaf.Subject; +import scala.Tuple2; + +/** + * @author miriam.baglioni + * @Date 04/10/22 + * + * This is for the selection of result with subject in subjetClassList + */ + +public class PrepareResultResultStep1 implements Serializable { + private static final Logger log = LoggerFactory.getLogger(PrepareResultResultStep1.class); + + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + PrepareResultResultStep1.class + .getResourceAsStream( + "/eu/dnetlib/dhp/subjectpropagation/input_preparesubjecttoresult_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); + + Class resultClazz = (Class) Class.forName(resultClassName); + + final String resultType = parser.get("resultType"); + log.info("resultType: {}", resultType); + + final List subjectClassList = Arrays + .asList( + parser.get("subjectlist").split(";")) + .stream() + .map(s -> s.toLowerCase()) + .collect(Collectors.toList()); + log.info("subjectClassList: {}", subjectClassList); + + final List allowedSemRel = Arrays + .asList( + parser.get("allowedsemrels").split(";")) + .stream() + .map(s -> s.toLowerCase()) + .collect(Collectors.toList()); + log.info("allowedSemRel: {}", allowedSemRel); + + SparkConf conf = new SparkConf(); + + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + prepareInfo(spark, inputPath, outputPath, subjectClassList, allowedSemRel, resultClazz, resultType); + }); + } + + private static void prepareInfo(SparkSession spark, + String inputPath, + String outputPath, + List subjectClassList, + List allowedSemRel, + Class resultClazz, + String resultType) { + + Dataset result = readPath(spark, inputPath + "/" + resultType, resultClazz) + .filter( + (FilterFunction) r -> !r.getDataInfo().getDeletedbyinference() && + !r.getDataInfo().getInvisible() && Optional.ofNullable(r.getSubject()).isPresent() && + r + .getSubject() + .stream() + .anyMatch(s -> subjectClassList.contains(s.getQualifier().getClassid().toLowerCase()))); + + Dataset relation = readPath(spark, inputPath + "/relation", Relation.class) + .filter( + (FilterFunction) r -> !r.getDataInfo().getDeletedbyinference() && + allowedSemRel.contains(r.getRelClass().toLowerCase())); + + result + .joinWith(relation, result.col("id").equalTo(relation.col("source")), "right") + .groupByKey((MapFunction, String>) t2 -> t2._2().getTarget(), Encoders.STRING()) + .mapGroups( + (MapGroupsFunction, ResultSubjectList>) (k, + it) -> getResultSubjectList(subjectClassList, k, it), + Encoders.bean(ResultSubjectList.class)) + .filter(Objects::nonNull) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath + "/" + resultType); + } + + private static ResultSubjectList getResultSubjectList(List subjectClassList, String k, + Iterator> it) { + Tuple2 first = it.next(); + if (!Optional.ofNullable(first._1()).isPresent()) { + return null; + } + ResultSubjectList rsl = new ResultSubjectList(); + rsl.setResId(k); + List sbjInfo = new ArrayList<>(); + Set subjectSet = new HashSet<>(); + extracted(subjectClassList, first._1().getSubject(), sbjInfo, subjectSet); + it.forEachRemaining(t2 -> { + if (Optional.ofNullable(t2._1()).isPresent()) + extracted(subjectClassList, t2._1().getSubject(), sbjInfo, subjectSet); + }); + rsl.setSubjectList(sbjInfo); + return rsl; + } + + private static void extracted(List subjectClassList, List resultSubject, + List sbjList, Set subjectSet) { + + resultSubject + .stream() + .filter(s -> subjectClassList.contains(s.getQualifier().getClassid().toLowerCase())) + .forEach(s -> { + if (!subjectSet.contains(s.getValue())) + sbjList + .add( + SubjectInfo + .newInstance( + s.getQualifier().getClassid(), s.getQualifier().getClassname(), s.getValue())); + subjectSet.add(s.getValue()); + }); + } + +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/subjecttoresultfromsemrel/ResultSubjectList.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/subjecttoresultfromsemrel/ResultSubjectList.java new file mode 100644 index 000000000..b7c61e15e --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/subjecttoresultfromsemrel/ResultSubjectList.java @@ -0,0 +1,34 @@ + +package eu.dnetlib.dhp.subjecttoresultfromsemrel; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import eu.dnetlib.dhp.schema.oaf.Subject; + +/** + * @author miriam.baglioni + * @Date 04/10/22 + */ +public class ResultSubjectList implements Serializable { + private String resId; + List subjectList; + + public String getResId() { + return resId; + } + + public void setResId(String resId) { + this.resId = resId; + } + + public List getSubjectList() { + return subjectList; + } + + public void setSubjectList(List subjectList) { + this.subjectList = subjectList; + } +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/subjecttoresultfromsemrel/SparkSubjectPropagationStep2.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/subjecttoresultfromsemrel/SparkSubjectPropagationStep2.java new file mode 100644 index 000000000..2a3bcff51 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/subjecttoresultfromsemrel/SparkSubjectPropagationStep2.java @@ -0,0 +1,192 @@ + +package eu.dnetlib.dhp.subjecttoresultfromsemrel; + +import static eu.dnetlib.dhp.PropagationConstant.*; +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +import java.io.Serializable; +import java.util.*; +import java.util.stream.Collectors; + +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.Result; +import eu.dnetlib.dhp.schema.oaf.Subject; +import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; +import scala.Tuple2; + +/** + * @author miriam.baglioni + * @Date 05/10/22 + */ +public class SparkSubjectPropagationStep2 implements Serializable { + private static final Logger log = LoggerFactory.getLogger(SparkSubjectPropagationStep2.class); + + public static void main(String[] args) throws Exception { + String jsonConfiguration = IOUtils + .toString( + SparkSubjectPropagationStep2.class + .getResourceAsStream( + "/eu/dnetlib/dhp/subjectpropagation/input_propagatesubject_parameters.json")); + + final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration); + parser.parseArgument(args); + + Boolean isSparkSessionManaged = isSparkSessionManaged(parser); + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + String preparedPath = parser.get("preparedPath"); + log.info("preparedPath: {}", preparedPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath: {}", outputPath); + + final String resultClassName = parser.get("resultTableName"); + log.info("resultTableName: {}", resultClassName); + + Class resultClazz = (Class) Class.forName(resultClassName); + + final String resultType = parser.get("resultType"); + log.info("resultType: {}", resultType); + + final String inputPath = parser.get("sourcePath"); + log.info("inputPath: {}", inputPath); + + final String workingPath = parser.get("workingPath"); + log.info("workingPath: {}", workingPath); + + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(getModelClasses()); + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> { + removeOutputDir(spark, outputPath); + execPropagation(spark, inputPath, outputPath, workingPath, preparedPath, resultClazz, resultType); + }); + } + + private static void execPropagation(SparkSession spark, + String inputPath, + String outputPath, + String workingPath, + String preparedPath, + Class resultClazz, + String resultType) { + + Dataset> results = readOafKryoPath(spark, inputPath + "/" + resultType, resultClazz) + .map( + (MapFunction>) r -> new Tuple2(r.getId(), r), + Encoders.tuple(Encoders.STRING(), Encoders.kryo(resultClazz))); + + Dataset preparedResult = readPath( + spark, preparedPath + "/publication", ResultSubjectList.class) + .union(readPath(spark, preparedPath + "/dataset", ResultSubjectList.class)) + .union(readPath(spark, preparedPath + "/software", ResultSubjectList.class)) + .union(readPath(spark, preparedPath + "/otherresearchproduct", ResultSubjectList.class)); + + results + .joinWith( + preparedResult, + results.col("_1").equalTo(preparedResult.col("resId")), + "left") + .map((MapFunction, ResultSubjectList>, String>) t2 -> { + R res = t2._1()._2(); + // estraggo le tipologie di subject dal result + Map> resultMap = new HashMap<>(); + if (Optional.ofNullable(t2._2()).isPresent()) { + if(Optional.ofNullable(res.getSubject()).isPresent()){ + res.getSubject().stream().forEach(s -> { + String cid = s.getQualifier().getClassid(); + if(!cid.equals(ModelConstants.DNET_SUBJECT_KEYWORD)){ + if (!resultMap.containsKey(cid)) { + resultMap.put(cid, new ArrayList<>()); + } + resultMap.get(cid).add(s.getValue()); + } + }); + }else{ + res.setSubject(new ArrayList<>()); + } + + // Remove from the list all the subjects with the same class already present in the result + List distinctClassId = t2 + ._2() + .getSubjectList() + .stream() + .map(si -> si.getClassid()) + .distinct() + .collect(Collectors.toList()); + List sbjInfo = new ArrayList<>(); + for (String k : distinctClassId) { + if (!resultMap.containsKey(k)) + sbjInfo = t2 + ._2() + .getSubjectList() + .stream() + .filter(s -> s.getClassid().equalsIgnoreCase(k)) + .collect(Collectors.toList()); + else + sbjInfo = t2 + ._2() + .getSubjectList() + .stream() + .filter( + s -> s.getClassid().equalsIgnoreCase(k) && + !resultMap.get(k).contains(s.getValue())) + .collect(Collectors.toList()); + // All the subjects not already present in the result are added + for (SubjectInfo si : sbjInfo) { + res.getSubject().add(getSubject(si)); + } + + } + + } + return OBJECT_MAPPER.writeValueAsString(res); + }, Encoders.STRING()) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .text(workingPath + "/" + resultType); + + readPath(spark, workingPath + "/" + resultType, resultClazz) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath + "/" + resultType); + + } + + private static Subject getSubject(SubjectInfo si) { + return OafMapperUtils + .subject( + si.getValue(), + si.getClassid(), si.getClassname(), + ModelConstants.DNET_SUBJECT_TYPOLOGIES, ModelConstants.DNET_SUBJECT_TYPOLOGIES, + OafMapperUtils + .dataInfo( + false, PROPAGATION_DATA_INFO_TYPE, + true, false, + OafMapperUtils + .qualifier( + PROPAGATION_SUBJECT_RESULT_SEMREL_CLASS_ID, + PROPAGATION_SUBJECT_RESULT_SEMREL_CLASS_NAME, + ModelConstants.DNET_PROVENANCE_ACTIONS, + ModelConstants.DNET_PROVENANCE_ACTIONS), + "0.85")); + + } + +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/subjecttoresultfromsemrel/SubjectInfo.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/subjecttoresultfromsemrel/SubjectInfo.java new file mode 100644 index 000000000..ace40a6d4 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/subjecttoresultfromsemrel/SubjectInfo.java @@ -0,0 +1,46 @@ + +package eu.dnetlib.dhp.subjecttoresultfromsemrel; + +import java.io.Serializable; + +/** + * @author miriam.baglioni + * @Date 06/10/22 + */ +public class SubjectInfo implements Serializable { + private String classid; + private String value; + private String classname; + + public static SubjectInfo newInstance(String classid, String classname, String value) { + SubjectInfo si = new SubjectInfo(); + si.classid = classid; + si.value = value; + si.classname = classname; + return si; + } + + public String getClassid() { + return classid; + } + + public void setClassid(String classid) { + this.classid = classid; + } + + public String getValue() { + return value; + } + + public void setValue(String value) { + this.value = value; + } + + public String getClassname() { + return classname; + } + + public void setClassname(String classname) { + this.classname = classname; + } +} diff --git a/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/subjecttoresultfromsemrel/Utils.java b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/subjecttoresultfromsemrel/Utils.java new file mode 100644 index 000000000..6879fb547 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/java/eu/dnetlib/dhp/subjecttoresultfromsemrel/Utils.java @@ -0,0 +1,17 @@ + +package eu.dnetlib.dhp.subjecttoresultfromsemrel; + +import java.io.Serializable; +import java.util.*; +import java.util.stream.Stream; + +import eu.dnetlib.dhp.schema.oaf.Result; +import eu.dnetlib.dhp.schema.oaf.Subject; + +/** + * @author miriam.baglioni + * @Date 05/10/22 + */ +public class Utils implements Serializable { + +} diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/subjectpropagation/input_preparesubjecttoresult_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/subjectpropagation/input_preparesubjecttoresult_parameters.json new file mode 100644 index 000000000..1e3ac1af4 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/subjectpropagation/input_preparesubjecttoresult_parameters.json @@ -0,0 +1,46 @@ +[ + + + { + "paramName":"asr", + "paramLongName":"allowedsemrels", + "paramDescription": "the set of semantic relations between the results to be exploited to perform the propagation", + "paramRequired": true + }, + { + "paramName":"sl", + "paramLongName":"subjectlist", + "paramDescription": "the list of classid for the subject we wanti to propagate", + "paramRequired": true + }, + { + "paramName":"rt", + "paramLongName":"resultType", + "paramDescription": "the result type", + "paramRequired": true + }, + { + "paramName":"sp", + "paramLongName":"sourcePath", + "paramDescription": "the path of the input graph", + "paramRequired": true + }, + { + "paramName":"rtn", + "paramLongName":"resultTableName", + "paramDescription": "the class of the result", + "paramRequired": true + }, + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "the path where prepared info have been stored", + "paramRequired": false + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/subjectpropagation/input_propagatesubject_parameters.json b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/subjectpropagation/input_propagatesubject_parameters.json new file mode 100644 index 000000000..76942cbe6 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/subjectpropagation/input_propagatesubject_parameters.json @@ -0,0 +1,45 @@ +[ + + { + "paramName":"pp", + "paramLongName":"preparedPath", + "paramDescription": "the path to the prepared information", + "paramRequired": true + }, + { + "paramName":"rt", + "paramLongName":"resultType", + "paramDescription": "the result type", + "paramRequired": true + }, + { + "paramName":"sp", + "paramLongName":"sourcePath", + "paramDescription": "the path of the input graph", + "paramRequired": true + }, + { + "paramName":"rtn", + "paramLongName":"resultTableName", + "paramDescription": "the class of the result", + "paramRequired": true + }, + { + "paramName": "ssm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "the path where prepared info have been stored", + "paramRequired": false + }, + { + "paramName": "out", + "paramLongName": "outputPath", + "paramDescription": "the path used to store output files", + "paramRequired": true + }, + { + "paramName": "wp", + "paramLongName": "workingPath", + "paramDescription": "the path used to store temporary output files", + "paramRequired": true +} +] \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/subjectpropagation/oozie_app/config-default.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/subjectpropagation/oozie_app/config-default.xml new file mode 100644 index 000000000..0ce8cef58 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/subjectpropagation/oozie_app/config-default.xml @@ -0,0 +1,63 @@ + + + jobTracker + yarnRM + + + + nameNode + + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + + hive_metastore_uris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + + + spark2YarnHistoryServerAddress + http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089 + + + spark2EventLogDir + /user/spark/spark2ApplicationHistory + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + + + sparkExecutorNumber + 4 + + + sparkDriverMemory + 15G + + + sparkExecutorMemory + 10G + + + sparkExecutorCores + 1 + + + spark2MaxExecutors + 50 + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/subjectpropagation/oozie_app/workflow.xml b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/subjectpropagation/oozie_app/workflow.xml new file mode 100644 index 000000000..02c015f97 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/main/resources/eu/dnetlib/dhp/subjectpropagation/oozie_app/workflow.xml @@ -0,0 +1,297 @@ + + + + sourcePath + the source path + + + subjectlist + fos;sdg + the list of subject classid to propagate (split by ;) + + + allowedsemrels + IsSupplementedBy;IsSupplementTo;IsPreviousVersionOf;IsNewVersionOf;IsIdenticalTo;Obsoletes;IsObsoletedBy;IsVersionOf + the allowed semantics + + + outputPath + the output path + + + + + ${jobTracker} + ${nameNode} + + + oozie.action.sharelib.for.spark + ${oozieActionShareLibForSpark2} + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + + + + + + + + + + + + yarn + cluster + PrepareSubjectResultsAssociation + eu.dnetlib.dhp.subjecttoresultfromsemrel.PrepareResultResultStep1 + dhp-enrichment-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath} + --allowedsemrels${allowedsemrels} + --subjectlist${subjectlist} + --resultTypepublication + --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication + --outputPath${workingDir}/preparedInfo + + + + + + + + yarn + cluster + PrepareSubjectResultsAssociation + eu.dnetlib.dhp.subjecttoresultfromsemrel.PrepareResultResultStep1 + dhp-enrichment-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath} + --allowedsemrels${allowedsemrels} + --subjectlist${subjectlist} + --resultTypedataset + --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset + --outputPath${workingDir}/preparedInfo + + + + + + + + yarn + cluster + PrepareSubjectResultsAssociation + eu.dnetlib.dhp.subjecttoresultfromsemrel.PrepareResultResultStep1 + dhp-enrichment-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath} + --allowedsemrels${allowedsemrels} + --subjectlist${subjectlist} + --resultTypesoftware + --resultTableNameeu.dnetlib.dhp.schema.oaf.Software + --outputPath${workingDir}/preparedInfo + + + + + + + + yarn + cluster + PrepareSubjectResultsAssociation + eu.dnetlib.dhp.subjecttoresultfromsemrel.PrepareResultResultStep1 + dhp-enrichment-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePath${sourcePath} + --allowedsemrels${allowedsemrels} + --subjectlist${subjectlist} + --resultTypeotherresearchproduct + --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct + --outputPath${workingDir}/preparedInfo + + + + + + + + + + + + + + + + + yarn + cluster + SubjectToResultPropagation + eu.dnetlib.dhp.subjecttoresultfromsemrel.SparkSubjectPropagationStep2 + dhp-enrichment-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=8G + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --conf spark.sql.shuffle.partitions=3840 + + --sourcePath${sourcePath} + --outputPath${outputPath} + --workingPath${workingDir}/working + --resultTableNameeu.dnetlib.dhp.schema.oaf.Publication + --resultTypepublication + --preparedPath${workingDir}/preparedInfo + + + + + + + + yarn + cluster + SubjectToResultPropagation + eu.dnetlib.dhp.subjecttoresultfromsemrel.SparkSubjectPropagationStep2 + dhp-enrichment-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --conf spark.sql.shuffle.partitions=3840 + + --sourcePath${sourcePath} + --outputPath${outputPath} + --workingPath${workingDir}/working + --resultTableNameeu.dnetlib.dhp.schema.oaf.OtherResearchProduct + --resultTypeotherresearchproduct + --preparedPath${workingDir}/preparedInfo + + + + + + + + yarn + cluster + SubjectToResultPropagation + eu.dnetlib.dhp.subjecttoresultfromsemrel.SparkSubjectPropagationStep2 + dhp-enrichment-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --conf spark.sql.shuffle.partitions=3840 + + --sourcePath${sourcePath} + --outputPath${outputPath} + --workingPath${workingDir}/working + --resultTableNameeu.dnetlib.dhp.schema.oaf.Dataset + --resultTypedataset + --preparedPath${workingDir}/preparedInfo + + + + + + + + yarn + cluster + SubjectToResultPropagation + eu.dnetlib.dhp.subjecttoresultfromsemrel.SparkSubjectPropagationStep2 + dhp-enrichment-${projectVersion}.jar + + --executor-cores=${sparkExecutorCores} + --executor-memory=${sparkExecutorMemory} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.dynamicAllocation.enabled=true + --conf spark.dynamicAllocation.maxExecutors=${spark2MaxExecutors} + --conf spark.sql.shuffle.partitions=3840 + + --sourcePath${sourcePath} + --outputPath${outputPath} + --workingPath${workingDir}/working + --resultTableNameeu.dnetlib.dhp.schema.oaf.Software + --resultTypesoftware + --preparedPath${workingDir}/preparedInfo + + + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/subjectpropagation/SubjectPreparationJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/subjectpropagation/SubjectPreparationJobTest.java new file mode 100644 index 000000000..0b3b45d7e --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/subjectpropagation/SubjectPreparationJobTest.java @@ -0,0 +1,159 @@ + +package eu.dnetlib.dhp.subjectpropagation; + +import static org.apache.spark.sql.functions.desc; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import org.apache.commons.io.FileUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.resulttocommunityfromsemrel.ResultToCommunityJobTest; +import eu.dnetlib.dhp.resulttocommunityfromsemrel.SparkResultToCommunityThroughSemRelJob; +import eu.dnetlib.dhp.schema.oaf.Dataset; +import eu.dnetlib.dhp.schema.oaf.Publication; +import eu.dnetlib.dhp.schema.oaf.Subject; +import eu.dnetlib.dhp.subjecttoresultfromsemrel.PrepareResultResultStep1; +import eu.dnetlib.dhp.subjecttoresultfromsemrel.ResultSubjectList; +import eu.dnetlib.dhp.subjecttoresultfromsemrel.SubjectInfo; + +/** + * @author miriam.baglioni + * @Date 05/10/22 + */ +public class SubjectPreparationJobTest { + private static final Logger log = LoggerFactory.getLogger(SubjectPreparationJobTest.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static SparkSession spark; + + private static Path workingDir; + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(SubjectPreparationJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(SubjectPreparationJobTest.class.getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + + spark = SparkSession + .builder() + .appName(SubjectPreparationJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + void testSparkSubjectToResultThroughSemRelJob() throws Exception { + PrepareResultResultStep1 + .main( + new String[] { + "-allowedsemrels", + "IsSupplementedBy;IsSupplementTo;IsPreviousVersionOf;IsNewVersionOf;IsIdenticalTo;Obsoletes;IsObsoletedBy;IsVersionOf", + "-subjectlist", "fos;sdg", + "-resultType", "publication", + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", getClass() + .getResource("/eu/dnetlib/dhp/subjectpropagation") + .getPath(), + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Publication", + "-outputPath", workingDir.toString() + + }); + + // 50|06cdd3ff4700::93859bd27121c3ee7c6ee4bfb1790cba fake_fos and fake_sdg IsVersionOf + // 50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98 + // 50|06cdd3ff4700::ff21e3c55d527fa7db171137c5fd1f1f fake_fos2 obsoletes + // 50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98 + // 50|355e65625b88::046477dc24819c5f1453166aa7bfb75e fake_fos2 isSupplementedBy + // 50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98 + // 50|355e65625b88::046477dc24819c5f1453166aa7bfb75e fake_fos2 issupplementto + // 50|06cdd3ff4700::93859bd27121c3ee7c6ee4bfb1790cba + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/publication") + .map(item -> OBJECT_MAPPER.readValue(item, ResultSubjectList.class)); + + Assertions.assertEquals(2, tmp.count()); + + Assertions + .assertEquals( + 1, tmp.filter(r -> r.getResId().equals("50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98")).count()); + Assertions + .assertEquals( + 1, tmp.filter(r -> r.getResId().equals("50|06cdd3ff4700::93859bd27121c3ee7c6ee4bfb1790cba")).count()); + + List sbjList = tmp + .filter(r -> r.getResId().equals("50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98")) + .first() + .getSubjectList(); + + Assertions.assertEquals(3, sbjList.size()); + Assertions.assertEquals(1, sbjList.stream().filter(s -> s.getClassid().equals("sdg")).count()); + Assertions.assertEquals(2, sbjList.stream().filter(s -> s.getClassid().equals("fos")).count()); + + Assertions + .assertEquals( + "fake_sdg", + sbjList.stream().filter(s -> s.getClassid().equalsIgnoreCase("sdg")).findFirst().get().getValue()); + Assertions + .assertTrue( + sbjList + .stream() + .filter(s -> s.getClassid().equalsIgnoreCase("fos")) + .anyMatch(s -> s.getValue().equals("fake_fos"))); + Assertions + .assertTrue( + sbjList + .stream() + .filter(s -> s.getClassid().equalsIgnoreCase("fos")) + .anyMatch(s -> s.getValue().equals("fake_fos2"))); + + sbjList = tmp + .filter(r -> r.getResId().equals("50|06cdd3ff4700::93859bd27121c3ee7c6ee4bfb1790cba")) + .first() + .getSubjectList(); + + Assertions.assertEquals(1, sbjList.size()); + Assertions.assertEquals("fos", sbjList.get(0).getClassid().toLowerCase()); + + Assertions.assertEquals("fake_fos2", sbjList.get(0).getValue()); + + tmp.foreach(s -> System.out.println(OBJECT_MAPPER.writeValueAsString(s))); + } + +} diff --git a/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/subjectpropagation/SubjectPropagationJobTest.java b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/subjectpropagation/SubjectPropagationJobTest.java new file mode 100644 index 000000000..48c425bbc --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/java/eu/dnetlib/dhp/subjectpropagation/SubjectPropagationJobTest.java @@ -0,0 +1,333 @@ + +package eu.dnetlib.dhp.subjectpropagation; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.List; + +import org.apache.commons.io.FileUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.PropagationConstant; +import eu.dnetlib.dhp.schema.oaf.Publication; +import eu.dnetlib.dhp.schema.oaf.Subject; +import eu.dnetlib.dhp.subjecttoresultfromsemrel.SparkSubjectPropagationStep2; + +/** + * @author miriam.baglioni + * @Date 06/10/22 + */ +public class SubjectPropagationJobTest { + private static final Logger log = LoggerFactory.getLogger(SubjectPropagationJobTest.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static SparkSession spark; + + private static Path workingDir; + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(SubjectPropagationJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(SubjectPropagationJobTest.class.getSimpleName()); + + conf.setMaster("local[*]"); + conf.set("spark.driver.host", "localhost"); + conf.set("hive.metastore.local", "true"); + conf.set("spark.ui.enabled", "false"); + conf.set("spark.sql.warehouse.dir", workingDir.toString()); + conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString()); + + spark = SparkSession + .builder() + .appName(SubjectPropagationJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + void testSparkSubjectToResultThroughSemRelJob() throws Exception { + SparkSubjectPropagationStep2 + .main( + new String[] { + "-preparedPath", getClass() + .getResource("/eu/dnetlib/dhp/subjectpropagation/preparedInfo") + .getPath(), + "-resultType", "publication", + "-sourcePath", getClass() + .getResource("/eu/dnetlib/dhp/subjectpropagation") + .getPath(), + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-resultTableName", "eu.dnetlib.dhp.schema.oaf.Publication", + "-workingPath", workingDir.toString() + "/working", + "-outputPath", workingDir.toString() + }); + + // 50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98 should receive fake_fos, fake_sdg and fake_fos2 + // 50|06cdd3ff4700::93859bd27121c3ee7c6ee4bfb1790cba should receive fake_fos2 + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/publication") + .map(item -> OBJECT_MAPPER.readValue(item, Publication.class)); + + Assertions.assertEquals(4, tmp.count()); + + Assertions + .assertEquals( + 2, tmp + .filter( + r -> r + .getSubject() + .stream() + .anyMatch( + s -> s + .getDataInfo() + .getInferenceprovenance() + .equals(PropagationConstant.PROPAGATION_DATA_INFO_TYPE))) + .count()); + + JavaRDD sbjs = tmp + .flatMap((FlatMapFunction) r -> r.getSubject().iterator()) + .filter( + s -> s.getDataInfo().getInferenceprovenance().equals(PropagationConstant.PROPAGATION_DATA_INFO_TYPE)); + + Assertions.assertEquals(4, sbjs.count()); + Assertions + .assertEquals( + 4, sbjs + .filter( + s -> s + .getDataInfo() + .getProvenanceaction() + .getClassid() + .equals(PropagationConstant.PROPAGATION_SUBJECT_RESULT_SEMREL_CLASS_ID)) + .count()); + Assertions + .assertEquals( + 4, + sbjs + .filter( + s -> s + .getDataInfo() + .getProvenanceaction() + .getClassname() + .equals(PropagationConstant.PROPAGATION_SUBJECT_RESULT_SEMREL_CLASS_NAME)) + .count()); + Assertions.assertEquals(3, sbjs.filter(s -> s.getQualifier().getClassid().equals("FOS")).count()); + Assertions + .assertEquals(3, sbjs.filter(s -> s.getQualifier().getClassname().equals("Field of Science")).count()); + Assertions.assertEquals(1, sbjs.filter(s -> s.getQualifier().getClassid().equals("SDG")).count()); + Assertions + .assertEquals( + 1, sbjs.filter(s -> s.getQualifier().getClassname().equals("Support and Development Goals")).count()); + + Assertions + .assertEquals( + 6, + tmp + .filter(r -> r.getId().equals("50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98")) + .first() + .getSubject() + .size()); + Assertions + .assertEquals( + 3, tmp + .filter( + r -> r + .getId() + .equals("50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98")) + .first() + .getSubject() + .stream() + .filter( + s -> s + .getDataInfo() + .getInferenceprovenance() + .equals(PropagationConstant.PROPAGATION_DATA_INFO_TYPE)) + .count()); + Assertions + .assertEquals( + 3, tmp + .filter( + r -> r + .getId() + .equals("50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98")) + .first() + .getSubject() + .stream() + .filter( + s -> !s + .getDataInfo() + .getInferenceprovenance() + .equals(PropagationConstant.PROPAGATION_DATA_INFO_TYPE)) + .count()); + Assertions + .assertEquals( + 2, tmp + .filter( + r -> r + .getId() + .equals("50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98")) + .first() + .getSubject() + .stream() + .filter( + s -> s + .getDataInfo() + .getInferenceprovenance() + .equals(PropagationConstant.PROPAGATION_DATA_INFO_TYPE) && + s.getQualifier().getClassid().equals("FOS")) + .count()); + Assertions + .assertEquals( + 1, tmp + .filter( + r -> r + .getId() + .equals("50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98")) + .first() + .getSubject() + .stream() + .filter( + s -> s + .getDataInfo() + .getInferenceprovenance() + .equals(PropagationConstant.PROPAGATION_DATA_INFO_TYPE) && + s.getQualifier().getClassid().equals("SDG")) + .count()); + + Assertions + .assertTrue( + tmp + .filter(r -> r.getId().equals("50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98")) + .first() + .getSubject() + .stream() + .anyMatch(s -> s.getValue().equals("fake_fos"))); + Assertions + .assertTrue( + tmp + .filter(r -> r.getId().equals("50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98")) + .first() + .getSubject() + .stream() + .anyMatch(s -> s.getValue().equals("fake_fos2"))); + Assertions + .assertTrue( + tmp + .filter(r -> r.getId().equals("50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98")) + .first() + .getSubject() + .stream() + .anyMatch(s -> s.getValue().equals("fake_sdg"))); + + Assertions + .assertEquals( + 6, + tmp + .filter(r -> r.getId().equals("50|06cdd3ff4700::93859bd27121c3ee7c6ee4bfb1790cba")) + .first() + .getSubject() + .size()); + Assertions + .assertEquals( + 1, tmp + .filter( + r -> r + .getId() + .equals("50|06cdd3ff4700::93859bd27121c3ee7c6ee4bfb1790cba")) + .first() + .getSubject() + .stream() + .filter( + s -> s + .getDataInfo() + .getInferenceprovenance() + .equals(PropagationConstant.PROPAGATION_DATA_INFO_TYPE)) + .count()); + Assertions + .assertEquals( + 5, tmp + .filter( + r -> r + .getId() + .equals("50|06cdd3ff4700::93859bd27121c3ee7c6ee4bfb1790cba")) + .first() + .getSubject() + .stream() + .filter( + s -> !s + .getDataInfo() + .getInferenceprovenance() + .equals(PropagationConstant.PROPAGATION_DATA_INFO_TYPE)) + .count()); + Assertions + .assertEquals( + 1, tmp + .filter( + r -> r + .getId() + .equals("50|06cdd3ff4700::93859bd27121c3ee7c6ee4bfb1790cba")) + .first() + .getSubject() + .stream() + .filter( + s -> s + .getDataInfo() + .getInferenceprovenance() + .equals(PropagationConstant.PROPAGATION_DATA_INFO_TYPE) && + s.getQualifier().getClassid().equals("FOS")) + .count()); + Assertions + .assertEquals( + 0, tmp + .filter( + r -> r + .getId() + .equals("50|06cdd3ff4700::93859bd27121c3ee7c6ee4bfb1790cba")) + .first() + .getSubject() + .stream() + .filter( + s -> s + .getDataInfo() + .getInferenceprovenance() + .equals(PropagationConstant.PROPAGATION_DATA_INFO_TYPE) && + s.getQualifier().getClassid().equals("SDG")) + .count()); + + Assertions + .assertTrue( + tmp + .filter(r -> r.getId().equals("50|06cdd3ff4700::93859bd27121c3ee7c6ee4bfb1790cba")) + .first() + .getSubject() + .stream() + .anyMatch(s -> s.getValue().equals("fake_fos2"))); + + } +} diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/subjectpropagation/preparedInfo/dataset/dataset.json b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/subjectpropagation/preparedInfo/dataset/dataset.json new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/subjectpropagation/preparedInfo/otherresearchproduct/orp.json b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/subjectpropagation/preparedInfo/otherresearchproduct/orp.json new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/subjectpropagation/preparedInfo/publication/preparedInfo b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/subjectpropagation/preparedInfo/publication/preparedInfo new file mode 100644 index 000000000..abbea5506 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/subjectpropagation/preparedInfo/publication/preparedInfo @@ -0,0 +1,2 @@ +{"resId":"50|06cdd3ff4700::93859bd27121c3ee7c6ee4bfb1790cba","subjectList":[{"classid":"FOS", "classname":"Field of Science","value":"fake_fos2"}]} +{"resId":"50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98","subjectList":[{"classid":"FOS", "classname":"Field of Science","value":"fake_fos"},{"classid":"SDG","classname":"Support and Development Goals","value":"fake_sdg"},{"classid":"FOS", "classname":"Field of Science","value":"fake_fos2"}]} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/subjectpropagation/preparedInfo/software/software.json b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/subjectpropagation/preparedInfo/software/software.json new file mode 100644 index 000000000..e69de29bb diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/subjectpropagation/publication/publication.json b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/subjectpropagation/publication/publication.json new file mode 100644 index 000000000..da5a26580 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/subjectpropagation/publication/publication.json @@ -0,0 +1,4 @@ +{"author":[{"fullname":"Levande, Paul","name":"Paul","pid":[],"rank":1,"surname":"Levande"}],"bestaccessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::6824b298c96ba906a3e6a70593affbf5","value":"Episciences"}],"context":[],"contributor":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Coordination Episciences iam"}],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2011-01-01"},"dateofcollection":"2022-04-12T19:57:46.9Z","dateoftransformation":"2022-04-12T20:18:26.16Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"We examine the $q=1$ and $t=0$ special cases of the parking functions conjecture. The parking functions conjecture states that the Hilbert series for the space of diagonal harmonics is equal to the bivariate generating function of $area$ and $dinv$ over the set of parking functions. Haglund recently proved that the Hilbert series for the space of diagonal harmonics is equal to a bivariate generating function over the set of Tesler matrices–upper-triangular matrices with every hook sum equal to one. We give a combinatorial interpretation of the Haglund generating function at $q=1$ and prove the corresponding case of the parking functions conjecture (first proven by Garsia and Haiman). We also discuss a possible proof of the $t = 0$ case consistent with this combinatorial interpretation. We conclude by briefly discussing possible refinements of the parking functions conjecture arising from this research and point of view. $\\textbf{Note added in proof}$: We have since found such a proof of the $t = 0$ case and conjectured more detailed refinements. This research will most likely be presented in full in a forthcoming article."},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"On examine les cas spéciaux $q=1$ et $t=0$ de la conjecture des fonctions de stationnement. Cette conjecture déclare que la série de Hilbert pour l'espace des harmoniques diagonaux est égale à la fonction génératrice bivariée (paramètres $area$ et $dinv$) sur l'ensemble des fonctions de stationnement. Haglund a prouvé récemment que la série de Hilbert pour l'espace des harmoniques diagonaux est égale à une fonction génératrice bivariée sur l'ensemble des matrices de Tesler triangulaires supérieures dont la somme de chaque équerre vaut un. On donne une interprétation combinatoire de la fonction génératrice de Haglund pour $q=1$ et on prouve le cas correspondant de la conjecture dans le cas des fonctions de stationnement (prouvé d'abord par Garsia et Haiman). On discute aussi d'une preuve possible du cas $t=0$, cohérente avec cette interprétation combinatoire. On conclut en discutant brièvement les raffinements possibles de la conjecture des fonctions de stationnement de ce point de vue. $\\textbf{Note ajoutée sur épreuve}$: j'ai trouvé depuis cet article une preuve du cas $t=0$ et conjecturé des raffinements possibles. Ces résultats seront probablement présentés dans un article ultérieur."}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|06cdd3ff4700::93859bd27121c3ee7c6ee4bfb1790cba","instance":[{"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.46298/dmtcs.2940"}],"collectedfrom":{"key":"10|openaire____::6824b298c96ba906a3e6a70593affbf5","value":"Episciences"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2011-01-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::6824b298c96ba906a3e6a70593affbf5","value":"Episciences"},"instancetype":{"classid":"0001","classname":"Article","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://dmtcs.episciences.org/2940"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1658994348190,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Foai.episciences.org%2F","datestamp":"2011-01-01","harvestDate":"2022-04-12T19:57:46.9Z","identifier":"oai:episciences.org:dmtcs:2940","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["50|06cdd3ff4700::93859bd27121c3ee7c6ee4bfb1790cba","oai:episciences.org:dmtcs:2940"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"Unknown","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"ISSN: 1365-8050"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Discrete Mathematics & Theoretical Computer Science"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Episciences.org"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"dmtcs:2940 - Discrete Mathematics & Theoretical Computer Science, 2011-01-01, DMTCS Proceedings vol. AO, 23rd International Conference on Formal Power Series and Algebraic Combinatorics (FPSAC 2011)"}],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"fos","classname":"Field of Science","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"fake_fos"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Hilbert series"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"sdg","classname":"Support Development Goal","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"fake_sdg"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"[MATH.MATH-CO] Mathematics [math]/Combinatorics [math.CO]"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"acm","classname":"acm","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"[INFO.INFO-DM] Computer Science [cs]/Discrete Mathematics [cs.DM]"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Special Cases of the Parking Functions Conjecture and Upper-Triangular Matrices"}]} +{"author":[{"fullname":"Blondin, Michael","name":"Michael","pid":[],"rank":1,"surname":"Blondin"},{"fullname":"Raskin, Mikhail","name":"Mikhail","pid":[],"rank":2,"surname":"Raskin"}],"bestaccessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::6824b298c96ba906a3e6a70593affbf5","value":"Episciences"}],"context":[],"contributor":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Michael Blondin"}],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2021-07-20"},"dateofcollection":"2022-04-12T19:57:21.4Z","dateoftransformation":"2022-04-12T20:22:30.288Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Vector addition systems with states (VASS) are widely used for the formalverification of concurrent systems. Given their tremendous computationalcomplexity, practical approaches have relied on techniques such as reachabilityrelaxations, e.g., allowing for negative intermediate counter values. It isnatural to question their feasibility for VASS enriched with primitives thattypically translate into undecidability. Spurred by this concern, we pinpointthe complexity of integer relaxations with respect to arbitrary classes ofaffine operations. More specifically, we provide a trichotomy on the complexity of integerreachability in VASS extended with affine operations (affine VASS). Namely, weshow that it is NP-complete for VASS with resets, PSPACE-complete for VASS with(pseudo-)transfers and VASS with (pseudo-)copies, and undecidable for any otherclass. We further present a dichotomy for standard reachability in affine VASS:it is decidable for VASS with permutations, and undecidable for any otherclass. This yields a complete and unified complexity landscape of reachabilityin affine VASS. We also consider the reachability problem parameterized by afixed affine VASS, rather than a class, and we show that the complexitylandscape is arbitrary in this setting."}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98","instance":[{"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.46298/lmcs-17(3:3)2021"}],"collectedfrom":{"key":"10|openaire____::6824b298c96ba906a3e6a70593affbf5","value":"Episciences"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2021-07-20"},"distributionlocation":"","hostedby":{"key":"10|openaire____::6824b298c96ba906a3e6a70593affbf5","value":"Episciences"},"instancetype":{"classid":"0001","classname":"Article","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://lmcs.episciences.org/7687"]}],"language":{"classid":"und","classname":"Undetermined","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1658994354107,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Foai.episciences.org%2F","datestamp":"2021-07-20","harvestDate":"2022-04-12T19:57:21.4Z","identifier":"oai:episciences.org:lmcs:7687","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98","oai:episciences.org:lmcs:7687"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"Unknown","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"ISSN: 1860-5974"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Logical Methods in Computer Science"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Episciences.org"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"lmcs:6872 - Logical Methods in Computer Science, 2021-07-20, Volume 17, Issue 3"}],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Computer Science - Logic in Computer Science"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Computer Science - Computational Complexity"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"Computer Science - Formal Languages and Automata Theory"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"The Complexity of Reachability in Affine Vector Addition Systems with States"}]} +{"author":[{"fullname":"Ward, Mark Daniel","name":"Mark Daniel","pid":[],"rank":1,"surname":"Ward"},{"fullname":"Szpankowski, Wojciech","name":"Wojciech","pid":[],"rank":2,"surname":"Szpankowski"}],"bestaccessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::6824b298c96ba906a3e6a70593affbf5","value":"Episciences"}],"context":[],"contributor":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Coordination Episciences iam"}],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2005-01-01"},"dateofcollection":"2022-04-12T19:57:43.247Z","dateoftransformation":"2022-04-12T20:25:54.387Z","description":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"In a suffix tree, the multiplicity matching parameter (MMP) $M_n$ is the number of leaves in the subtree rooted at the branching point of the $(n+1)$st insertion. Equivalently, the MMP is the number of pointers into the database in the Lempel-Ziv '77 data compression algorithm. We prove that the MMP asymptotically follows the logarithmic series distribution plus some fluctuations. In the proof we compare the distribution of the MMP in suffix trees to its distribution in tries built over independent strings. Our results are derived by both probabilistic and analytic techniques of the analysis of algorithms. In particular, we utilize combinatorics on words, bivariate generating functions, pattern matching, recurrence relations, analytical poissonization and depoissonization, the Mellin transform, and complex analysis."}],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|06cdd3ff4700::ff21e3c55d527fa7db171137c5fd1f1f","instance":[{"accessright":{"classid":"OPEN","classname":"Open Access","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"doi","classname":"Digital Object Identifier","schemeid":"dnet:pid_types","schemename":"dnet:pid_types"},"value":"10.46298/dmtcs.3387"}],"collectedfrom":{"key":"10|openaire____::6824b298c96ba906a3e6a70593affbf5","value":"Episciences"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"2005-01-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::6824b298c96ba906a3e6a70593affbf5","value":"Episciences"},"instancetype":{"classid":"0001","classname":"Article","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://dmtcs.episciences.org/3387"]}],"language":{"classid":"eng","classname":"English","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1658994359132,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Foai.episciences.org%2F","datestamp":"2005-01-01","harvestDate":"2022-04-12T19:57:43.247Z","identifier":"oai:episciences.org:dmtcs:3387","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["50|06cdd3ff4700::ff21e3c55d527fa7db171137c5fd1f1f","oai:episciences.org:dmtcs:3387"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"Unknown","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"ISSN: 1365-8050"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Discrete Mathematics & Theoretical Computer Science"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Episciences.org"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"dmtcs:3387 - Discrete Mathematics & Theoretical Computer Science, 2005-01-01, DMTCS Proceedings vol. AD, International Conference on Analysis of Algorithms"}],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"fos","classname":"Fild of Science","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"fake_fos2"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"complex asymptotics"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"suffix trees"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"combinatorics on words"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"pattern matching"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"autocorrelation polynomial"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"[INFO.INFO-DS] Computer Science [cs]/Data Structures and Algorithms [cs.DS]"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"[INFO.INFO-DM] Computer Science [cs]/Discrete Mathematics [cs.DM]"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"[MATH.MATH-CO] Mathematics [math]/Combinatorics [math.CO]"},{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"keyword","classname":"keyword","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"[INFO.INFO-CG] Computer Science [cs]/Computational Geometry [cs.CG]"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Analysis of the multiplicity matching parameter in suffix trees"}]} +{"author":[{"fullname":"Södergård, Caj","name":"Caj","pid":[],"rank":1,"surname":"Södergård"}],"bestaccessright":{"classid":"RESTRICTED","classname":"Restricted","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"collectedfrom":[{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"}],"context":[],"contributor":[],"country":[],"coverage":[],"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"1989-01-01"},"dateofcollection":"2022-07-09T12:22:11.472Z","dateoftransformation":"2022-07-09T12:45:18.112Z","description":[],"externalReference":[],"extraInfo":[],"format":[],"fulltext":[],"id":"50|355e65625b88::046477dc24819c5f1453166aa7bfb75e","instance":[{"accessright":{"classid":"RESTRICTED","classname":"Restricted","schemeid":"dnet:access_modes","schemename":"dnet:access_modes"},"alternateIdentifier":[],"collectedfrom":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"dateofacceptance":{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"1989-01-01"},"distributionlocation":"","hostedby":{"key":"10|openaire____::4692342f0992d91f9e705c26959f09e0","value":"VTT Research Information System"},"instancetype":{"classid":"0004","classname":"Conference object","schemeid":"dnet:publication_resource","schemename":"dnet:publication_resource"},"pid":[],"refereed":{"classid":"0000","classname":"UNKNOWN","schemeid":"dnet:review_levels","schemename":"dnet:review_levels"},"url":["https://cris.vtt.fi/en/publications/42136eb0-696d-4861-b587-3b451a46a914"]}],"language":{"classid":"fin","classname":"Finnish","schemeid":"dnet:languages","schemename":"dnet:languages"},"lastupdatetimestamp":1658994245711,"oaiprovenance":{"originDescription":{"altered":true,"baseURL":"https%3A%2F%2Fcris.vtt.fi%2Fws%2Foai","datestamp":"2022-06-27T08:32:36Z","harvestDate":"2022-07-09T12:22:11.472Z","identifier":"oai:cris.vtt.fi:publications/42136eb0-696d-4861-b587-3b451a46a914","metadataNamespace":"http://www.openarchives.org/OAI/2.0/oai_dc/"}},"originalId":["50|355e65625b88::046477dc24819c5f1453166aa7bfb75e","oai:cris.vtt.fi:publications/42136eb0-696d-4861-b587-3b451a46a914"],"pid":[],"relevantdate":[],"resourcetype":{"classid":"UNKNOWN","classname":"Unknown","schemeid":"dnet:dataCite_resource","schemename":"dnet:dataCite_resource"},"resulttype":{"classid":"publication","classname":"publication","schemeid":"dnet:result_typologies","schemename":"dnet:result_typologies"},"source":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"value":"Södergård , C 1989 , ' Telefax ja telefoto ' , Kehittyvä tiedonsiirto graafisessa yrityksessä , Helsinki , Finland , 29/05/89 - 30/05/89 ."}],"subject":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"fos","classname":"Fild of Science","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"value":"fake_fos2"}],"title":[{"dataInfo":{"deletedbyinference":false,"inferenceprovenance":"","inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:repository","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.9"},"qualifier":{"classid":"main title","classname":"main title","schemeid":"dnet:dataCite_title","schemename":"dnet:dataCite_title"},"value":"Telefax ja telefoto"}]} \ No newline at end of file diff --git a/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/subjectpropagation/relation/relation.json b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/subjectpropagation/relation/relation.json new file mode 100644 index 000000000..03f9d1298 --- /dev/null +++ b/dhp-workflows/dhp-enrichment/src/test/resources/eu/dnetlib/dhp/subjectpropagation/relation/relation.json @@ -0,0 +1,10 @@ +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"lastupdatetimestamp":1658906673376,"properties":[],"relClass":"IsVersionOf","relType":"datasourceOrganization","source":"50|06cdd3ff4700::93859bd27121c3ee7c6ee4bfb1790cba","subRelType":"provision","target":"50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98","validated":false} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"lastupdatetimestamp":1658906673376,"properties":[],"relClass":"Obsolets","relType":"datasourceOrganization","source":"50|06cdd3ff4700::ff21e3c55d527fa7db171137c5fd1f1f","subRelType":"provision","target":"50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98","validated":false} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"lastupdatetimestamp":1658906673376,"properties":[],"relClass":"IsSupplementedBy","relType":"datasourceOrganization","source":"50|355e65625b88::046477dc24819c5f1453166aa7bfb75e","subRelType":"provision","target":"50|06cdd3ff4700::cd7711c65d518859f1d87056e2c45d98","validated":false} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"lastupdatetimestamp":1658906673376,"properties":[],"relClass":"IsSupplementTo","relType":"datasourceOrganization","source":"50|355e65625b88::046477dc24819c5f1453166aa7bfb75e","subRelType":"provision","target":"50|06cdd3ff4700::93859bd27121c3ee7c6ee4bfb1790cba","validated":false} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"lastupdatetimestamp":1658906673376,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::8b75543067b50076e70764917e188178","subRelType":"provision","target":"20|doajarticles::50cb15ff7a6a3f8531f063770179e346","validated":false} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"lastupdatetimestamp":1658906673376,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::9f3ff882f023209d9ffb4dc32b77d376","subRelType":"provision","target":"20|doajarticles::ffc1811633b3222e4764c7b0517f83e8","validated":false} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"lastupdatetimestamp":1658906673376,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::ccc002b33c28ea8a1eab16db2eebd7a5","subRelType":"provision","target":"20|pending_org_::ab7b11bb317a6249f9b6becc7dd98043","validated":false} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"lastupdatetimestamp":1658906673376,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::d70ec79cbd10ebee6e6f54689d2cc7a9","subRelType":"provision","target":"20|openorgs____::9d6dceaf5e56ef060226e4ef7faa28a0","validated":false} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"lastupdatetimestamp":1658906673376,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::e2d4766f243c621e9a9300e6aa74d5a0","subRelType":"provision","target":"20|pending_org_::cfea8083b4f958b42314b50497aebd59","validated":false} +{"collectedfrom":[{"key":"10|driver______::bee53aa31dc2cbb538c10c2b65fa5824","value":"DOAJ-Articles"}],"dataInfo":{"deletedbyinference":false,"inferred":false,"invisible":false,"provenanceaction":{"classid":"sysimport:crosswalk:entityregistry","classname":"Harvested","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"},"trust":"0.900"},"lastupdatetimestamp":1658906673376,"properties":[],"relClass":"isProvidedBy","relType":"datasourceOrganization","source":"10|doajarticles::fc496a0a4bb49d654649598b04008682","subRelType":"provision","target":"20|pending_org_::a41c5f55ae53f44abb8bc0e89398074e","validated":false} \ No newline at end of file