From 7827a2d66b30577e75b509c706289275db5e0126 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 7 Mar 2024 10:36:30 +0100 Subject: [PATCH 1/2] [OCNEW] added creation of the actionset for the results classified with FoS based ont he OpenAIRE identifier --- .../GetFOSSparkJob.java | 1 + .../PrepareFOSSparkJob.java | 114 +++++++----- .../fosnodoi/CreateActionSetSparkJob.java | 92 +++++++++ .../prepare_parameters.json | 7 +- .../actionmanager/fosnodoi/as_parameters.json | 20 ++ .../fosnodoi/oozie_app/config-default.xml | 30 +++ .../fosnodoi/oozie_app/workflow.xml | 153 +++++++++++++++ .../fosnodoi/CreateActionSetSparkJobTest.java | 176 ++++++++++++++++++ .../actionmanager/fosnodoi/GetFosTest.java | 104 +++++++++++ .../actionmanager/fosnodoi/PrepareTest.java | 99 ++++++++++ .../dhp/actionmanager/fosnodoi/fosnodoi.csv | 19 ++ .../dhp/actionmanager/fosnodoi/fosnodoi.json | 18 ++ .../dhp/actionmanager/fosnodoi/results.json | 9 + 13 files changed, 799 insertions(+), 43 deletions(-) create mode 100644 dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/fosnodoi/CreateActionSetSparkJob.java create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/as_parameters.json create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/oozie_app/workflow.xml create mode 100644 dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/fosnodoi/CreateActionSetSparkJobTest.java create mode 100644 dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/fosnodoi/GetFosTest.java create mode 100644 dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/fosnodoi/PrepareTest.java create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/fosnodoi.csv create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/fosnodoi.json create mode 100644 dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/results.json diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/GetFOSSparkJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/GetFOSSparkJob.java index abea6acd7..c98f4cffc 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/GetFOSSparkJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/GetFOSSparkJob.java @@ -75,6 +75,7 @@ public class GetFOSSparkJob implements Serializable { fosData.map((MapFunction) r -> { FOSDataModel fosDataModel = new FOSDataModel(); fosDataModel.setDoi(r.getString(0).toLowerCase()); + fosDataModel.setOaid(r.getString(1).toLowerCase()); fosDataModel.setLevel1(r.getString(2)); fosDataModel.setLevel2(r.getString(3)); fosDataModel.setLevel3(r.getString(4)); diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareFOSSparkJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareFOSSparkJob.java index b1ffe7f37..ffcaedda7 100644 --- a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareFOSSparkJob.java +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/createunresolvedentities/PrepareFOSSparkJob.java @@ -16,12 +16,14 @@ 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.jetbrains.annotations.NotNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import eu.dnetlib.dhp.actionmanager.createunresolvedentities.model.FOSDataModel; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.Result; import eu.dnetlib.dhp.schema.oaf.StructuredProperty; import eu.dnetlib.dhp.schema.oaf.Subject; @@ -52,62 +54,90 @@ public class PrepareFOSSparkJob implements Serializable { final String outputPath = parser.get("outputPath"); log.info("outputPath: {}", outputPath); + final Boolean distributeDOI = Optional + .ofNullable(parser.get("distributeDoi")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + SparkConf conf = new SparkConf(); runWithSparkSession( conf, isSparkSessionManaged, spark -> { - distributeFOSdois( - spark, - sourcePath, + if (distributeDOI) + distributeFOSdois( + spark, + sourcePath, - outputPath); + outputPath); + else + distributeFOSoaid(spark, sourcePath, outputPath); }); } + private static void distributeFOSoaid(SparkSession spark, String sourcePath, String outputPath) { + Dataset fosDataset = readPath(spark, sourcePath, FOSDataModel.class); + + fosDataset + .groupByKey((MapFunction) v -> v.getOaid().toLowerCase(), Encoders.STRING()) + .mapGroups((MapGroupsFunction) (k, it) -> { + return getResult(ModelSupport.getIdPrefix(Result.class) + "|" + k, it); + }, Encoders.bean(Result.class)) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath + "/fos"); + } + + @NotNull + private static Result getResult(String k, Iterator it) { + Result r = new Result(); + FOSDataModel first = it.next(); + r.setId(k); + + HashSet level1 = new HashSet<>(); + HashSet level2 = new HashSet<>(); + HashSet level3 = new HashSet<>(); + HashSet level4 = new HashSet<>(); + addLevels(level1, level2, level3, level4, first); + it.forEachRemaining(v -> addLevels(level1, level2, level3, level4, v)); + List sbjs = new ArrayList<>(); + level1 + .forEach(l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID))); + level2 + .forEach(l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID))); + level3 + .forEach( + l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID, true))); + level4 + .forEach( + l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID, true))); + r.setSubject(sbjs); + r + .setDataInfo( + OafMapperUtils + .dataInfo( + false, null, true, + false, + OafMapperUtils + .qualifier( + ModelConstants.PROVENANCE_ENRICH, + null, + ModelConstants.DNET_PROVENANCE_ACTIONS, + ModelConstants.DNET_PROVENANCE_ACTIONS), + null)); + return r; + } + private static void distributeFOSdois(SparkSession spark, String sourcePath, String outputPath) { Dataset fosDataset = readPath(spark, sourcePath, FOSDataModel.class); fosDataset .groupByKey((MapFunction) v -> v.getDoi().toLowerCase(), Encoders.STRING()) - .mapGroups((MapGroupsFunction) (k, it) -> { - Result r = new Result(); - FOSDataModel first = it.next(); - r.setId(DHPUtils.generateUnresolvedIdentifier(k, DOI)); - - HashSet level1 = new HashSet<>(); - HashSet level2 = new HashSet<>(); - HashSet level3 = new HashSet<>(); - HashSet level4 = new HashSet<>(); - addLevels(level1, level2, level3, level4, first); - it.forEachRemaining(v -> addLevels(level1, level2, level3, level4, v)); - List sbjs = new ArrayList<>(); - level1 - .forEach(l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID))); - level2 - .forEach(l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID))); - level3 - .forEach( - l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID, true))); - level4 - .forEach( - l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID, true))); - r.setSubject(sbjs); - r - .setDataInfo( - OafMapperUtils - .dataInfo( - false, null, true, - false, - OafMapperUtils - .qualifier( - ModelConstants.PROVENANCE_ENRICH, - null, - ModelConstants.DNET_PROVENANCE_ACTIONS, - ModelConstants.DNET_PROVENANCE_ACTIONS), - null)); - return r; - }, Encoders.bean(Result.class)) + .mapGroups( + (MapGroupsFunction) (k, + it) -> getResult(DHPUtils.generateUnresolvedIdentifier(k, DOI), it), + Encoders.bean(Result.class)) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") diff --git a/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/fosnodoi/CreateActionSetSparkJob.java b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/fosnodoi/CreateActionSetSparkJob.java new file mode 100644 index 000000000..e86fccb84 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/fosnodoi/CreateActionSetSparkJob.java @@ -0,0 +1,92 @@ + +package eu.dnetlib.dhp.actionmanager.fosnodoi; + +import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession; + +import java.io.IOException; +import java.io.Serializable; +import java.util.*; + +import org.apache.commons.cli.ParseException; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.hadoop.mapred.SequenceFileOutputFormat; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.function.FilterFunction; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.MapFunction; +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.opencitations.model.COCI; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.Result; +import eu.dnetlib.dhp.schema.oaf.utils.*; +import scala.Tuple2; + +public class CreateActionSetSparkJob implements Serializable { + + private static final Logger log = LoggerFactory.getLogger(CreateActionSetSparkJob.class); + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + public static void main(final String[] args) throws IOException, ParseException { + + final ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + Objects + .requireNonNull( + CreateActionSetSparkJob.class + .getResourceAsStream( + "/eu/dnetlib/dhp/actionmanager/fosnodoi/as_parameters.json")))); + + parser.parseArgument(args); + + Boolean isSparkSessionManaged = Optional + .ofNullable(parser.get("isSparkSessionManaged")) + .map(Boolean::valueOf) + .orElse(Boolean.TRUE); + + log.info("isSparkSessionManaged: {}", isSparkSessionManaged); + + final String inputPath = parser.get("sourcePath"); + log.info("inputPath {}", inputPath); + + final String outputPath = parser.get("outputPath"); + log.info("outputPath {}", outputPath); + + SparkConf conf = new SparkConf(); + runWithSparkSession( + conf, + isSparkSessionManaged, + spark -> createActionSet(spark, inputPath, outputPath)); + + } + + private static void createActionSet(SparkSession spark, String inputPath, String outputPath) { + spark + .read() + .textFile(inputPath) + .map( + (MapFunction) value -> OBJECT_MAPPER.readValue(value, Result.class), + Encoders.bean(Result.class)) + .toJavaRDD() + .map(p -> new AtomicAction(p.getClass(), p)) + .mapToPair( + aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()), + new Text(OBJECT_MAPPER.writeValueAsString(aa)))) + .saveAsHadoopFile( + outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class); + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/createunresolvedentities/prepare_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/createunresolvedentities/prepare_parameters.json index b7bad73e6..b7f0f2aa3 100644 --- a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/createunresolvedentities/prepare_parameters.json +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/createunresolvedentities/prepare_parameters.json @@ -16,5 +16,10 @@ "paramLongName": "outputPath", "paramDescription": "the path of the new ActionSet", "paramRequired": true - } + }, { + "paramName": "fd", + "paramLongName": "distributeDoi", + "paramDescription": "the path of the new ActionSet", + "paramRequired": false +} ] \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/as_parameters.json b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/as_parameters.json new file mode 100644 index 000000000..3f056edf7 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/as_parameters.json @@ -0,0 +1,20 @@ +[ + { + "paramName": "sp", + "paramLongName": "sourcePath", + "paramDescription": "the zipped opencitations file", + "paramRequired": true + }, + { + "paramName": "op", + "paramLongName": "outputPath", + "paramDescription": "the working path", + "paramRequired": true + }, + { + "paramName": "issm", + "paramLongName": "isSparkSessionManaged", + "paramDescription": "the hdfs name node", + "paramRequired": false + } +] diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/oozie_app/config-default.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/oozie_app/config-default.xml new file mode 100644 index 000000000..d262cb6e0 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/oozie_app/config-default.xml @@ -0,0 +1,30 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + hiveMetastoreUris + thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083 + + + hiveJdbcUrl + jdbc:hive2://iis-cdh5-test-m3.ocean.icm.edu.pl:10000 + + + hiveDbName + openaire + + + oozie.launcher.mapreduce.user.classpath.first + true + + diff --git a/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/oozie_app/workflow.xml b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/oozie_app/workflow.xml new file mode 100644 index 000000000..ec3e25ac0 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/main/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/oozie_app/workflow.xml @@ -0,0 +1,153 @@ + + + + + fosPath + the input path of the resources to be extended + + + outputPath + the path where to store the actionset + + + sparkDriverMemory + memory for driver process + + + sparkExecutorMemory + memory for individual executor + + + sparkExecutorCores + number of cores used by single executor + + + oozieActionShareLibForSpark2 + oozie action sharelib for spark 2.* + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + spark 2.* extra listeners classname + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + spark 2.* sql query execution listeners classname + + + spark2YarnHistoryServerAddress + spark 2.* yarn history server address + + + spark2EventLogDir + spark 2.* event log dir location + + + + + ${jobTracker} + ${nameNode} + + + mapreduce.job.queuename + ${queueName} + + + oozie.launcher.mapred.job.queue.name + ${oozieLauncherQueueName} + + + oozie.action.sharelib.for.spark + ${oozieActionShareLibForSpark2} + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + yarn + cluster + Gets Data from FOS csv file + eu.dnetlib.dhp.actionmanager.createunresolvedentities.GetFOSSparkJob + dhp-aggregation-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + + --sourcePath${fosPath} + --outputPath${workingDir}/input/fos + --delimiter${delimiter} + + + + + + + + + yarn + cluster + Produces the results from FOS + eu.dnetlib.dhp.actionmanager.createunresolvedentities.PrepareFOSSparkJob + dhp-aggregation-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + + --sourcePath${workingDir}/input/fos + --outputPath${workingDir}/prepared + --distributeDoifalse + + + + + + + + + + yarn + cluster + Save the action set grouping results with the same id + eu.dnetlib.dhp.actionmanager.fosnodoi.CreateActionSetSparkJob + dhp-aggregation-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir} + + --sourcePath${workingDir}/prepared/fos + --outputPath${outputPath} + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/fosnodoi/CreateActionSetSparkJobTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/fosnodoi/CreateActionSetSparkJobTest.java new file mode 100644 index 000000000..fa98e96d2 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/fosnodoi/CreateActionSetSparkJobTest.java @@ -0,0 +1,176 @@ + +package eu.dnetlib.dhp.actionmanager.fosnodoi; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.io.Text; +import org.apache.spark.SparkConf; +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.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.schema.action.AtomicAction; +import eu.dnetlib.dhp.schema.oaf.OafEntity; + +public class CreateActionSetSparkJobTest { + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private static SparkSession spark; + + private static Path workingDir; + + private static final Logger log = LoggerFactory.getLogger(CreateActionSetSparkJobTest.class); + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files + .createTempDirectory(CreateActionSetSparkJobTest.class.getSimpleName()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(CreateActionSetSparkJobTest.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(CreateActionSetSparkJobTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + private void runJob(String resultsInputPath, String outputPath) throws Exception { + CreateActionSetSparkJob + .main( + new String[] { + "-isSparkSessionManaged", Boolean.FALSE.toString(), + "-sourcePath", resultsInputPath, + + "-outputPath", outputPath, + }); + } + + @Test + void testScores() throws Exception { + + String resultsInputPath = getClass() + .getResource("/eu/dnetlib/dhp/actionmanager/fosnodoi/results.json") + .getPath(); + + String outputPath = workingDir.toString() + "/actionSet"; + + // execute the job to generate the action sets for result scores + runJob(resultsInputPath, outputPath); + final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); + + sc + .sequenceFile(outputPath, Text.class, Text.class) + .foreach(t -> System.out.println(new ObjectMapper().writeValueAsString(t))); + + } + + void assertResultImpactScores(Dataset testDataset) { + Assertions + .assertEquals( + "6.63451994567e-09", testDataset + .filter( + "oaid='50|arXiv_dedup_::4a2d5fd8d71daec016c176ec71d957b1' " + + "and id = 'influence'") + .select("value") + .collectAsList() + .get(0) + .getString(0)); + Assertions + .assertEquals( + "0.348694533145", testDataset + .filter( + "oaid='50|arXiv_dedup_::4a2d5fd8d71daec016c176ec71d957b1' " + + "and id = 'popularity_alt'") + .select("value") + .collectAsList() + .get(0) + .getString(0)); + Assertions + .assertEquals( + "2.16094680115e-09", testDataset + .filter( + "oaid='50|arXiv_dedup_::4a2d5fd8d71daec016c176ec71d957b1' " + + "and id = 'popularity'") + .select("value") + .collectAsList() + .get(0) + .getString(0)); + } + + void assertProjectImpactScores(Dataset testDataset) throws Exception { + + Assertions + .assertEquals( + "0", testDataset + .filter( + "oaid='40|nih_________::c02a8233e9b60f05bb418f0c9b714833' " + + "and id = 'numOfInfluentialResults'") + .select("value") + .collectAsList() + .get(0) + .getString(0)); + Assertions + .assertEquals( + "1", testDataset + .filter( + "oaid='40|nih_________::c02a8233e9b60f05bb418f0c9b714833' " + + "and id = 'numOfPopularResults'") + .select("value") + .collectAsList() + .get(0) + .getString(0)); + Assertions + .assertEquals( + "25", testDataset + .filter( + "oaid='40|nih_________::c02a8233e9b60f05bb418f0c9b714833' " + + "and id = 'totalImpulse'") + .select("value") + .collectAsList() + .get(0) + .getString(0)); + Assertions + .assertEquals( + "43", testDataset + .filter( + "oaid='40|nih_________::c02a8233e9b60f05bb418f0c9b714833' " + + "and id = 'totalCitationCount'") + .select("value") + .collectAsList() + .get(0) + .getString(0)); + } +} diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/fosnodoi/GetFosTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/fosnodoi/GetFosTest.java new file mode 100644 index 000000000..4e558dbd4 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/fosnodoi/GetFosTest.java @@ -0,0 +1,104 @@ + +package eu.dnetlib.dhp.actionmanager.fosnodoi; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.actionmanager.createunresolvedentities.GetFOSSparkJob; +import eu.dnetlib.dhp.actionmanager.createunresolvedentities.PrepareTest; +import eu.dnetlib.dhp.actionmanager.createunresolvedentities.ProduceTest; +import eu.dnetlib.dhp.actionmanager.createunresolvedentities.model.FOSDataModel; + +/** + * @author miriam.baglioni + * @Date 13/02/23 + */ +public class GetFosTest { + + private static final Logger log = LoggerFactory.getLogger(ProduceTest.class); + + private static Path workingDir; + private static SparkSession spark; + private static LocalFileSystem fs; + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(PrepareTest.class.getSimpleName()); + + fs = FileSystem.getLocal(new Configuration()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(ProduceTest.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(PrepareTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + @Disabled + void test3() throws Exception { + final String sourcePath = getClass() + .getResource("/eu/dnetlib/dhp/actionmanager/fosnodoi/fosnodoi.csv") + .getPath(); + + final String outputPath = workingDir.toString() + "/fos.json"; + GetFOSSparkJob + .main( + new String[] { + "--isSparkSessionManaged", Boolean.FALSE.toString(), + "--sourcePath", sourcePath, + + "-outputPath", outputPath, + "-delimiter", "," + + }); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(outputPath) + .map(item -> OBJECT_MAPPER.readValue(item, FOSDataModel.class)); + + tmp.foreach(t -> Assertions.assertTrue(t.getOaid() != null)); + tmp.foreach(t -> Assertions.assertTrue(t.getLevel1() != null)); + tmp.foreach(t -> Assertions.assertTrue(t.getLevel2() != null)); + tmp.foreach(t -> Assertions.assertTrue(t.getLevel3() != null)); + + tmp.foreach(t -> System.out.println(new ObjectMapper().writeValueAsString(t))); + + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/fosnodoi/PrepareTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/fosnodoi/PrepareTest.java new file mode 100644 index 000000000..f0cc471d8 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/fosnodoi/PrepareTest.java @@ -0,0 +1,99 @@ + +package eu.dnetlib.dhp.actionmanager.fosnodoi; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import eu.dnetlib.dhp.actionmanager.createunresolvedentities.PrepareFOSSparkJob; +import eu.dnetlib.dhp.actionmanager.createunresolvedentities.PrepareSDGSparkJob; +import eu.dnetlib.dhp.actionmanager.createunresolvedentities.ProduceTest; +import eu.dnetlib.dhp.schema.oaf.Result; + +public class PrepareTest { + + private static final Logger log = LoggerFactory.getLogger(ProduceTest.class); + + private static Path workingDir; + private static SparkSession spark; + private static LocalFileSystem fs; + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + @BeforeAll + public static void beforeAll() throws IOException { + workingDir = Files.createTempDirectory(PrepareTest.class.getSimpleName()); + + fs = FileSystem.getLocal(new Configuration()); + log.info("using work dir {}", workingDir); + + SparkConf conf = new SparkConf(); + conf.setAppName(ProduceTest.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(PrepareTest.class.getSimpleName()) + .config(conf) + .getOrCreate(); + } + + @AfterAll + public static void afterAll() throws IOException { + FileUtils.deleteDirectory(workingDir.toFile()); + spark.stop(); + } + + @Test + void fosPrepareTest() throws Exception { + final String sourcePath = getClass() + .getResource("/eu/dnetlib/dhp/actionmanager/fosnodoi/fosnodoi.json") + .getPath(); + + PrepareFOSSparkJob + .main( + new String[] { + "--isSparkSessionManaged", Boolean.FALSE.toString(), + "--sourcePath", sourcePath, + + "-outputPath", workingDir.toString() + "/work", + "-distributeDoi", Boolean.FALSE.toString() + + }); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + JavaRDD tmp = sc + .textFile(workingDir.toString() + "/work/fos") + .map(item -> OBJECT_MAPPER.readValue(item, Result.class)); + + tmp.foreach(t -> System.out.println(new ObjectMapper().writeValueAsString(t))); + + } + +} diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/fosnodoi.csv b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/fosnodoi.csv new file mode 100644 index 000000000..00c97cff0 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/fosnodoi.csv @@ -0,0 +1,19 @@ +DOI,OAID,level1,level2,level3,level4,score_for_L3,score_for_L4 +N/A,78975075580c::e680668c98366c9cd6349afc62486a7f,03 medical and health sciences,0301 basic medicine,030104 developmental biology,N/A,0.5,0.0 +N/A,78975075580c::e680668c98366c9cd6349afc62486a7f,03 medical and health sciences,0303 health sciences,030304 developmental biology,N/A,0.5,0.0 +N/A,od______2806::a1da9d2678b12969a9ab5f50b5e71d0a,05 social sciences,0501 psychology and cognitive sciences,050109 social psychology,05010904 Group processes/Collective identity,0.5589094161987305,0.5166763067245483 +N/A,od______2806::a1da9d2678b12969a9ab5f50b5e71d0a,05 social sciences,0501 psychology and cognitive sciences,050105 experimental psychology,05010501 Emotion/Affective science,0.44109055399894714,0.4833236634731293 +N/A,doajarticles::76535d77fd2a5fe9810aefafffb8ef6c,05 social sciences,0502 economics and business,050203 business & management,05020302 Supply chain management/Business terms,0.5459638833999634,0.5460261106491089 +N/A,doajarticles::76535d77fd2a5fe9810aefafffb8ef6c,05 social sciences,0502 economics and business,050211 marketing,05021102 Services marketing/Retailing,0.4540362060070038,0.4539738595485687 +N/A,od_______156::a3a0119c6d9d3a66943f8da042e97a5e,01 natural sciences,0105 earth and related environmental sciences,010504 meteorology & atmospheric sciences,01050407 Geomagnetism/Ionosphere,0.5131047964096069,0.4990350902080536 +N/A,od_______156::a3a0119c6d9d3a66943f8da042e97a5e,01 natural sciences,0105 earth and related environmental sciences,010502 geochemistry & geophysics,01050203 Seismology/Seismology measurement,0.4868951737880707,0.500964879989624 +N/A,od______2806::4b9a664dd6b8b04204cb613e7bc9c873,03 medical and health sciences,0302 clinical medicine,030220 oncology & carcinogenesis,03022002 Medical imaging/Medical physics,0.5068133473396301,0.10231181626910052 +N/A,od______2806::4b9a664dd6b8b04204cb613e7bc9c873,03 medical and health sciences,0302 clinical medicine,030204 cardiovascular system & hematology,N/A,0.49318668246269226,0.0 +N/A,od______3341::ef754de29464abf9bc9b99664630ce74,03 medical and health sciences,0302 clinical medicine,030220 oncology & carcinogenesis,03022012 Oncology/Infectious causes of cancer,0.5,0.5 +N/A,od______3341::ef754de29464abf9bc9b99664630ce74,03 medical and health sciences,0302 clinical medicine,030220 oncology & carcinogenesis,03022012 Oncology/Infectious causes of cancer,0.5,0.5 +N/A,od______3978::6704dcced0fe3dd6fbf985dc2507f61c,03 medical and health sciences,0302 clinical medicine,030217 neurology & neurosurgery,03021702 Aging-associated diseases/Cognitive disorders,0.5134317874908447,0.09614889098529535 +N/A,od______3978::6704dcced0fe3dd6fbf985dc2507f61c,03 medical and health sciences,0301 basic medicine,030104 developmental biology,N/A,0.48656824231147766,0.0 +N/A,dedup_wf_001::b77264819800b90c0328c4d17eea5c1a,02 engineering and technology,0209 industrial biotechnology,020901 industrial engineering & automation,02090105 Control theory/Advanced driver assistance systems,0.5178514122962952,0.5198937654495239 +N/A,dedup_wf_001::b77264819800b90c0328c4d17eea5c1a,02 engineering and technology,"0202 electrical engineering, electronic engineering, information engineering",020201 artificial intelligence & image processing,02020108 Fuzzy logic/Artificial neural networks/Computational neuroscience,0.48214852809906006,0.4801062345504761 +N/A,od______2806::a938609e9f36ada6629a1bcc50c88230,03 medical and health sciences,0302 clinical medicine,030217 neurology & neurosurgery,03021708 Neurotrauma/Stroke,0.5014800429344177,0.5109656453132629 +N/A,od______2806::a938609e9f36ada6629a1bcc50c88230,02 engineering and technology,0206 medical engineering,020601 biomedical engineering,02060102 Medical terminology/Patient,0.4985199570655823,0.4890343248844147 \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/fosnodoi.json b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/fosnodoi.json new file mode 100644 index 000000000..782954d49 --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/fosnodoi.json @@ -0,0 +1,18 @@ +{"doi":"n/a","oaid":"od______3341::ef754de29464abf9bc9b99664630ce74","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030220 oncology & carcinogenesis","level4":"03022012 Oncology/Infectious causes of cancer","scoreL3":"0.5","scoreL4":"0.5"} +{"doi":"n/a","oaid":"78975075580c::e680668c98366c9cd6349afc62486a7f","level1":"03 medical and health sciences","level2":"0301 basic medicine","level3":"030104 developmental biology","level4":"N/A","scoreL3":"0.5","scoreL4":"0.0"} +{"doi":"n/a","oaid":"od______3341::ef754de29464abf9bc9b99664630ce74","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030220 oncology & carcinogenesis","level4":"03022012 Oncology/Infectious causes of cancer","scoreL3":"0.5","scoreL4":"0.5"} +{"doi":"n/a","oaid":"78975075580c::e680668c98366c9cd6349afc62486a7f","level1":"03 medical and health sciences","level2":"0303 health sciences","level3":"030304 developmental biology","level4":"N/A","scoreL3":"0.5","scoreL4":"0.0"} +{"doi":"n/a","oaid":"od______3978::6704dcced0fe3dd6fbf985dc2507f61c","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030217 neurology & neurosurgery","level4":"03021702 Aging-associated diseases/Cognitive disorders","scoreL3":"0.5134317874908447","scoreL4":"0.09614889098529535"} +{"doi":"n/a","oaid":"od______2806::a1da9d2678b12969a9ab5f50b5e71d0a","level1":"05 social sciences","level2":"0501 psychology and cognitive sciences","level3":"050109 social psychology","level4":"05010904 Group processes/Collective identity","scoreL3":"0.5589094161987305","scoreL4":"0.5166763067245483"} +{"doi":"n/a","oaid":"od______3978::6704dcced0fe3dd6fbf985dc2507f61c","level1":"03 medical and health sciences","level2":"0301 basic medicine","level3":"030104 developmental biology","level4":"N/A","scoreL3":"0.48656824231147766","scoreL4":"0.0"} +{"doi":"n/a","oaid":"od______2806::a1da9d2678b12969a9ab5f50b5e71d0a","level1":"05 social sciences","level2":"0501 psychology and cognitive sciences","level3":"050105 experimental psychology","level4":"05010501 Emotion/Affective science","scoreL3":"0.44109055399894714","scoreL4":"0.4833236634731293"} +{"doi":"n/a","oaid":"dedup_wf_001::b77264819800b90c0328c4d17eea5c1a","level1":"02 engineering and technology","level2":"0209 industrial biotechnology","level3":"020901 industrial engineering & automation","level4":"02090105 Control theory/Advanced driver assistance systems","scoreL3":"0.5178514122962952","scoreL4":"0.5198937654495239"} +{"doi":"n/a","oaid":"doajarticles::76535d77fd2a5fe9810aefafffb8ef6c","level1":"05 social sciences","level2":"0502 economics and business","level3":"050203 business & management","level4":"05020302 Supply chain management/Business terms","scoreL3":"0.5459638833999634","scoreL4":"0.5460261106491089"} +{"doi":"n/a","oaid":"doajarticles::76535d77fd2a5fe9810aefafffb8ef6c","level1":"05 social sciences","level2":"0502 economics and business","level3":"050211 marketing","level4":"05021102 Services marketing/Retailing","scoreL3":"0.4540362060070038","scoreL4":"0.4539738595485687"} +{"doi":"n/a","oaid":"dedup_wf_001::b77264819800b90c0328c4d17eea5c1a","level1":"02 engineering and technology","level2":"0202 electrical engineering, electronic engineering, information engineering","level3":"020201 artificial intelligence & image processing","level4":"02020108 Fuzzy logic/Artificial neural networks/Computational neuroscience","scoreL3":"0.48214852809906006","scoreL4":"0.4801062345504761"} +{"doi":"n/a","oaid":"od_______156::a3a0119c6d9d3a66943f8da042e97a5e","level1":"01 natural sciences","level2":"0105 earth and related environmental sciences","level3":"010504 meteorology & atmospheric sciences","level4":"01050407 Geomagnetism/Ionosphere","scoreL3":"0.5131047964096069","scoreL4":"0.4990350902080536"} +{"doi":"n/a","oaid":"od______2806::a938609e9f36ada6629a1bcc50c88230","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030217 neurology & neurosurgery","level4":"03021708 Neurotrauma/Stroke","scoreL3":"0.5014800429344177","scoreL4":"0.5109656453132629"} +{"doi":"n/a","oaid":"od_______156::a3a0119c6d9d3a66943f8da042e97a5e","level1":"01 natural sciences","level2":"0105 earth and related environmental sciences","level3":"010502 geochemistry & geophysics","level4":"01050203 Seismology/Seismology measurement","scoreL3":"0.4868951737880707","scoreL4":"0.500964879989624"} +{"doi":"n/a","oaid":"od______2806::a938609e9f36ada6629a1bcc50c88230","level1":"02 engineering and technology","level2":"0206 medical engineering","level3":"020601 biomedical engineering","level4":"02060102 Medical terminology/Patient","scoreL3":"0.4985199570655823","scoreL4":"0.4890343248844147"} +{"doi":"n/a","oaid":"od______2806::4b9a664dd6b8b04204cb613e7bc9c873","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030220 oncology & carcinogenesis","level4":"03022002 Medical imaging/Medical physics","scoreL3":"0.5068133473396301","scoreL4":"0.10231181626910052"} +{"doi":"n/a","oaid":"od______2806::4b9a664dd6b8b04204cb613e7bc9c873","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030204 cardiovascular system & hematology","level4":"N/A","scoreL3":"0.49318668246269226","scoreL4":"0.0"} \ No newline at end of file diff --git a/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/results.json b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/results.json new file mode 100644 index 000000000..9209111bb --- /dev/null +++ b/dhp-workflows/dhp-aggregation/src/test/resources/eu/dnetlib/dhp/actionmanager/fosnodoi/results.json @@ -0,0 +1,9 @@ +{"collectedfrom":null,"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":null,"inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:enrich","classname":null,"schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":null,"id":"null|dedup_wf_001::b77264819800b90c0328c4d17eea5c1a","originalId":null,"pid":null,"dateofcollection":null,"dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"processingchargeamount":null,"processingchargecurrency":null,"author":null,"resulttype":null,"metaResourceType":null,"language":null,"country":null,"subject":[{"value":"02 engineering and technology","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"0202 electrical engineering, electronic engineering, information engineering","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"0209 industrial biotechnology","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"020201 artificial intelligence & image processing","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.48214852809906006","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"020901 industrial engineering & automation","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.5178514122962952","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"02020108 Fuzzy logic/Artificial neural networks/Computational neuroscience","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.4801062345504761","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"02090105 Control theory/Advanced driver assistance systems","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.5198937654495239","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":null,"relevantdate":null,"description":null,"dateofacceptance":null,"publisher":null,"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":null,"eoscifguidelines":null,"openAccessColor":null,"publiclyFunded":null,"isGreen":null,"isInDiamondJournal":null} +{"collectedfrom":null,"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":null,"inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:enrich","classname":null,"schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":null,"id":"null|od______2806::4b9a664dd6b8b04204cb613e7bc9c873","originalId":null,"pid":null,"dateofcollection":null,"dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"processingchargeamount":null,"processingchargecurrency":null,"author":null,"resulttype":null,"metaResourceType":null,"language":null,"country":null,"subject":[{"value":"03 medical and health sciences","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"0302 clinical medicine","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"030220 oncology & carcinogenesis","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.5068133473396301","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"030204 cardiovascular system & hematology","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.49318668246269226","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"03022002 Medical imaging/Medical physics","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.10231181626910052","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":null,"relevantdate":null,"description":null,"dateofacceptance":null,"publisher":null,"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":null,"eoscifguidelines":null,"openAccessColor":null,"publiclyFunded":null,"isGreen":null,"isInDiamondJournal":null} +{"collectedfrom":null,"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":null,"inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:enrich","classname":null,"schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":null,"id":"null|od______3978::6704dcced0fe3dd6fbf985dc2507f61c","originalId":null,"pid":null,"dateofcollection":null,"dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"processingchargeamount":null,"processingchargecurrency":null,"author":null,"resulttype":null,"metaResourceType":null,"language":null,"country":null,"subject":[{"value":"03 medical and health sciences","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"0302 clinical medicine","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"0301 basic medicine","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"030217 neurology & neurosurgery","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.5134317874908447","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"030104 developmental biology","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.48656824231147766","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"03021702 Aging-associated diseases/Cognitive disorders","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.09614889098529535","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":null,"relevantdate":null,"description":null,"dateofacceptance":null,"publisher":null,"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":null,"eoscifguidelines":null,"openAccessColor":null,"publiclyFunded":null,"isGreen":null,"isInDiamondJournal":null} +{"collectedfrom":null,"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":null,"inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:enrich","classname":null,"schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":null,"id":"null|doajarticles::76535d77fd2a5fe9810aefafffb8ef6c","originalId":null,"pid":null,"dateofcollection":null,"dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"processingchargeamount":null,"processingchargecurrency":null,"author":null,"resulttype":null,"metaResourceType":null,"language":null,"country":null,"subject":[{"value":"05 social sciences","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"0502 economics and business","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"050203 business & management","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.5459638833999634","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"050211 marketing","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.4540362060070038","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"05021102 Services marketing/Retailing","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.4539738595485687","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"05020302 Supply chain management/Business terms","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.5460261106491089","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":null,"relevantdate":null,"description":null,"dateofacceptance":null,"publisher":null,"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":null,"eoscifguidelines":null,"openAccessColor":null,"publiclyFunded":null,"isGreen":null,"isInDiamondJournal":null} +{"collectedfrom":null,"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":null,"inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:enrich","classname":null,"schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":null,"id":"null|78975075580c::e680668c98366c9cd6349afc62486a7f","originalId":null,"pid":null,"dateofcollection":null,"dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"processingchargeamount":null,"processingchargecurrency":null,"author":null,"resulttype":null,"metaResourceType":null,"language":null,"country":null,"subject":[{"value":"03 medical and health sciences","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"0301 basic medicine","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"0303 health sciences","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"030104 developmental biology","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.5","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"030304 developmental biology","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.5","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":null,"relevantdate":null,"description":null,"dateofacceptance":null,"publisher":null,"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":null,"eoscifguidelines":null,"openAccessColor":null,"publiclyFunded":null,"isGreen":null,"isInDiamondJournal":null} +{"collectedfrom":null,"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":null,"inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:enrich","classname":null,"schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":null,"id":"null|od_______156::a3a0119c6d9d3a66943f8da042e97a5e","originalId":null,"pid":null,"dateofcollection":null,"dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"processingchargeamount":null,"processingchargecurrency":null,"author":null,"resulttype":null,"metaResourceType":null,"language":null,"country":null,"subject":[{"value":"01 natural sciences","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"0105 earth and related environmental sciences","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"010504 meteorology & atmospheric sciences","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.5131047964096069","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"010502 geochemistry & geophysics","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.4868951737880707","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"01050203 Seismology/Seismology measurement","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.500964879989624","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"01050407 Geomagnetism/Ionosphere","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.4990350902080536","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":null,"relevantdate":null,"description":null,"dateofacceptance":null,"publisher":null,"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":null,"eoscifguidelines":null,"openAccessColor":null,"publiclyFunded":null,"isGreen":null,"isInDiamondJournal":null} +{"collectedfrom":null,"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":null,"inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:enrich","classname":null,"schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":null,"id":"null|od______2806::a938609e9f36ada6629a1bcc50c88230","originalId":null,"pid":null,"dateofcollection":null,"dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"processingchargeamount":null,"processingchargecurrency":null,"author":null,"resulttype":null,"metaResourceType":null,"language":null,"country":null,"subject":[{"value":"03 medical and health sciences","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"02 engineering and technology","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"0206 medical engineering","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"0302 clinical medicine","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"030217 neurology & neurosurgery","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.5014800429344177","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"020601 biomedical engineering","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.4985199570655823","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"03021708 Neurotrauma/Stroke","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.5109656453132629","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"02060102 Medical terminology/Patient","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.4890343248844147","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":null,"relevantdate":null,"description":null,"dateofacceptance":null,"publisher":null,"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":null,"eoscifguidelines":null,"openAccessColor":null,"publiclyFunded":null,"isGreen":null,"isInDiamondJournal":null} +{"collectedfrom":null,"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":null,"inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:enrich","classname":null,"schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":null,"id":"null|od______3341::ef754de29464abf9bc9b99664630ce74","originalId":null,"pid":null,"dateofcollection":null,"dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"processingchargeamount":null,"processingchargecurrency":null,"author":null,"resulttype":null,"metaResourceType":null,"language":null,"country":null,"subject":[{"value":"03 medical and health sciences","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"0302 clinical medicine","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"030220 oncology & carcinogenesis","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.5","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"03022012 Oncology/Infectious causes of cancer","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.5","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":null,"relevantdate":null,"description":null,"dateofacceptance":null,"publisher":null,"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":null,"eoscifguidelines":null,"openAccessColor":null,"publiclyFunded":null,"isGreen":null,"isInDiamondJournal":null} +{"collectedfrom":null,"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":null,"inferenceprovenance":null,"provenanceaction":{"classid":"sysimport:enrich","classname":null,"schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}},"lastupdatetimestamp":null,"id":"null|od______2806::a1da9d2678b12969a9ab5f50b5e71d0a","originalId":null,"pid":null,"dateofcollection":null,"dateoftransformation":null,"extraInfo":null,"oaiprovenance":null,"measures":null,"processingchargeamount":null,"processingchargecurrency":null,"author":null,"resulttype":null,"metaResourceType":null,"language":null,"country":null,"subject":[{"value":"05 social sciences","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"0501 psychology and cognitive sciences","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"050105 experimental psychology","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.44109055399894714","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"050109 social psychology","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.5589094161987305","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"05010904 Group processes/Collective identity","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.5166763067245483","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}},{"value":"05010501 Emotion/Affective science","qualifier":{"classid":"FOS","classname":"Fields of Science and Technology classification","schemeid":"dnet:subject_classification_typologies","schemename":"dnet:subject_classification_typologies"},"dataInfo":{"invisible":false,"inferred":true,"deletedbyinference":false,"trust":"0.4833236634731293","inferenceprovenance":"update","provenanceaction":{"classid":"subject:fos","classname":"Inferred by OpenAIRE","schemeid":"dnet:provenanceActions","schemename":"dnet:provenanceActions"}}}],"title":null,"relevantdate":null,"description":null,"dateofacceptance":null,"publisher":null,"embargoenddate":null,"source":null,"fulltext":null,"format":null,"contributor":null,"resourcetype":null,"coverage":null,"bestaccessright":null,"context":null,"externalReference":null,"instance":null,"eoscifguidelines":null,"openAccessColor":null,"publiclyFunded":null,"isGreen":null,"isInDiamondJournal":null} \ No newline at end of file From 5180b6ec8a764ac52a92dbcf10cc20742f3d8fd1 Mon Sep 17 00:00:00 2001 From: "miriam.baglioni" Date: Thu, 7 Mar 2024 10:47:13 +0100 Subject: [PATCH 2/2] [FOSNEW] removed test class --- .../fosnodoi/CreateActionSetSparkJobTest.java | 176 ------------------ 1 file changed, 176 deletions(-) delete mode 100644 dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/fosnodoi/CreateActionSetSparkJobTest.java diff --git a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/fosnodoi/CreateActionSetSparkJobTest.java b/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/fosnodoi/CreateActionSetSparkJobTest.java deleted file mode 100644 index fa98e96d2..000000000 --- a/dhp-workflows/dhp-aggregation/src/test/java/eu/dnetlib/dhp/actionmanager/fosnodoi/CreateActionSetSparkJobTest.java +++ /dev/null @@ -1,176 +0,0 @@ - -package eu.dnetlib.dhp.actionmanager.fosnodoi; - -import static org.junit.jupiter.api.Assertions.assertEquals; - -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; - -import org.apache.commons.io.FileUtils; -import org.apache.hadoop.io.Text; -import org.apache.spark.SparkConf; -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.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.schema.action.AtomicAction; -import eu.dnetlib.dhp.schema.oaf.OafEntity; - -public class CreateActionSetSparkJobTest { - - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - - private static SparkSession spark; - - private static Path workingDir; - - private static final Logger log = LoggerFactory.getLogger(CreateActionSetSparkJobTest.class); - - @BeforeAll - public static void beforeAll() throws IOException { - workingDir = Files - .createTempDirectory(CreateActionSetSparkJobTest.class.getSimpleName()); - log.info("using work dir {}", workingDir); - - SparkConf conf = new SparkConf(); - conf.setAppName(CreateActionSetSparkJobTest.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(CreateActionSetSparkJobTest.class.getSimpleName()) - .config(conf) - .getOrCreate(); - } - - @AfterAll - public static void afterAll() throws IOException { - FileUtils.deleteDirectory(workingDir.toFile()); - spark.stop(); - } - - private void runJob(String resultsInputPath, String outputPath) throws Exception { - CreateActionSetSparkJob - .main( - new String[] { - "-isSparkSessionManaged", Boolean.FALSE.toString(), - "-sourcePath", resultsInputPath, - - "-outputPath", outputPath, - }); - } - - @Test - void testScores() throws Exception { - - String resultsInputPath = getClass() - .getResource("/eu/dnetlib/dhp/actionmanager/fosnodoi/results.json") - .getPath(); - - String outputPath = workingDir.toString() + "/actionSet"; - - // execute the job to generate the action sets for result scores - runJob(resultsInputPath, outputPath); - final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext()); - - sc - .sequenceFile(outputPath, Text.class, Text.class) - .foreach(t -> System.out.println(new ObjectMapper().writeValueAsString(t))); - - } - - void assertResultImpactScores(Dataset testDataset) { - Assertions - .assertEquals( - "6.63451994567e-09", testDataset - .filter( - "oaid='50|arXiv_dedup_::4a2d5fd8d71daec016c176ec71d957b1' " + - "and id = 'influence'") - .select("value") - .collectAsList() - .get(0) - .getString(0)); - Assertions - .assertEquals( - "0.348694533145", testDataset - .filter( - "oaid='50|arXiv_dedup_::4a2d5fd8d71daec016c176ec71d957b1' " + - "and id = 'popularity_alt'") - .select("value") - .collectAsList() - .get(0) - .getString(0)); - Assertions - .assertEquals( - "2.16094680115e-09", testDataset - .filter( - "oaid='50|arXiv_dedup_::4a2d5fd8d71daec016c176ec71d957b1' " + - "and id = 'popularity'") - .select("value") - .collectAsList() - .get(0) - .getString(0)); - } - - void assertProjectImpactScores(Dataset testDataset) throws Exception { - - Assertions - .assertEquals( - "0", testDataset - .filter( - "oaid='40|nih_________::c02a8233e9b60f05bb418f0c9b714833' " + - "and id = 'numOfInfluentialResults'") - .select("value") - .collectAsList() - .get(0) - .getString(0)); - Assertions - .assertEquals( - "1", testDataset - .filter( - "oaid='40|nih_________::c02a8233e9b60f05bb418f0c9b714833' " + - "and id = 'numOfPopularResults'") - .select("value") - .collectAsList() - .get(0) - .getString(0)); - Assertions - .assertEquals( - "25", testDataset - .filter( - "oaid='40|nih_________::c02a8233e9b60f05bb418f0c9b714833' " + - "and id = 'totalImpulse'") - .select("value") - .collectAsList() - .get(0) - .getString(0)); - Assertions - .assertEquals( - "43", testDataset - .filter( - "oaid='40|nih_________::c02a8233e9b60f05bb418f0c9b714833' " + - "and id = 'totalCitationCount'") - .select("value") - .collectAsList() - .get(0) - .getString(0)); - } -}