Merge pull request 'FOS ActionSet for the classification of results without a doi' (#397) from FOSNew into beta

Reviewed-on: #397
pull/414/head
Claudio Atzori 1 month ago
commit e9eb590f87

@ -75,6 +75,7 @@ public class GetFOSSparkJob implements Serializable {
fosData.map((MapFunction<Row, FOSDataModel>) 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));

@ -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,61 +54,34 @@ 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,
outputPath);
if (distributeDOI)
distributeFOSdois(
spark,
sourcePath,
outputPath);
else
distributeFOSoaid(spark, sourcePath, outputPath);
});
}
private static void distributeFOSdois(SparkSession spark, String sourcePath, String outputPath) {
private static void distributeFOSoaid(SparkSession spark, String sourcePath, String outputPath) {
Dataset<FOSDataModel> fosDataset = readPath(spark, sourcePath, FOSDataModel.class);
fosDataset
.groupByKey((MapFunction<FOSDataModel, String>) v -> v.getDoi().toLowerCase(), Encoders.STRING())
.groupByKey((MapFunction<FOSDataModel, String>) v -> v.getOaid().toLowerCase(), Encoders.STRING())
.mapGroups((MapGroupsFunction<String, FOSDataModel, Result>) (k, it) -> {
Result r = new Result();
FOSDataModel first = it.next();
r.setId(DHPUtils.generateUnresolvedIdentifier(k, DOI));
HashSet<String> level1 = new HashSet<>();
HashSet<String> level2 = new HashSet<>();
HashSet<String> level3 = new HashSet<>();
HashSet<String> level4 = new HashSet<>();
addLevels(level1, level2, level3, level4, first);
it.forEachRemaining(v -> addLevels(level1, level2, level3, level4, v));
List<Subject> 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;
return getResult(ModelSupport.getIdPrefix(Result.class) + "|" + k, it);
}, Encoders.bean(Result.class))
.write()
.mode(SaveMode.Overwrite)
@ -114,6 +89,61 @@ public class PrepareFOSSparkJob implements Serializable {
.json(outputPath + "/fos");
}
@NotNull
private static Result getResult(String k, Iterator<FOSDataModel> it) {
Result r = new Result();
FOSDataModel first = it.next();
r.setId(k);
HashSet<String> level1 = new HashSet<>();
HashSet<String> level2 = new HashSet<>();
HashSet<String> level3 = new HashSet<>();
HashSet<String> level4 = new HashSet<>();
addLevels(level1, level2, level3, level4, first);
it.forEachRemaining(v -> addLevels(level1, level2, level3, level4, v));
List<Subject> 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<FOSDataModel> fosDataset = readPath(spark, sourcePath, FOSDataModel.class);
fosDataset
.groupByKey((MapFunction<FOSDataModel, String>) v -> v.getDoi().toLowerCase(), Encoders.STRING())
.mapGroups(
(MapGroupsFunction<String, FOSDataModel, Result>) (k,
it) -> getResult(DHPUtils.generateUnresolvedIdentifier(k, DOI), it),
Encoders.bean(Result.class))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath + "/fos");
}
private static void add(List<Subject> sbsjs, Subject sbj) {
if (sbj != null)
sbsjs.add(sbj);

@ -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<String, Result>) 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);
}
}

@ -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
}
]

@ -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
}
]

@ -0,0 +1,30 @@
<configuration>
<property>
<name>jobTracker</name>
<value>yarnRM</value>
</property>
<property>
<name>nameNode</name>
<value>hdfs://nameservice1</value>
</property>
<property>
<name>oozie.use.system.libpath</name>
<value>true</value>
</property>
<property>
<name>hiveMetastoreUris</name>
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
</property>
<property>
<name>hiveJdbcUrl</name>
<value>jdbc:hive2://iis-cdh5-test-m3.ocean.icm.edu.pl:10000</value>
</property>
<property>
<name>hiveDbName</name>
<value>openaire</value>
</property>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>

@ -0,0 +1,153 @@
<workflow-app name="FOS no doi" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>fosPath</name>
<description>the input path of the resources to be extended</description>
</property>
<property>
<name>outputPath</name>
<description>the path where to store the actionset</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
<property>
<name>oozieActionShareLibForSpark2</name>
<description>oozie action sharelib for spark 2.*</description>
</property>
<property>
<name>spark2ExtraListeners</name>
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
<description>spark 2.* extra listeners classname</description>
</property>
<property>
<name>spark2SqlQueryExecutionListeners</name>
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
<description>spark 2.* sql query execution listeners classname</description>
</property>
<property>
<name>spark2YarnHistoryServerAddress</name>
<description>spark 2.* yarn history server address</description>
</property>
<property>
<name>spark2EventLogDir</name>
<description>spark 2.* event log dir location</description>
</property>
</parameters>
<global>
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<configuration>
<property>
<name>mapreduce.job.queuename</name>
<value>${queueName}</value>
</property>
<property>
<name>oozie.launcher.mapred.job.queue.name</name>
<value>${oozieLauncherQueueName}</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>${oozieActionShareLibForSpark2}</value>
</property>
</configuration>
</global>
<start to="getFOS"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="getFOS">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Gets Data from FOS csv file</name>
<class>eu.dnetlib.dhp.actionmanager.createunresolvedentities.GetFOSSparkJob</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--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}
</spark-opts>
<arg>--sourcePath</arg><arg>${fosPath}</arg>
<arg>--outputPath</arg><arg>${workingDir}/input/fos</arg>
<arg>--delimiter</arg><arg>${delimiter}</arg>
</spark>
<ok to="prepareFos"/>
<error to="Kill"/>
</action>
<action name="prepareFos">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Produces the results from FOS</name>
<class>eu.dnetlib.dhp.actionmanager.createunresolvedentities.PrepareFOSSparkJob</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--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}
</spark-opts>
<arg>--sourcePath</arg><arg>${workingDir}/input/fos</arg>
<arg>--outputPath</arg><arg>${workingDir}/prepared</arg>
<arg>--distributeDoi</arg><arg>false</arg>
</spark>
<ok to="produceActionSet"/>
<error to="Kill"/>
</action>
<action name="produceActionSet">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Save the action set grouping results with the same id</name>
<class>eu.dnetlib.dhp.actionmanager.fosnodoi.CreateActionSetSparkJob</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--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}
</spark-opts>
<arg>--sourcePath</arg><arg>${workingDir}/prepared/fos</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

@ -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<FOSDataModel> 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)));
}
}

@ -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<Result> 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)));
}
}

@ -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
1 DOI OAID level1 level2 level3 level4 score_for_L3 score_for_L4
2 N/A 78975075580c::e680668c98366c9cd6349afc62486a7f 03 medical and health sciences 0301 basic medicine 030104 developmental biology N/A 0.5 0.0
3 N/A 78975075580c::e680668c98366c9cd6349afc62486a7f 03 medical and health sciences 0303 health sciences 030304 developmental biology N/A 0.5 0.0
4 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
5 N/A od______2806::a1da9d2678b12969a9ab5f50b5e71d0a 05 social sciences 0501 psychology and cognitive sciences 050105 experimental psychology 05010501 Emotion/Affective science 0.44109055399894714 0.4833236634731293
6 N/A doajarticles::76535d77fd2a5fe9810aefafffb8ef6c 05 social sciences 0502 economics and business 050203 business & management 05020302 Supply chain management/Business terms 0.5459638833999634 0.5460261106491089
7 N/A doajarticles::76535d77fd2a5fe9810aefafffb8ef6c 05 social sciences 0502 economics and business 050211 marketing 05021102 Services marketing/Retailing 0.4540362060070038 0.4539738595485687
8 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
9 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
10 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
11 N/A od______2806::4b9a664dd6b8b04204cb613e7bc9c873 03 medical and health sciences 0302 clinical medicine 030204 cardiovascular system & hematology N/A 0.49318668246269226 0.0
12 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
13 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
14 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
15 N/A od______3978::6704dcced0fe3dd6fbf985dc2507f61c 03 medical and health sciences 0301 basic medicine 030104 developmental biology N/A 0.48656824231147766 0.0
16 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
17 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
18 N/A od______2806::a938609e9f36ada6629a1bcc50c88230 03 medical and health sciences 0302 clinical medicine 030217 neurology & neurosurgery 03021708 Neurotrauma/Stroke 0.5014800429344177 0.5109656453132629
19 N/A od______2806::a938609e9f36ada6629a1bcc50c88230 02 engineering and technology 0206 medical engineering 020601 biomedical engineering 02060102 Medical terminology/Patient 0.4985199570655823 0.4890343248844147

@ -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"}
Loading…
Cancel
Save