Compare commits

...

79 Commits

Author SHA1 Message Date
Antonis Lempesis cb3adb90f4 Merge branch 'beta' into beta 2021-11-17 14:33:45 +01:00
Claudio Atzori e0395719d7 Merge branch 'beta' of https://code-repo.d4science.org/D-Net/dnet-hadoop into beta 2021-11-17 14:17:27 +01:00
Claudio Atzori 82a4e4efae [cleaning wf] fixed methodology to rule out invalid result titles, based on https://support.openaire.eu/issues/7206 2021-11-17 14:17:22 +01:00
Miriam Baglioni 6d4a1c57ee [Resolve Entities] Change test dataset to mirror the modification in the creation of the map between the pids and the unresolved 2021-11-17 12:41:52 +01:00
Claudio Atzori 49f897ef29 [cleaning wf] fixed regex used to spot garbage in result titles; adjusted threshold for filtering titles 2021-11-16 15:24:23 +01:00
Claudio Atzori 0a727d325d [dedup] increased number of partitions in the consistency phase 2021-11-16 08:43:41 +01:00
Claudio Atzori bafa2990f3 code formatting 2021-11-15 17:07:16 +01:00
Claudio Atzori 668ac25224 [graph resolution] using existing argument parser file name 2021-11-15 17:02:45 +01:00
Claudio Atzori 7d0a03f607 [graph resolution] minor 2021-11-15 14:45:54 +01:00
Claudio Atzori 941a50a2fc Merge branch 'beta' of https://code-repo.d4science.org/D-Net/dnet-hadoop into beta 2021-11-15 14:42:49 +01:00
Claudio Atzori 7c804acda8 [graph resolution] minor 2021-11-15 14:42:43 +01:00
Sandro La Bruzzo efa09057db Merge branch 'beta' of code-repo.d4science.org:D-Net/dnet-hadoop into beta 2021-11-15 14:32:09 +01:00
Sandro La Bruzzo 48923e46a1 added documentation to Pubmed Class and also added mvn site for dhp-aggregations 2021-11-15 14:32:01 +01:00
Claudio Atzori d2c787d416 [graph resolution] fixed sequence of the workflow steps 2021-11-15 14:31:15 +01:00
Claudio Atzori 975b10b711 [actionmanager] increased spark.sql.shuffle.partitions to 5000 2021-11-15 12:31:45 +01:00
Claudio Atzori 1ecceea788 Merge pull request 'Open Citations' (#158) from openCitations into beta
Reviewed-on: D-Net/dnet-hadoop#158
2021-11-15 10:59:19 +01:00
Miriam Baglioni 4ec88c718c merge with beta - resolved conflict in pom 2021-11-15 10:52:16 +01:00
Miriam Baglioni 6f1a434e90 [Bypass Action Set] Fixed test to consider the new identifier utils 2021-11-15 09:59:23 +01:00
Miriam Baglioni 157d33ebf9 [Bypass Action Set] Refactoring 2021-11-15 09:58:48 +01:00
Claudio Atzori 7b81607035 Merge pull request 'PR: Bypass Action Set' (#157) from bypass_acstionset into beta
Reviewed-on: D-Net/dnet-hadoop#157
2021-11-12 12:01:05 +01:00
Miriam Baglioni 92d0e18b55 [Bypass Action Set] used constant DOI instead of "doi" 2021-11-12 10:56:58 +01:00
Miriam Baglioni 881113743f [Bypass Action Set] refactoring 2021-11-12 10:55:50 +01:00
Miriam Baglioni 47ccb53c4f [Bypass Action Set] modification for comment D-Net/dnet-hadoop#157 (comment) 2021-11-12 10:54:09 +01:00
Miriam Baglioni ffb0ce1d59 merge with beta - resolved conflict in pom 2021-11-12 10:19:59 +01:00
Miriam Baglioni 716021546e [Bypass Action Set] minor fix 2021-11-12 10:18:01 +01:00
Claudio Atzori 1f2a3d1af0 depending on dhp-schemas:2.8.22 (release) 2021-11-12 10:15:11 +01:00
Sandro La Bruzzo 3469cc2b1d Merge branch 'beta' of code-repo.d4science.org:D-Net/dnet-hadoop into beta 2021-11-12 09:56:52 +01:00
Sandro La Bruzzo a7763d2492 removed alternate identifier in resolutionMap 2021-11-12 09:56:45 +01:00
Miriam Baglioni 935062edec [Bypass Action Set] creation of unresolved entities 2021-11-11 16:11:25 +01:00
Claudio Atzori 8bdca3413f Merge pull request 'DOIBoost Mapping: change the creation of the instance in the DOIBoost result' (#155) from doiboost_url into beta
Reviewed-on: D-Net/dnet-hadoop#155
2021-11-11 10:40:32 +01:00
Claudio Atzori 148289150f Merge branch 'beta' into doiboost_url 2021-11-11 10:40:19 +01:00
Sandro La Bruzzo 2ca0a436ad added SparkResolveEntities node to the oozie wf 2021-11-11 10:25:42 +01:00
Sandro La Bruzzo 9cb195314f implemented and tested resolution of entities 2021-11-11 10:17:40 +01:00
Miriam Baglioni 6d3c4c4abe mergin with branch beta 2021-11-11 08:59:53 +01:00
Miriam Baglioni c371b23077 - 2021-11-10 17:00:37 +01:00
Miriam Baglioni 9e214ce0eb [BypassAS] addition of OC relations 2021-11-09 12:07:19 +01:00
Sandro La Bruzzo 6477a40670 implement filter of openCitation 2021-11-09 11:27:12 +01:00
Miriam Baglioni 6f7ca539c6 [BypassAS] update of results for bipFinder and FOS 2021-11-09 11:25:41 +01:00
Miriam Baglioni a7d50c499b [BypassAS] prepare FOS subject, test and model for FOS and BipFinder scores 2021-11-08 16:44:19 +01:00
Miriam Baglioni df7ee77c7a [DOIBoost Mapping] removed not needed comments 2021-11-04 16:24:07 +01:00
Miriam Baglioni de63d29b6f [DOIBoost Mapping] Fix to avoid to produce results with null as identifier (probably due to the filtering function in the factory for the creation of the id) 2021-11-04 16:16:40 +01:00
Miriam Baglioni d50057b2d9 [DOIBoost Mapping] changed the way to create the url for the instance: we use the crooref guidelines https://doi.org/doi 2021-11-03 16:59:37 +01:00
Miriam Baglioni edf55395e9 added test resourse 2021-11-03 16:49:30 +01:00
Miriam Baglioni d97ea82a29 [DOIBoost Mapping] Added test to verify the instance created for Crossref will have just the url related to the doi 2021-11-03 16:45:15 +01:00
Miriam Baglioni 96769b4481 [DOIBoost - Mapping] Changed the logic which brought in in the instance urls that should not be there: The urld of the doi in the json is reachable from the root (json/"URL") other urls where added from the links element. Now the mapping from the link element has been removed 2021-11-03 16:43:36 +01:00
Miriam Baglioni 683fe093cf [DOIBoost - Mapping] Remove the addition of the instance to the MAG publication record 2021-11-03 15:51:26 +01:00
Miriam Baglioni b2bb8d9d79 [DOIBoost - Mapping] selecting the url from Crossref containing the doi 2021-11-03 15:44:57 +01:00
Miriam Baglioni 779318961c [DOIBoost - Mapping] removed the url from crossref containing the api.elsevier.com... string in the url 2021-11-03 14:38:52 +01:00
Miriam Baglioni 2480e590d1 [DOIBoost - Mapping] changed the type on which to map dissertation from Crossref: from 006 Doctoral thesis to 0044 Thesis since dissertation could be either Doctoral or master thesis 2021-11-03 14:25:23 +01:00
Sandro La Bruzzo 7bd224f051 implement first version of scholexplorer integration for the generation of final graph 2021-11-02 15:58:15 +01:00
Claudio Atzori 7fa49f6956 Merge pull request 'removed hardcoded reference' (#154) from antonis.lempesis/dnet-hadoop:beta into beta
Reviewed-on: D-Net/dnet-hadoop#154
2021-11-02 09:11:30 +01:00
Claudio Atzori 1225ba0b92 [resolution] increasing number of partitions to avoid OOM 2021-10-28 16:18:17 +02:00
Sandro La Bruzzo d9cbca83f7 moved filter on next phase 2021-10-28 16:13:24 +02:00
Sandro La Bruzzo 1be9aa0a5f Removed filter of datacite items from the raw graph merging phase, Datacite is not an actionset anymore in beta 2021-10-26 17:52:20 +02:00
Sandro La Bruzzo 4acfa8fa2e Scholexplorer Datasource Aggregation:
- Added collectedfrom in the inverse relation generated
Relation resolution:
- increased number of partitions in workflow.xml
- using classid instead of classname to build the pid-dnetId mapping
2021-10-26 17:51:20 +02:00
Sandro La Bruzzo aafdffa6b3 resolved conflict 2021-10-26 09:45:46 +02:00
Sandro La Bruzzo 034304b33a conflict resolved on merge 2021-10-26 09:40:47 +02:00
Claudio Atzori 6b34ba737e minor 2021-10-21 14:16:18 +02:00
Claudio Atzori d147295c2f avoiding java.io.NotSerializableException: java.util.HashMap 2021-10-21 14:15:57 +02:00
Claudio Atzori 3702fe478d cleanup 2021-10-21 12:05:02 +02:00
Sandro La Bruzzo ac36aa7d1c fixed wrong Encoding during a map phase 2021-10-21 11:35:02 +02:00
Sandro La Bruzzo aeeebd573b code refactor renamed datacite package 2021-10-20 17:37:42 +02:00
Sandro La Bruzzo ab3a99d3e9 removed old datacite oozie workflow 2021-10-20 17:19:47 +02:00
Sandro La Bruzzo ae4e99a471 Adapted workflow of resolution of PID to work into OpenAIRE data workflow
- Added relations in both verse on all Scholexplorer datasources
2021-10-20 17:12:16 +02:00
Claudio Atzori 4f8970f8ed [stats] reducing the step22 wait time 2021-10-20 14:14:53 +02:00
Claudio Atzori 00b78b9c58 cleanup: mapping contents in the graph already defined in the OAF graph model doesn't require to be aware of the vocabularies 2021-10-20 14:04:45 +02:00
Claudio Atzori c01dd0c925 registered oaf model classes for the KryoSerializer 2021-10-20 13:55:07 +02:00
Claudio Atzori d0cf2963f0 Merge pull request 'hierarchical_orgs_relations' (#150) from hierarchical_orgs_relations into beta
Reviewed-on: D-Net/dnet-hadoop#150
2021-10-20 10:13:47 +02:00
Claudio Atzori 59f76b50d4 Merge branch 'beta' into hierarchical_orgs_relations 2021-10-20 09:42:35 +02:00
Claudio Atzori bc3372093e Merge pull request '[stats] affiliations in stats and monitor dbs' (#152) from antonis.lempesis/dnet-hadoop:beta into beta
Reviewed-on: D-Net/dnet-hadoop#152
2021-10-20 09:40:34 +02:00
Claudio Atzori 515e068a78 Merge branch 'beta' into hierarchical_orgs_relations 2021-10-19 16:46:06 +02:00
miconis 5f780a6ba1 bug fix in migrate entities: parameter name was wrong 2021-10-18 23:30:40 +02:00
Miriam Baglioni 1cc09adfaa Opencitations: chenaged the test class to mirror the creation or not of duplicate dois for .refs oc original plus added optional parameter to duplicate the relation 2021-10-18 14:11:27 +02:00
miconis 995c1eddaf minor change 2021-10-13 17:07:10 +02:00
miconis 326bf63775 integration of parent child orgs relations 2021-10-13 12:24:48 +02:00
Miriam Baglioni 476a4708d6 mergin with branch beta 2021-09-27 16:02:32 +02:00
Miriam Baglioni 5ec69889db OpenCitations: creation of AS from OC 2021-09-27 16:02:06 +02:00
Miriam Baglioni eedf7c3310 mergin with branch beta 2021-09-22 15:18:34 +02:00
Miriam Baglioni f2118d771a first steps in the implementation of the integration of opencitations 2021-09-22 15:18:05 +02:00
116 changed files with 4771 additions and 795 deletions

View File

@ -28,7 +28,7 @@ public class HdfsSupport {
* @param configuration Configuration of hadoop env
*/
public static boolean exists(String path, Configuration configuration) {
logger.info("Removing path: {}", path);
logger.info("Checking existence for path: {}", path);
return rethrowAsRuntimeException(
() -> {
Path f = new Path(path);

View File

@ -27,8 +27,11 @@ public class GraphCleaningFunctions extends CleaningFunctions {
public static final int ORCID_LEN = 19;
public static final String CLEANING_REGEX = "(?:\\n|\\r|\\t)";
public static final String INVALID_AUTHOR_REGEX = ".*deactivated.*";
public static final String TITLE_FILTER_REGEX = "[.*test.*\\W\\d]";
public static final int TITLE_FILTER_RESIDUAL_LENGTH = 10;
public static final String TITLE_TEST = "test";
public static final String TITLE_FILTER_REGEX = String.format("(%s)|\\W|\\d", TITLE_TEST);
public static final int TITLE_FILTER_RESIDUAL_LENGTH = 5;
public static <T extends Oaf> T fixVocabularyNames(T value) {
if (value instanceof Datasource) {
@ -195,10 +198,16 @@ public class GraphCleaningFunctions extends CleaningFunctions {
final String title = sp
.getValue()
.toLowerCase();
final String residual = Unidecode
.decode(title)
.replaceAll(TITLE_FILTER_REGEX, "");
return residual.length() > TITLE_FILTER_RESIDUAL_LENGTH;
final String decoded = Unidecode.decode(title);
if (StringUtils.contains(decoded, TITLE_TEST)) {
return decoded
.replaceAll(TITLE_FILTER_REGEX, "")
.length() > TITLE_FILTER_RESIDUAL_LENGTH;
}
return !decoded
.replaceAll("\\W|\\d", "")
.isEmpty();
})
.map(GraphCleaningFunctions::cleanValue)
.collect(Collectors.toList()));

View File

@ -4,19 +4,19 @@ package eu.dnetlib.dhp.utils;
import java.io.*;
import java.nio.charset.StandardCharsets;
import java.security.MessageDigest;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.zip.GZIPInputStream;
import java.util.zip.GZIPOutputStream;
import java.util.*;
import java.util.stream.Collectors;
import org.apache.commons.codec.binary.Base64;
import org.apache.commons.codec.binary.Base64OutputStream;
import org.apache.commons.codec.binary.Hex;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.http.client.methods.CloseableHttpResponse;
import org.apache.http.client.methods.HttpGet;
import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.http.impl.client.HttpClients;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.SaveMode;
import org.slf4j.Logger;
@ -26,6 +26,8 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Maps;
import com.jayway.jsonpath.JsonPath;
import eu.dnetlib.dhp.schema.mdstore.MDStoreWithInfo;
import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions;
import net.minidev.json.JSONArray;
import scala.collection.JavaConverters;
import scala.collection.Seq;
@ -52,10 +54,56 @@ public class DHPUtils {
}
}
/**
* Retrieves from the metadata store manager application the list of paths associated with mdstores characterized
* by he given format, layout, interpretation
* @param mdstoreManagerUrl the URL of the mdstore manager service
* @param format the mdstore format
* @param layout the mdstore layout
* @param interpretation the mdstore interpretation
* @param includeEmpty include Empty mdstores
* @return the set of hdfs paths
* @throws IOException in case of HTTP communication issues
*/
public static Set<String> mdstorePaths(final String mdstoreManagerUrl,
final String format,
final String layout,
final String interpretation,
boolean includeEmpty) throws IOException {
final String url = mdstoreManagerUrl + "/mdstores/";
final ObjectMapper objectMapper = new ObjectMapper();
final HttpGet req = new HttpGet(url);
try (final CloseableHttpClient client = HttpClients.createDefault()) {
try (final CloseableHttpResponse response = client.execute(req)) {
final String json = IOUtils.toString(response.getEntity().getContent());
final MDStoreWithInfo[] mdstores = objectMapper.readValue(json, MDStoreWithInfo[].class);
return Arrays
.stream(mdstores)
.filter(md -> md.getFormat().equalsIgnoreCase(format))
.filter(md -> md.getLayout().equalsIgnoreCase(layout))
.filter(md -> md.getInterpretation().equalsIgnoreCase(interpretation))
.filter(md -> StringUtils.isNotBlank(md.getHdfsPath()))
.filter(md -> StringUtils.isNotBlank(md.getCurrentVersion()))
.filter(md -> includeEmpty || md.getSize() > 0)
.map(md -> md.getHdfsPath() + "/" + md.getCurrentVersion() + "/store")
.collect(Collectors.toSet());
}
}
}
public static String generateIdentifier(final String originalId, final String nsPrefix) {
return String.format("%s::%s", nsPrefix, DHPUtils.md5(originalId));
}
public static String generateUnresolvedIdentifier(final String pid, final String pidType) {
final String cleanedPid = CleaningFunctions.normalizePidValue(pidType, pid);
return String.format("unresolved::%s::%s", cleanedPid, pidType.toLowerCase().trim());
}
public static String getJPathString(final String jsonPath, final String json) {
try {
Object o = JsonPath.read(json, jsonPath);

View File

@ -107,7 +107,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=2560
--conf spark.sql.shuffle.partitions=5000
</spark-opts>
<arg>--inputGraphTablePath</arg><arg>${inputGraphRootPath}/publication</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
@ -159,7 +159,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=2560
--conf spark.sql.shuffle.partitions=5000
</spark-opts>
<arg>--inputGraphTablePath</arg><arg>${workingDir}/publication</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>

View File

@ -99,7 +99,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=2560
--conf spark.sql.shuffle.partitions=5000
</spark-opts>
<arg>--inputGraphTablePath</arg><arg>${inputGraphRootPath}/relation</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Relation</arg>

View File

@ -29,6 +29,13 @@
<goal>testCompile</goal>
</goals>
</execution>
<execution>
<id>scala-doc</id>
<phase>process-resources</phase> <!-- or wherever -->
<goals>
<goal>doc</goal>
</goals>
</execution>
</executions>
<configuration>
<scalaVersion>${scala.version}</scalaVersion>

View File

@ -0,0 +1,49 @@
package eu.dnetlib.dhp.actionmanager.createunresolvedentities;
import java.util.Optional;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SparkSession;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
public class Constants {
public static final String DOI = "doi";
public static final String UPDATE_DATA_INFO_TYPE = "update";
public static final String UPDATE_SUBJECT_FOS_CLASS_ID = "subject:fos";
public static final String UPDATE_CLASS_NAME = "Inferred by OpenAIRE";
public static final String UPDATE_MEASURE_BIP_CLASS_ID = "measure:bip";
public static final String FOS_CLASS_ID = "FOS";
public static final String FOS_CLASS_NAME = "Fields of Science and Technology classification";
public static final String NULL = "NULL";
public static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private Constants() {
}
public static Boolean isSparkSessionManaged(ArgumentApplicationParser parser) {
return Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
}
public static <R> Dataset<R> readPath(
SparkSession spark, String inputPath, Class<R> clazz) {
return spark
.read()
.textFile(inputPath)
.map((MapFunction<String, R>) value -> OBJECT_MAPPER.readValue(value, clazz), Encoders.bean(clazz));
}
}

View File

@ -0,0 +1,77 @@
package eu.dnetlib.dhp.actionmanager.createunresolvedentities;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.Serializable;
import java.util.Objects;
import java.util.Optional;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.collection.GetCSV;
public class GetFOSData implements Serializable {
private static final Logger log = LoggerFactory.getLogger(GetFOSData.class);
public static final char DEFAULT_DELIMITER = '\t';
public static void main(final String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
Objects
.requireNonNull(
GetFOSData.class
.getResourceAsStream(
"/eu/dnetlib/dhp/actionmanager/createunresolvedentities/get_fos_parameters.json"))));
parser.parseArgument(args);
// the path where the original fos csv file is stored
final String sourcePath = parser.get("sourcePath");
log.info("sourcePath {}", sourcePath);
// the path where to put the file as json
final String outputPath = parser.get("outputPath");
log.info("outputPath {}", outputPath);
final String hdfsNameNode = parser.get("hdfsNameNode");
log.info("hdfsNameNode {}", hdfsNameNode);
final String classForName = parser.get("classForName");
log.info("classForName {}", classForName);
final char delimiter = Optional
.ofNullable(parser.get("delimiter"))
.map(s -> s.charAt(0))
.orElse(DEFAULT_DELIMITER);
log.info("delimiter {}", delimiter);
Configuration conf = new Configuration();
conf.set("fs.defaultFS", hdfsNameNode);
FileSystem fileSystem = FileSystem.get(conf);
new GetFOSData().doRewrite(sourcePath, outputPath, classForName, delimiter, fileSystem);
}
public void doRewrite(String inputPath, String outputFile, String classForName, char delimiter, FileSystem fs)
throws IOException, ClassNotFoundException {
// reads the csv and writes it as its json equivalent
try (InputStreamReader reader = new InputStreamReader(fs.open(new Path(inputPath)))) {
GetCSV.getCsv(fs, reader, outputFile, classForName, delimiter);
}
}
}

View File

@ -0,0 +1,145 @@
package eu.dnetlib.dhp.actionmanager.createunresolvedentities;
import static eu.dnetlib.dhp.actionmanager.createunresolvedentities.Constants.*;
import static eu.dnetlib.dhp.actionmanager.createunresolvedentities.Constants.UPDATE_CLASS_NAME;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.Serializable;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.hdfs.client.HdfsUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.model.BipDeserialize;
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.model.BipScore;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.KeyValue;
import eu.dnetlib.dhp.schema.oaf.Measure;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
import eu.dnetlib.dhp.utils.DHPUtils;
public class PrepareBipFinder implements Serializable {
private static final Logger log = LoggerFactory.getLogger(PrepareBipFinder.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
public static <I extends Result> void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils
.toString(
PrepareBipFinder.class
.getResourceAsStream(
"/eu/dnetlib/dhp/actionmanager/createunresolvedentities/prepare_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String sourcePath = parser.get("sourcePath");
log.info("sourcePath {}: ", sourcePath);
final String outputPath = parser.get("outputPath");
log.info("outputPath {}: ", outputPath);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
HdfsSupport.remove(outputPath, spark.sparkContext().hadoopConfiguration());
prepareResults(spark, sourcePath, outputPath);
});
}
private static <I extends Result> void prepareResults(SparkSession spark, String inputPath, String outputPath) {
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<BipDeserialize> bipDeserializeJavaRDD = sc
.textFile(inputPath)
.map(item -> OBJECT_MAPPER.readValue(item, BipDeserialize.class));
spark
.createDataset(bipDeserializeJavaRDD.flatMap(entry -> entry.keySet().stream().map(key -> {
BipScore bs = new BipScore();
bs.setId(key);
bs.setScoreList(entry.get(key));
return bs;
}).collect(Collectors.toList()).iterator()).rdd(), Encoders.bean(BipScore.class))
.map((MapFunction<BipScore, Result>) v -> {
Result r = new Result();
r.setId(DHPUtils.generateUnresolvedIdentifier(v.getId(), DOI));
r.setMeasures(getMeasure(v));
return r;
}, Encoders.bean(Result.class))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath + "/bip");
}
private static List<Measure> getMeasure(BipScore value) {
return value
.getScoreList()
.stream()
.map(score -> {
Measure m = new Measure();
m.setId(score.getId());
m
.setUnit(
score
.getUnit()
.stream()
.map(unit -> {
KeyValue kv = new KeyValue();
kv.setValue(unit.getValue());
kv.setKey(unit.getKey());
kv
.setDataInfo(
OafMapperUtils
.dataInfo(
false,
UPDATE_DATA_INFO_TYPE,
true,
false,
OafMapperUtils
.qualifier(
UPDATE_MEASURE_BIP_CLASS_ID,
UPDATE_CLASS_NAME,
ModelConstants.DNET_PROVENANCE_ACTIONS,
ModelConstants.DNET_PROVENANCE_ACTIONS),
""));
return kv;
})
.collect(Collectors.toList()));
return m;
})
.collect(Collectors.toList());
}
}

View File

@ -0,0 +1,133 @@
package eu.dnetlib.dhp.actionmanager.createunresolvedentities;
import static eu.dnetlib.dhp.actionmanager.createunresolvedentities.Constants.*;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.Serializable;
import java.util.*;
import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.model.FOSDataModel;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
import eu.dnetlib.dhp.utils.DHPUtils;
public class PrepareFOSSparkJob implements Serializable {
private static final Logger log = LoggerFactory.getLogger(PrepareFOSSparkJob.class);
public static void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils
.toString(
PrepareFOSSparkJob.class
.getResourceAsStream(
"/eu/dnetlib/dhp/actionmanager/createunresolvedentities/prepare_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
Boolean isSparkSessionManaged = isSparkSessionManaged(parser);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
String sourcePath = parser.get("sourcePath");
log.info("sourcePath: {}", sourcePath);
final String outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
distributeFOSdois(
spark,
sourcePath,
outputPath);
});
}
private static void distributeFOSdois(SparkSession spark, String sourcePath, String outputPath) {
Dataset<FOSDataModel> fosDataset = readPath(spark, sourcePath, FOSDataModel.class);
fosDataset.flatMap((FlatMapFunction<FOSDataModel, FOSDataModel>) v -> {
List<FOSDataModel> fosList = new ArrayList<>();
final String level1 = v.getLevel1();
final String level2 = v.getLevel2();
final String level3 = v.getLevel3();
Arrays
.stream(v.getDoi().split("\u0002"))
.forEach(d -> fosList.add(FOSDataModel.newInstance(d, level1, level2, level3)));
return fosList.iterator();
}, Encoders.bean(FOSDataModel.class))
.map((MapFunction<FOSDataModel, Result>) value -> {
Result r = new Result();
r.setId(DHPUtils.generateUnresolvedIdentifier(value.getDoi(), DOI));
r.setSubject(getSubjects(value));
return r;
}, Encoders.bean(Result.class))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath + "/fos");
}
private static List<StructuredProperty> getSubjects(FOSDataModel fos) {
return Arrays
.asList(getSubject(fos.getLevel1()), getSubject(fos.getLevel2()), getSubject(fos.getLevel3()))
.stream()
.filter(Objects::nonNull)
.collect(Collectors.toList());
}
private static StructuredProperty getSubject(String sbj) {
if (sbj.equals(NULL))
return null;
StructuredProperty sp = new StructuredProperty();
sp.setValue(sbj);
sp
.setQualifier(
OafMapperUtils
.qualifier(
FOS_CLASS_ID,
FOS_CLASS_NAME,
ModelConstants.DNET_SUBJECT_TYPOLOGIES,
ModelConstants.DNET_SUBJECT_TYPOLOGIES));
sp
.setDataInfo(
OafMapperUtils
.dataInfo(
false,
UPDATE_DATA_INFO_TYPE,
true,
false,
OafMapperUtils
.qualifier(
UPDATE_SUBJECT_FOS_CLASS_ID,
UPDATE_CLASS_NAME,
ModelConstants.DNET_PROVENANCE_ACTIONS,
ModelConstants.DNET_PROVENANCE_ACTIONS),
""));
return sp;
}
}

View File

@ -0,0 +1,79 @@
package eu.dnetlib.dhp.actionmanager.createunresolvedentities;
import static eu.dnetlib.dhp.actionmanager.createunresolvedentities.Constants.*;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.Serializable;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.MapGroupsFunction;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.oaf.Result;
public class SparkSaveUnresolved implements Serializable {
private static final Logger log = LoggerFactory.getLogger(PrepareFOSSparkJob.class);
public static void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils
.toString(
PrepareFOSSparkJob.class
.getResourceAsStream(
"/eu/dnetlib/dhp/actionmanager/createunresolvedentities/produce_unresolved_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
Boolean isSparkSessionManaged = isSparkSessionManaged(parser);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
String sourcePath = parser.get("sourcePath");
log.info("sourcePath: {}", sourcePath);
final String outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
saveUnresolved(
spark,
sourcePath,
outputPath);
});
}
private static void saveUnresolved(SparkSession spark, String sourcePath, String outputPath) {
spark
.read()
.textFile(sourcePath + "/*")
.map(
(MapFunction<String, Result>) l -> OBJECT_MAPPER.readValue(l, Result.class),
Encoders.bean(Result.class))
.groupByKey((MapFunction<Result, String>) r -> r.getId(), Encoders.STRING())
.mapGroups((MapGroupsFunction<String, Result, Result>) (k, it) -> {
Result ret = it.next();
it.forEachRemaining(r -> ret.mergeFrom(r));
return ret;
}, Encoders.bean(Result.class))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath);
}
}

View File

@ -0,0 +1,28 @@
package eu.dnetlib.dhp.actionmanager.createunresolvedentities.model;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
/**
* Class that maps the model of the bipFinder! input data.
* Only needed for deserialization purposes
*/
public class BipDeserialize extends HashMap<String, List<Score>> implements Serializable {
public BipDeserialize() {
super();
}
public List<Score> get(String key) {
if (super.get(key) == null) {
return new ArrayList<>();
}
return super.get(key);
}
}

View File

@ -0,0 +1,30 @@
package eu.dnetlib.dhp.actionmanager.createunresolvedentities.model;
import java.io.Serializable;
import java.util.List;
/**
* Rewriting of the bipFinder input data by extracting the identifier of the result (doi)
*/
public class BipScore implements Serializable {
private String id; // doi
private List<Score> scoreList; // unit as given in the inputfile
public String getId() {
return id;
}
public void setId(String id) {
this.id = id;
}
public List<Score> getScoreList() {
return scoreList;
}
public void setScoreList(List<Score> scoreList) {
this.scoreList = scoreList;
}
}

View File

@ -0,0 +1,71 @@
package eu.dnetlib.dhp.actionmanager.createunresolvedentities.model;
import java.io.Serializable;
import com.opencsv.bean.CsvBindByPosition;
public class FOSDataModel implements Serializable {
@CsvBindByPosition(position = 1)
// @CsvBindByName(column = "doi")
private String doi;
@CsvBindByPosition(position = 2)
// @CsvBindByName(column = "level1")
private String level1;
@CsvBindByPosition(position = 3)
// @CsvBindByName(column = "level2")
private String level2;
@CsvBindByPosition(position = 4)
// @CsvBindByName(column = "level3")
private String level3;
public FOSDataModel() {
}
public FOSDataModel(String doi, String level1, String level2, String level3) {
this.doi = doi;
this.level1 = level1;
this.level2 = level2;
this.level3 = level3;
}
public static FOSDataModel newInstance(String d, String level1, String level2, String level3) {
return new FOSDataModel(d, level1, level2, level3);
}
public String getDoi() {
return doi;
}
public void setDoi(String doi) {
this.doi = doi;
}
public String getLevel1() {
return level1;
}
public void setLevel1(String level1) {
this.level1 = level1;
}
public String getLevel2() {
return level2;
}
public void setLevel2(String level2) {
this.level2 = level2;
}
public String getLevel3() {
return level3;
}
public void setLevel3(String level3) {
this.level3 = level3;
}
}

View File

@ -0,0 +1,26 @@
package eu.dnetlib.dhp.actionmanager.createunresolvedentities.model;
import java.io.Serializable;
public class KeyValue implements Serializable {
private String key;
private String value;
public String getKey() {
return key;
}
public void setKey(String key) {
this.key = key;
}
public String getValue() {
return value;
}
public void setValue(String value) {
this.value = value;
}
}

View File

@ -0,0 +1,30 @@
package eu.dnetlib.dhp.actionmanager.createunresolvedentities.model;
import java.io.Serializable;
import java.util.List;
/**
* represents the score in the input file
*/
public class Score implements Serializable {
private String id;
private List<KeyValue> unit;
public String getId() {
return id;
}
public void setId(String id) {
this.id = id;
}
public List<KeyValue> getUnit() {
return unit;
}
public void setUnit(List<KeyValue> unit) {
this.unit = unit;
}
}

View File

@ -1,41 +0,0 @@
package eu.dnetlib.dhp.actionmanager.datacite
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.schema.oaf.Oaf
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.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
import org.slf4j.{Logger, LoggerFactory}
import scala.io.Source
object ExportActionSetJobNode {
val log: Logger = LoggerFactory.getLogger(ExportActionSetJobNode.getClass)
def main(args: Array[String]): Unit = {
val conf = new SparkConf
val parser = new ArgumentApplicationParser(Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/actionmanager/datacite/exportDataset_parameters.json")).mkString)
parser.parseArgument(args)
val master = parser.get("master")
val sourcePath = parser.get("sourcePath")
val targetPath = parser.get("targetPath")
val spark: SparkSession = SparkSession.builder().config(conf)
.appName(ExportActionSetJobNode.getClass.getSimpleName)
.master(master)
.getOrCreate()
implicit val resEncoder: Encoder[Oaf] = Encoders.kryo[Oaf]
implicit val tEncoder:Encoder[(String,String)] = Encoders.tuple(Encoders.STRING,Encoders.STRING)
spark.read.load(sourcePath).as[Oaf]
.map(o =>DataciteToOAFTransformation.toActionSet(o))
.filter(o => o!= null)
.rdd.map(s => (new Text(s._1), new Text(s._2))).saveAsHadoopFile(s"$targetPath", classOf[Text], classOf[Text], classOf[SequenceFileOutputFormat[Text,Text]], classOf[GzipCodec])
}
}

View File

@ -1,46 +0,0 @@
package eu.dnetlib.dhp.actionmanager.datacite
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup
import eu.dnetlib.dhp.schema.mdstore.MetadataRecord
import eu.dnetlib.dhp.schema.oaf.{Oaf, Result}
import eu.dnetlib.dhp.utils.ISLookupClientFactory
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
import org.slf4j.{Logger, LoggerFactory}
import scala.io.Source
object FilterCrossrefEntitiesSpark {
val log: Logger = LoggerFactory.getLogger(getClass.getClass)
def main(args: Array[String]): Unit = {
val conf = new SparkConf
val parser = new ArgumentApplicationParser(Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/actionmanager/datacite/filter_crossref_param.json")).mkString)
parser.parseArgument(args)
val master = parser.get("master")
val sourcePath = parser.get("sourcePath")
log.info("sourcePath: {}", sourcePath)
val targetPath = parser.get("targetPath")
log.info("targetPath: {}", targetPath)
val spark: SparkSession = SparkSession.builder().config(conf)
.appName(getClass.getSimpleName)
.master(master)
.getOrCreate()
implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo[Oaf]
implicit val resEncoder: Encoder[Result] = Encoders.kryo[Result]
val d:Dataset[Oaf]= spark.read.load(sourcePath).as[Oaf]
d.filter(r => r.isInstanceOf[Result]).map(r => r.asInstanceOf[Result]).write.mode(SaveMode.Overwrite).save(targetPath)
}
}

View File

@ -0,0 +1,181 @@
package eu.dnetlib.dhp.actionmanager.opencitations;
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.mapred.SequenceFileOutputFormat;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.FlatMapFunction;
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.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.action.AtomicAction;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions;
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory;
import scala.Tuple2;
public class CreateActionSetSparkJob implements Serializable {
public static final String OPENCITATIONS_CLASSID = "sysimport:crosswalk:opencitations";
public static final String OPENCITATIONS_CLASSNAME = "Imported from OpenCitations";
private static final String ID_PREFIX = "50|doi_________::";
private static final String TRUST = "0.91";
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/opencitations/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("inputPath");
log.info("inputPath {}", inputPath.toString());
final String outputPath = parser.get("outputPath");
log.info("outputPath {}", outputPath);
final boolean shouldDuplicateRels = Optional
.ofNullable(parser.get("shouldDuplicateRels"))
.map(Boolean::valueOf)
.orElse(Boolean.FALSE);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
extractContent(spark, inputPath, outputPath, shouldDuplicateRels);
});
}
private static void extractContent(SparkSession spark, String inputPath, String outputPath,
boolean shouldDuplicateRels) {
spark
.sqlContext()
.createDataset(spark.sparkContext().textFile(inputPath + "/*", 6000), Encoders.STRING())
.flatMap(
(FlatMapFunction<String, Relation>) value -> createRelation(value, shouldDuplicateRels).iterator(),
Encoders.bean(Relation.class))
.filter((FilterFunction<Relation>) value -> value != null)
.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);
}
private static List<Relation> createRelation(String value, boolean duplicate) {
String[] line = value.split(",");
if (!line[1].startsWith("10.")) {
return new ArrayList<>();
}
List<Relation> relationList = new ArrayList<>();
String citing = ID_PREFIX + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", line[1]));
final String cited = ID_PREFIX + IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", line[2]));
relationList
.addAll(
getRelations(
citing,
cited));
if (duplicate && line[1].endsWith(".refs")) {
citing = ID_PREFIX + IdentifierFactory
.md5(CleaningFunctions.normalizePidValue("doi", line[1].substring(0, line[1].indexOf(".refs"))));
relationList.addAll(getRelations(citing, cited));
}
return relationList;
}
private static Collection<Relation> getRelations(String citing, String cited) {
return Arrays
.asList(
getRelation(citing, cited, ModelConstants.CITES),
getRelation(cited, citing, ModelConstants.IS_CITED_BY));
}
public static Relation getRelation(
String source,
String target,
String relclass) {
Relation r = new Relation();
r.setCollectedfrom(getCollectedFrom());
r.setSource(source);
r.setTarget(target);
r.setRelClass(relclass);
r.setRelType(ModelConstants.RESULT_RESULT);
r.setSubRelType(ModelConstants.CITATION);
r
.setDataInfo(
getDataInfo());
return r;
}
public static List<KeyValue> getCollectedFrom() {
KeyValue kv = new KeyValue();
kv.setKey(ModelConstants.OPENOCITATIONS_ID);
kv.setValue(ModelConstants.OPENOCITATIONS_NAME);
return Arrays.asList(kv);
}
public static DataInfo getDataInfo() {
DataInfo di = new DataInfo();
di.setInferred(false);
di.setDeletedbyinference(false);
di.setTrust(TRUST);
di
.setProvenanceaction(
getQualifier(OPENCITATIONS_CLASSID, OPENCITATIONS_CLASSNAME, ModelConstants.DNET_PROVENANCE_ACTIONS));
return di;
}
public static Qualifier getQualifier(String class_id, String class_name,
String qualifierSchema) {
Qualifier pa = new Qualifier();
pa.setClassid(class_id);
pa.setClassname(class_name);
pa.setSchemeid(qualifierSchema);
pa.setSchemename(qualifierSchema);
return pa;
}
}

View File

@ -0,0 +1,93 @@
package eu.dnetlib.dhp.actionmanager.opencitations;
import java.io.*;
import java.io.Serializable;
import java.util.Objects;
import java.util.zip.GZIPOutputStream;
import java.util.zip.ZipEntry;
import java.util.zip.ZipInputStream;
import org.apache.commons.cli.ParseException;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
public class GetOpenCitationsRefs implements Serializable {
private static final Logger log = LoggerFactory.getLogger(GetOpenCitationsRefs.class);
public static void main(final String[] args) throws IOException, ParseException {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
Objects
.requireNonNull(
GetOpenCitationsRefs.class
.getResourceAsStream(
"/eu/dnetlib/dhp/actionmanager/opencitations/input_parameters.json"))));
parser.parseArgument(args);
final String[] inputFile = parser.get("inputFile").split(";");
log.info("inputFile {}", inputFile.toString());
final String workingPath = parser.get("workingPath");
log.info("workingPath {}", workingPath);
final String hdfsNameNode = parser.get("hdfsNameNode");
log.info("hdfsNameNode {}", hdfsNameNode);
Configuration conf = new Configuration();
conf.set("fs.defaultFS", hdfsNameNode);
FileSystem fileSystem = FileSystem.get(conf);
GetOpenCitationsRefs ocr = new GetOpenCitationsRefs();
for (String file : inputFile) {
ocr.doExtract(workingPath + "/Original/" + file, workingPath, fileSystem);
}
}
private void doExtract(String inputFile, String workingPath, FileSystem fileSystem)
throws IOException {
final Path path = new Path(inputFile);
FSDataInputStream oc_zip = fileSystem.open(path);
int count = 1;
try (ZipInputStream zis = new ZipInputStream(oc_zip)) {
ZipEntry entry = null;
while ((entry = zis.getNextEntry()) != null) {
if (!entry.isDirectory()) {
String fileName = entry.getName();
fileName = fileName.substring(0, fileName.indexOf("T")) + "_" + count;
count++;
try (
FSDataOutputStream out = fileSystem
.create(new Path(workingPath + "/COCI/" + fileName + ".gz"));
GZIPOutputStream gzipOs = new GZIPOutputStream(new BufferedOutputStream(out))) {
IOUtils.copy(zis, gzipOs);
}
}
}
}
}
}

View File

@ -0,0 +1,49 @@
package eu.dnetlib.dhp.collection
import eu.dnetlib.dhp.schema.common.ModelSupport
import eu.dnetlib.dhp.schema.oaf.{Oaf, OafEntity, Relation}
object CollectionUtils {
/**
* This method in pipeline to the transformation phase,
* generates relations in both verse, typically it should be a phase of flatMap
*
* @param i input OAF
* @return
* If the input OAF is an entity -> List(i)
* If the input OAF is a relation -> List(relation, inverseRelation)
*
*/
def fixRelations(i: Oaf): List[Oaf] = {
if (i.isInstanceOf[OafEntity])
return List(i)
else {
val r: Relation = i.asInstanceOf[Relation]
val currentRel = ModelSupport.findRelation(r.getRelClass)
if (currentRel != null) {
// Cleaning relation
r.setRelType(currentRel.getRelType)
r.setSubRelType(currentRel.getSubReltype)
r.setRelClass(currentRel.getRelClass)
val inverse = new Relation
inverse.setSource(r.getTarget)
inverse.setTarget(r.getSource)
inverse.setRelType(currentRel.getRelType)
inverse.setSubRelType(currentRel.getSubReltype)
inverse.setRelClass(currentRel.getInverseRelClass)
inverse.setCollectedfrom(r.getCollectedfrom)
inverse.setDataInfo(r.getDataInfo)
inverse.setProperties(r.getProperties)
inverse.setLastupdatetimestamp(r.getLastupdatetimestamp)
inverse.setValidated(r.getValidated)
inverse.setValidationDate(r.getValidationDate)
return List(r, inverse)
}
}
List()
}
}

View File

@ -1,12 +1,10 @@
package eu.dnetlib.dhp.actionmanager.datacite
package eu.dnetlib.dhp.datacite
import org.apache.commons.io.IOUtils
import org.apache.http.client.config.RequestConfig
import org.apache.http.client.methods.{HttpGet, HttpPost, HttpRequestBase, HttpUriRequest}
import org.apache.http.client.methods.{HttpGet, HttpPost, HttpUriRequest}
import org.apache.http.entity.StringEntity
import org.apache.http.impl.client.{HttpClientBuilder, HttpClients}
import java.io.IOException
import org.apache.http.impl.client.HttpClientBuilder
abstract class AbstractRestClient extends Iterator[String] {

View File

@ -1,7 +1,7 @@
package eu.dnetlib.dhp.actionmanager.datacite
package eu.dnetlib.dhp.datacite
import org.json4s.{DefaultFormats, JValue}
import org.json4s.jackson.JsonMethods.{compact, parse, render}
import org.json4s.{DefaultFormats, JValue}
class DataciteAPIImporter(timestamp: Long = 0, blocks: Long = 10, until:Long = -1) extends AbstractRestClient {

View File

@ -1,4 +1,4 @@
package eu.dnetlib.dhp.actionmanager.datacite
package eu.dnetlib.dhp.datacite
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup
@ -325,8 +325,9 @@ object DataciteToOAFTransformation {
val grantId = m.matcher(awardUri).replaceAll("$2")
val targetId = s"$p${DHPUtils.md5(grantId)}"
List(
generateRelation(sourceId, targetId, "isProducedBy", DATACITE_COLLECTED_FROM, dataInfo),
generateRelation(targetId, sourceId, "produces", DATACITE_COLLECTED_FROM, dataInfo)
generateRelation(sourceId, targetId, "isProducedBy", DATACITE_COLLECTED_FROM, dataInfo)
// REMOVED INVERSE RELATION since there is a specific method that should generate later
// generateRelation(targetId, sourceId, "produces", DATACITE_COLLECTED_FROM, dataInfo)
)
}
else
@ -580,11 +581,11 @@ object DataciteToOAFTransformation {
rel.setProperties(List(dateProps).asJava)
rel.setSource(id)
rel.setTarget(s"unresolved::${r.relatedIdentifier}::${r.relatedIdentifierType}")
rel.setTarget(DHPUtils.generateUnresolvedIdentifier(r.relatedIdentifier,r.relatedIdentifierType))
rel.setCollectedfrom(List(DATACITE_COLLECTED_FROM).asJava)
rel.getCollectedfrom.asScala.map(c => c.getValue)(collection.breakOut)
rel.getCollectedfrom.asScala.map(c => c.getValue).toList
rel
})(collection breakOut)
}).toList
}
def generateDataInfo(trust: String): DataInfo = {

View File

@ -1,9 +1,14 @@
package eu.dnetlib.dhp.actionmanager.datacite
package eu.dnetlib.dhp.datacite
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.collection.CollectionUtils.fixRelations
import eu.dnetlib.dhp.common.Constants.MDSTORE_DATA_PATH
import eu.dnetlib.dhp.common.Constants.MDSTORE_SIZE_PATH
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup
import eu.dnetlib.dhp.schema.mdstore.MetadataRecord
import eu.dnetlib.dhp.schema.mdstore.{MDStoreVersion, MetadataRecord}
import eu.dnetlib.dhp.schema.oaf.Oaf
import eu.dnetlib.dhp.utils.DHPUtils.writeHdfsFile
import eu.dnetlib.dhp.utils.ISLookupClientFactory
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}
@ -17,11 +22,10 @@ object GenerateDataciteDatasetSpark {
def main(args: Array[String]): Unit = {
val conf = new SparkConf
val parser = new ArgumentApplicationParser(Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/actionmanager/datacite/generate_dataset_params.json")).mkString)
val parser = new ArgumentApplicationParser(Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/datacite/generate_dataset_params.json")).mkString)
parser.parseArgument(args)
val master = parser.get("master")
val sourcePath = parser.get("sourcePath")
val targetPath = parser.get("targetPath")
val exportLinks = "true".equalsIgnoreCase(parser.get("exportLinks"))
val isLookupUrl: String = parser.get("isLookupUrl")
log.info("isLookupUrl: {}", isLookupUrl)
@ -33,16 +37,28 @@ object GenerateDataciteDatasetSpark {
.master(master)
.getOrCreate()
import spark.implicits._
implicit val mrEncoder: Encoder[MetadataRecord] = Encoders.kryo[MetadataRecord]
implicit val resEncoder: Encoder[Oaf] = Encoders.kryo[Oaf]
import spark.implicits._
val mdstoreOutputVersion = parser.get("mdstoreOutputVersion")
val mapper = new ObjectMapper()
val cleanedMdStoreVersion = mapper.readValue(mdstoreOutputVersion, classOf[MDStoreVersion])
val outputBasePath = cleanedMdStoreVersion.getHdfsPath
log.info("outputBasePath: {}", outputBasePath)
val targetPath = s"$outputBasePath/$MDSTORE_DATA_PATH"
spark.read.load(sourcePath).as[DataciteType]
.filter(d => d.isActive)
.flatMap(d => DataciteToOAFTransformation.generateOAF(d.json, d.timestamp, d.timestamp, vocabularies, exportLinks))
.filter(d => d != null)
.flatMap(i => fixRelations(i)).filter(i => i != null)
.write.mode(SaveMode.Overwrite).save(targetPath)
val total_items = spark.read.load(targetPath).as[Oaf].count()
writeHdfsFile(spark.sparkContext.hadoopConfiguration, s"$total_items", outputBasePath + MDSTORE_SIZE_PATH)
}
}
}

View File

@ -1,6 +1,5 @@
package eu.dnetlib.dhp.actionmanager.datacite
package eu.dnetlib.dhp.datacite
import eu.dnetlib.dhp.actionmanager.datacite.DataciteToOAFTransformation.df_it
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, LocalFileSystem, Path}
@ -9,14 +8,14 @@ import org.apache.hadoop.io.{IntWritable, SequenceFile, Text}
import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.expressions.Aggregator
import org.apache.spark.sql.functions.max
import org.apache.spark.sql.{Dataset, Encoder, SaveMode, SparkSession}
import org.json4s.DefaultFormats
import org.json4s.jackson.JsonMethods.parse
import org.apache.spark.sql.functions.max
import org.slf4j.{Logger, LoggerFactory}
import java.time.format.DateTimeFormatter._
import java.time.{LocalDate, LocalDateTime, ZoneOffset}
import java.time.format.DateTimeFormatter.ISO_DATE_TIME
import java.time.{LocalDateTime, ZoneOffset}
import scala.io.Source
object ImportDatacite {
@ -138,11 +137,11 @@ object ImportDatacite {
}
}
private def writeSequenceFile(hdfsTargetPath: Path, timestamp: Long, conf: Configuration, bs:Int): Long = {
var from:Long = timestamp * 1000
val delta:Long = 100000000L
private def writeSequenceFile(hdfsTargetPath: Path, timestamp: Long, conf: Configuration, bs: Int): Long = {
var from: Long = timestamp * 1000
val delta: Long = 100000000L
var client: DataciteAPIImporter = null
val now :Long =System.currentTimeMillis()
val now: Long = System.currentTimeMillis()
var i = 0
try {
val writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(hdfsTargetPath), SequenceFile.Writer.keyClass(classOf[IntWritable]), SequenceFile.Writer.valueClass(classOf[Text]))
@ -168,7 +167,7 @@ object ImportDatacite {
start = System.currentTimeMillis
}
}
println(s"updating from value: $from -> ${from+delta}")
println(s"updating from value: $from -> ${from + delta}")
from = from + delta
}
} catch {
@ -183,4 +182,4 @@ object ImportDatacite {
i
}
}
}

View File

@ -1,18 +1,14 @@
package eu.dnetlib.dhp.actionmanager.datacite
package eu.dnetlib.dhp.datacite
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.schema.oaf.{Oaf, Result}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.LocalFileSystem
import org.apache.hadoop.hdfs.DistributedFileSystem
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Encoder, Encoders, SparkSession}
import org.apache.spark.sql.functions.max
import org.apache.spark.sql.{Encoder, Encoders, SparkSession}
import org.slf4j.{Logger, LoggerFactory}
import java.text.SimpleDateFormat
import java.util.{Date, Locale}
import java.util.Locale
import scala.io.Source
object SparkDownloadUpdateDatacite {
@ -21,7 +17,7 @@ object SparkDownloadUpdateDatacite {
def main(args: Array[String]): Unit = {
val conf = new SparkConf
val parser = new ArgumentApplicationParser(Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/actionmanager/datacite/generate_dataset_params.json")).mkString)
val parser = new ArgumentApplicationParser(Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/datacite/generate_dataset_params.json")).mkString)
parser.parseArgument(args)
val master = parser.get("master")
val sourcePath = parser.get("sourcePath")
@ -42,9 +38,9 @@ object SparkDownloadUpdateDatacite {
import spark.implicits._
val maxDate:String = spark.read.load(workingPath).as[Oaf].filter(s => s.isInstanceOf[Result]).map(r => r.asInstanceOf[Result].getDateofcollection).select(max("value")).first().getString(0)
val maxDate: String = spark.read.load(workingPath).as[Oaf].filter(s => s.isInstanceOf[Result]).map(r => r.asInstanceOf[Result].getDateofcollection).select(max("value")).first().getString(0)
val ISO8601FORMAT = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ssZ", Locale.US)
val string_to_date =ISO8601FORMAT.parse(maxDate)
val string_to_date = ISO8601FORMAT.parse(maxDate)
val ts = string_to_date.getTime

View File

@ -3,6 +3,7 @@ package eu.dnetlib.dhp.sx.bio
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.schema.oaf.Oaf
import BioDBToOAF.ScholixResolved
import eu.dnetlib.dhp.collection.CollectionUtils
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}
@ -35,13 +36,13 @@ object SparkTransformBioDatabaseToOAF {
import spark.implicits._
database.toUpperCase() match {
case "UNIPROT" =>
spark.createDataset(sc.textFile(dbPath).flatMap(i => BioDBToOAF.uniprotToOAF(i))).write.mode(SaveMode.Overwrite).save(targetPath)
spark.createDataset(sc.textFile(dbPath).flatMap(i => BioDBToOAF.uniprotToOAF(i))).flatMap(i=> CollectionUtils.fixRelations(i)).filter(i => i != null).write.mode(SaveMode.Overwrite).save(targetPath)
case "PDB" =>
spark.createDataset(sc.textFile(dbPath).flatMap(i => BioDBToOAF.pdbTOOaf(i))).write.mode(SaveMode.Overwrite).save(targetPath)
spark.createDataset(sc.textFile(dbPath).flatMap(i => BioDBToOAF.pdbTOOaf(i))).flatMap(i=> CollectionUtils.fixRelations(i)).filter(i => i != null).write.mode(SaveMode.Overwrite).save(targetPath)
case "SCHOLIX" =>
spark.read.load(dbPath).as[ScholixResolved].map(i => BioDBToOAF.scholixResolvedToOAF(i)).write.mode(SaveMode.Overwrite).save(targetPath)
spark.read.load(dbPath).as[ScholixResolved].map(i => BioDBToOAF.scholixResolvedToOAF(i)).flatMap(i=> CollectionUtils.fixRelations(i)).filter(i => i != null).write.mode(SaveMode.Overwrite).save(targetPath)
case "CROSSREF_LINKS" =>
spark.createDataset(sc.textFile(dbPath).map(i => BioDBToOAF.crossrefLinksToOaf(i))).write.mode(SaveMode.Overwrite).save(targetPath)
spark.createDataset(sc.textFile(dbPath).map(i => BioDBToOAF.crossrefLinksToOaf(i))).flatMap(i=> CollectionUtils.fixRelations(i)).filter(i => i != null).write.mode(SaveMode.Overwrite).save(targetPath)
}
}

View File

@ -5,6 +5,7 @@ import eu.dnetlib.dhp.schema.oaf.Oaf
import eu.dnetlib.dhp.sx.bio.BioDBToOAF
import eu.dnetlib.dhp.sx.bio.BioDBToOAF.EBILinkItem
import BioDBToOAF.EBILinkItem
import eu.dnetlib.dhp.collection.CollectionUtils
import org.apache.commons.io.IOUtils
import org.apache.spark.SparkConf
import org.apache.spark.sql._
@ -37,6 +38,7 @@ object SparkEBILinksToOaf {
ebLinks.flatMap(j => BioDBToOAF.parse_ebi_links(j.links))
.filter(p => BioDBToOAF.EBITargetLinksFilter(p))
.flatMap(p => BioDBToOAF.convertEBILinksToOaf(p))
.flatMap(i=> CollectionUtils.fixRelations(i)).filter(i => i != null)
.write.mode(SaveMode.Overwrite).save(targetPath)
}
}

View File

@ -5,94 +5,249 @@ import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
/**
* This class represent an instance of Pubmed Article extracted from the native XML
*
* @author Sandro La Bruzzo
*/
public class PMArticle implements Serializable {
/**
* the Pubmed Identifier
*/
private String pmid;
/**
* the DOI
*/
private String doi;
/**
* the Pubmed Date extracted from <PubmedPubDate> Specifies a date significant to either the article's history or the citation's processing.
* All <History> dates will have a <Year>, <Month>, and <Day> elements. Some may have an <Hour>, <Minute>, and <Second> element(s).
*/
private String date;
/**
* This is an 'envelop' element that contains various elements describing the journal cited; i.e., ISSN, Volume, Issue, and PubDate and author name(s), however, it does not contain data itself.
*/
private PMJournal journal;
/**
* The full journal title (taken from NLM cataloging data following NLM rules for how to compile a serial name) is exported in this element. Some characters that are not part of the NLM MEDLINE/PubMed Character Set reside in a relatively small number of full journal titles. The NLM journal title abbreviation is exported in the <MedlineTA> element.
*/
private String title;
/**
* English-language abstracts are taken directly from the published article.
* If the article does not have a published abstract, the National Library of Medicine does not create one,
* thus the record lacks the <Abstract> and <AbstractText> elements. However, in the absence of a formally
* labeled abstract in the published article, text from a substantive "summary", "summary and conclusions" or "conclusions and summary" may be used.
*/
private String description;
/**
* the language in which an article was published is recorded in <Language>.
* All entries are three letter abbreviations stored in lower case, such as eng, fre, ger, jpn, etc. When a single
* record contains more than one language value the XML export program extracts the languages in alphabetic order by the 3-letter language value.
* Some records provided by collaborating data producers may contain the value und to identify articles whose language is undetermined.
*/
private String language;
/**
* NLM controlled vocabulary, Medical Subject Headings (MeSH®), is used to characterize the content of the articles represented by MEDLINE citations. *
*/
private final List<PMSubject> subjects = new ArrayList<>();
/**
* This element is used to identify the type of article indexed for MEDLINE;
* it characterizes the nature of the information or the manner in which it is conveyed as well as the type of
* research support received (e.g., Review, Letter, Retracted Publication, Clinical Conference, Research Support, N.I.H., Extramural).
*/
private final List<PMSubject> publicationTypes = new ArrayList<>();
/**
* Personal and collective (corporate) author names published with the article are found in <AuthorList>.
*/
private List<PMAuthor> authors = new ArrayList<>();
public List<PMSubject> getPublicationTypes() {
return publicationTypes;
}
/**
* <GrantID> contains the research grant or contract number (or both) that designates financial support by any agency of the United States Public Health Service
* or any institute of the National Institutes of Health. Additionally, beginning in late 2005, grant numbers are included for many other US and non-US funding agencies and organizations.
*/
private final List<PMGrant> grants = new ArrayList<>();
public List<PMGrant> getGrants() {
return grants;
}
/**
* get the DOI
* @return a DOI
*/
public String getDoi() {
return doi;
}
/**
* Set the DOI
* @param doi a DOI
*/
public void setDoi(String doi) {
this.doi = doi;
}
/**
* get the Pubmed Identifier
* @return the PMID
*/
public String getPmid() {
return pmid;
}
/**
* set the Pubmed Identifier
* @param pmid the Pubmed Identifier
*/
public void setPmid(String pmid) {
this.pmid = pmid;
}
/**
* the Pubmed Date extracted from <PubmedPubDate> Specifies a date significant to either the article's history or the citation's processing.
* All <History> dates will have a <Year>, <Month>, and <Day> elements. Some may have an <Hour>, <Minute>, and <Second> element(s).
*
* @return the Pubmed Date
*/
public String getDate() {
return date;
}
/**
* Set the pubmed Date
* @param date
*/
public void setDate(String date) {
this.date = date;
}
/**
* The full journal title (taken from NLM cataloging data following NLM rules for how to compile a serial name) is exported in this element.
* Some characters that are not part of the NLM MEDLINE/PubMed Character Set reside in a relatively small number of full journal titles.
* The NLM journal title abbreviation is exported in the <MedlineTA> element.
*
* @return the pubmed Journal Extracted
*/
public PMJournal getJournal() {
return journal;
}
/**
* Set the mapped pubmed Journal
* @param journal
*/
public void setJournal(PMJournal journal) {
this.journal = journal;
}
/**
* English-language abstracts are taken directly from the published article.
* If the article does not have a published abstract, the National Library of Medicine does not create one,
* thus the record lacks the <Abstract> and <AbstractText> elements. However, in the absence of a formally
* labeled abstract in the published article, text from a substantive "summary", "summary and conclusions" or "conclusions and summary" may be used.
*
* @return the extracted pubmed Title
*/
public String getTitle() {
return title;
}
/**
* set the pubmed title
* @param title
*/
public void setTitle(String title) {
this.title = title;
}
/**
* English-language abstracts are taken directly from the published article.
* If the article does not have a published abstract, the National Library of Medicine does not create one,
* thus the record lacks the <Abstract> and <AbstractText> elements. However, in the absence of a formally
* labeled abstract in the published article, text from a substantive "summary", "summary and conclusions" or "conclusions and summary" may be used.
*
* @return the Mapped Pubmed Article Abstracts
*/
public String getDescription() {
return description;
}
/**
* Set the Mapped Pubmed Article Abstracts
* @param description
*/
public void setDescription(String description) {
this.description = description;
}
/**
* Personal and collective (corporate) author names published with the article are found in <AuthorList>.
*
* @return get the Mapped Authors lists
*/
public List<PMAuthor> getAuthors() {
return authors;
}
/**
* Set the Mapped Authors lists
* @param authors
*/
public void setAuthors(List<PMAuthor> authors) {
this.authors = authors;
}
/**
* This element is used to identify the type of article indexed for MEDLINE;
* it characterizes the nature of the information or the manner in which it is conveyed as well as the type of
* research support received (e.g., Review, Letter, Retracted Publication, Clinical Conference, Research Support, N.I.H., Extramural).
*
* @return the mapped Subjects
*/
public List<PMSubject> getSubjects() {
return subjects;
}
/**
*
* the language in which an article was published is recorded in <Language>.
* All entries are three letter abbreviations stored in lower case, such as eng, fre, ger, jpn, etc. When a single
* record contains more than one language value the XML export program extracts the languages in alphabetic order by the 3-letter language value.
* Some records provided by collaborating data producers may contain the value und to identify articles whose language is undetermined.
*
* @return The mapped Language
*/
public String getLanguage() {
return language;
}
/**
*
* Set The mapped Language
*
* @param language the mapped Language
*/
public void setLanguage(String language) {
this.language = language;
}
/**
* This element is used to identify the type of article indexed for MEDLINE;
* it characterizes the nature of the information or the manner in which it is conveyed as well as the type of
* research support received (e.g., Review, Letter, Retracted Publication, Clinical Conference, Research Support, N.I.H., Extramural).
*
* @return the mapped Publication Type
*/
public List<PMSubject> getPublicationTypes() {
return publicationTypes;
}
/**
* <GrantID> contains the research grant or contract number (or both) that designates financial support by any agency of the United States Public Health Service
* or any institute of the National Institutes of Health. Additionally, beginning in late 2005, grant numbers are included for many other US and non-US funding agencies and organizations.
* @return the mapped grants
*/
public List<PMGrant> getGrants() {
return grants;
}
}

View File

@ -3,27 +3,57 @@ package eu.dnetlib.dhp.sx.bio.pubmed;
import java.io.Serializable;
/**
* The type Pubmed author.
*
* @author Sandro La Bruzzo
*/
public class PMAuthor implements Serializable {
private String lastName;
private String foreName;
/**
* Gets last name.
*
* @return the last name
*/
public String getLastName() {
return lastName;
}
/**
* Sets last name.
*
* @param lastName the last name
*/
public void setLastName(String lastName) {
this.lastName = lastName;
}
/**
* Gets fore name.
*
* @return the fore name
*/
public String getForeName() {
return foreName;
}
/**
* Sets fore name.
*
* @param foreName the fore name
*/
public void setForeName(String foreName) {
this.foreName = foreName;
}
/**
* Gets full name.
*
* @return the full name
*/
public String getFullName() {
return String
.format("%s, %s", this.foreName != null ? this.foreName : "", this.lastName != null ? this.lastName : "");

View File

@ -1,41 +1,86 @@
package eu.dnetlib.dhp.sx.bio.pubmed;
/**
* The type Pm grant.
*
* @author Sandro La Bruzzo
*/
public class PMGrant {
private String grantID;
private String agency;
private String country;
/**
* Instantiates a new Pm grant.
*/
public PMGrant() {
}
/**
* Instantiates a new Pm grant.
*
* @param grantID the grant id
* @param agency the agency
* @param country the country
*/
public PMGrant(String grantID, String agency, String country) {
this.grantID = grantID;
this.agency = agency;
this.country = country;
}
/**
* Gets grant id.
*
* @return the grant id
*/
public String getGrantID() {
return grantID;
}
/**
* Sets grant id.
*
* @param grantID the grant id
*/
public void setGrantID(String grantID) {
this.grantID = grantID;
}
/**
* Gets agency.
*
* @return the agency
*/
public String getAgency() {
return agency;
}
/**
* Sets agency.
*
* @param agency the agency
*/
public void setAgency(String agency) {
this.agency = agency;
}
/**
* Gets country.
*
* @return the country
*/
public String getCountry() {
return country;
}
/**
* Sets country.
*
* @param country the country
*/
public void setCountry(String country) {
this.country = country;
}

View File

@ -3,6 +3,11 @@ package eu.dnetlib.dhp.sx.bio.pubmed;
import java.io.Serializable;
/**
* The type Pm journal.
*
* @author Sandro La Bruzzo
*/
public class PMJournal implements Serializable {
private String issn;
@ -11,42 +16,92 @@ public class PMJournal implements Serializable {
private String date;
private String title;
/**
* Gets issn.
*
* @return the issn
*/
public String getIssn() {
return issn;
}
/**
* Sets issn.
*
* @param issn the issn
*/
public void setIssn(String issn) {
this.issn = issn;
}
/**
* Gets volume.
*
* @return the volume
*/
public String getVolume() {
return volume;
}
/**
* Sets volume.
*
* @param volume the volume
*/
public void setVolume(String volume) {
this.volume = volume;
}
/**
* Gets issue.
*
* @return the issue
*/
public String getIssue() {
return issue;
}
/**
* Sets issue.
*
* @param issue the issue
*/
public void setIssue(String issue) {
this.issue = issue;
}
/**
* Gets date.
*
* @return the date
*/
public String getDate() {
return date;
}
/**
* Sets date.
*
* @param date the date
*/
public void setDate(String date) {
this.date = date;
}
/**
* Gets title.
*
* @return the title
*/
public String getTitle() {
return title;
}
/**
* Sets title.
*
* @param title the title
*/
public void setTitle(String title) {
this.title = title;
}

View File

@ -2,6 +2,12 @@ package eu.dnetlib.dhp.sx.bio.pubmed
import scala.xml.MetaData
import scala.xml.pull.{EvElemEnd, EvElemStart, EvText, XMLEventReader}
/**
*
* @param xml
*/
class PMParser(xml:XMLEventReader) extends Iterator[PMArticle] {
var currentArticle:PMArticle = generateNextArticle()

View File

@ -1,40 +1,83 @@
package eu.dnetlib.dhp.sx.bio.pubmed;
/**
* The type Pubmed subject.
*/
public class PMSubject {
private String value;
private String meshId;
private String registryNumber;
/**
* Instantiates a new Pm subject.
*/
public PMSubject() {
}
/**
* Instantiates a new Pm subject.
*
* @param value the value
* @param meshId the mesh id
* @param registryNumber the registry number
*/
public PMSubject(String value, String meshId, String registryNumber) {
this.value = value;
this.meshId = meshId;
this.registryNumber = registryNumber;
}
/**
* Gets value.
*
* @return the value
*/
public String getValue() {
return value;
}
/**
* Sets value.
*
* @param value the value
*/
public void setValue(String value) {
this.value = value;
}
/**
* Gets mesh id.
*
* @return the mesh id
*/
public String getMeshId() {
return meshId;
}
/**
* Sets mesh id.
*
* @param meshId the mesh id
*/
public void setMeshId(String meshId) {
this.meshId = meshId;
}
/**
* Gets registry number.
*
* @return the registry number
*/
public String getRegistryNumber() {
return registryNumber;
}
/**
* Sets registry number.
*
* @param registryNumber the registry number
*/
public void setRegistryNumber(String registryNumber) {
this.registryNumber = registryNumber;
}

View File

@ -8,6 +8,9 @@ import scala.collection.JavaConverters._
import java.util.regex.Pattern
/**
*
*/
object PubMedToOaf {
val SUBJ_CLASS = "keywords"
@ -15,7 +18,17 @@ object PubMedToOaf {
"pmid" -> "https://pubmed.ncbi.nlm.nih.gov/",
"doi" -> "https://dx.doi.org/"
)
val dataInfo: DataInfo = OafMapperUtils.dataInfo(false, null, false, false, ModelConstants.PROVENANCE_ACTION_SET_QUALIFIER, "0.9")
val collectedFrom: KeyValue = OafMapperUtils.keyValue(ModelConstants.EUROPE_PUBMED_CENTRAL_ID, "Europe PubMed Central")
/**
* Cleaning the DOI Applying regex in order to
* remove doi starting with URL
* @param doi input DOI
* @return cleaned DOI
*/
def cleanDoi(doi: String): String = {
val regex = "^10.\\d{4,9}\\/[\\[\\]\\-\\<\\>._;()\\/:A-Z0-9]+$"
@ -30,6 +43,15 @@ object PubMedToOaf {
null
}
/**
*
* Create an instance of class extends Result
* starting from OAF instanceType value
*
* @param cobjQualifier OAF instance type
* @param vocabularies All dnet vocabularies
* @return the correct instance
*/
def createResult(cobjQualifier: Qualifier, vocabularies: VocabularyGroup): Result = {
val result_typologies = getVocabularyTerm(ModelConstants.DNET_RESULT_TYPOLOGIES, vocabularies, cobjQualifier.getClassid)
result_typologies.getClassid match {
@ -42,6 +64,12 @@ object PubMedToOaf {
}
}
/**
* Mapping the Pubmedjournal info into the OAF Journale
*
* @param j the pubmedJournal
* @return the OAF Journal
*/
def mapJournal(j: PMJournal): Journal = {
if (j == null)
return null
@ -49,6 +77,7 @@ object PubMedToOaf {
journal.setDataInfo(dataInfo)
journal.setName(j.getTitle)
journal.setConferencedate(j.getDate)
journal.setVol(j.getVolume)
journal.setIssnPrinted(j.getIssn)
journal.setIss(j.getIssue)
@ -57,25 +86,43 @@ object PubMedToOaf {
}
/**
*
* Find vocabulary term into synonyms and term in the vocabulary
*
* @param vocabularyName the input vocabulary name
* @param vocabularies all the vocabularies
* @param term the term to find
*
* @return the cleaned term value
*/
def getVocabularyTerm(vocabularyName: String, vocabularies: VocabularyGroup, term: String): Qualifier = {
val a = vocabularies.getSynonymAsQualifier(vocabularyName, term)
val b = vocabularies.getTermAsQualifier(vocabularyName, term)
if (a == null) b else a
}
val dataInfo: DataInfo = OafMapperUtils.dataInfo(false, null, false, false, ModelConstants.PROVENANCE_ACTION_SET_QUALIFIER, "0.9")
val collectedFrom: KeyValue = OafMapperUtils.keyValue(ModelConstants.EUROPE_PUBMED_CENTRAL_ID, "Europe PubMed Central")
/**
* Map the Pubmed Article into the OAF instance
*
*
* @param article the pubmed articles
* @param vocabularies the vocabularies
* @return The OAF instance if the mapping did not fail
*/
def convert(article: PMArticle, vocabularies: VocabularyGroup): Result = {
if (article.getPublicationTypes == null)
return null
val i = new Instance
// MAP PMID into pid with classid = classname = pmid
val pidList: List[StructuredProperty] = List(OafMapperUtils.structuredProperty(article.getPmid, PidType.pmid.toString, PidType.pmid.toString, ModelConstants.DNET_PID_TYPES, ModelConstants.DNET_PID_TYPES, dataInfo))
if (pidList == null)
return null
// MAP //ArticleId[./@IdType="doi"] into alternateIdentifier with classid = classname = doi
var alternateIdentifier: StructuredProperty = null
if (article.getDoi != null) {
val normalizedPid = cleanDoi(article.getDoi)
@ -83,43 +130,64 @@ object PubMedToOaf {
alternateIdentifier = OafMapperUtils.structuredProperty(normalizedPid, PidType.doi.toString, PidType.doi.toString, ModelConstants.DNET_PID_TYPES, ModelConstants.DNET_PID_TYPES, dataInfo)
}
// INSTANCE MAPPING
//--------------------------------------------------------------------------------------
// If the article contains the typology Journal Article then we apply this type
//else We have to find a terms that match the vocabulary otherwise we discard it
val ja = article.getPublicationTypes.asScala.find(s => "Journal Article".equalsIgnoreCase(s.getValue))
val pubmedInstance = new Instance
if (ja.isDefined) {
val cojbCategory = getVocabularyTerm(ModelConstants.DNET_PUBLICATION_RESOURCE, vocabularies, ja.get.getValue)
i.setInstancetype(cojbCategory)
pubmedInstance.setInstancetype(cojbCategory)
} else {
val i_type = article.getPublicationTypes.asScala
.map(s => getVocabularyTerm(ModelConstants.DNET_PUBLICATION_RESOURCE, vocabularies, s.getValue))
.find(q => q != null)
if (i_type.isDefined)
i.setInstancetype(i_type.get)
pubmedInstance.setInstancetype(i_type.get)
else
return null
}
val result = createResult(i.getInstancetype, vocabularies)
val result = createResult(pubmedInstance.getInstancetype, vocabularies)
if (result == null)
return result
result.setDataInfo(dataInfo)
i.setPid(pidList.asJava)
pubmedInstance.setPid(pidList.asJava)
if (alternateIdentifier != null)
i.setAlternateIdentifier(List(alternateIdentifier).asJava)
result.setInstance(List(i).asJava)
i.getPid.asScala.filter(p => "pmid".equalsIgnoreCase(p.getQualifier.getClassid)).map(p => p.getValue)(collection.breakOut)
pubmedInstance.setAlternateIdentifier(List(alternateIdentifier).asJava)
result.setInstance(List(pubmedInstance).asJava)
pubmedInstance.getPid.asScala.filter(p => "pmid".equalsIgnoreCase(p.getQualifier.getClassid)).map(p => p.getValue)(collection.breakOut)
//CREATE URL From pmid
val urlLists: List[String] = pidList
.map(s => (urlMap.getOrElse(s.getQualifier.getClassid, ""), s.getValue))
.filter(t => t._1.nonEmpty)
.map(t => t._1 + t._2)
if (urlLists != null)
i.setUrl(urlLists.asJava)
i.setDateofacceptance(OafMapperUtils.field(GraphCleaningFunctions.cleanDate(article.getDate), dataInfo))
i.setCollectedfrom(collectedFrom)
pubmedInstance.setUrl(urlLists.asJava)
//ASSIGN DateofAcceptance
pubmedInstance.setDateofacceptance(OafMapperUtils.field(GraphCleaningFunctions.cleanDate(article.getDate), dataInfo))
//ASSIGN COLLECTEDFROM
pubmedInstance.setCollectedfrom(collectedFrom)
result.setPid(pidList.asJava)
//END INSTANCE MAPPING
//--------------------------------------------------------------------------------------
// JOURNAL MAPPING
//--------------------------------------------------------------------------------------
if (article.getJournal != null && result.isInstanceOf[Publication])
result.asInstanceOf[Publication].setJournal(mapJournal(article.getJournal))
result.setCollectedfrom(List(collectedFrom).asJava)
//END JOURNAL MAPPING
//--------------------------------------------------------------------------------------
// RESULT MAPPING
//--------------------------------------------------------------------------------------
result.setDateofacceptance(OafMapperUtils.field(GraphCleaningFunctions.cleanDate(article.getDate), dataInfo))
if (article.getTitle == null || article.getTitle.isEmpty)
@ -159,6 +227,9 @@ object PubMedToOaf {
result.setId(article.getPmid)
// END RESULT MAPPING
//--------------------------------------------------------------------------------------
val id = IdentifierFactory.createIdentifier(result)
if (article.getPmid.equalsIgnoreCase(id))
return null

View File

@ -0,0 +1,33 @@
[
{
"paramName":"s",
"paramLongName":"sourcePath",
"paramDescription": "the path of the sequencial file to read",
"paramRequired": true
},
{
"paramName":"out",
"paramLongName":"outputPath",
"paramDescription": "the output path",
"paramRequired": true
},
{
"paramName": "ssm",
"paramLongName": "isSparkSessionManaged",
"paramDescription": "true if the spark session is managed, false otherwise",
"paramRequired": false
},
{
"paramName": "hnn",
"paramLongName": "hdfsNameNode",
"paramDescription": "the path used to store the HostedByMap",
"paramRequired": true
},
{
"paramName": "cfn",
"paramLongName": "classForName",
"paramDescription": "the path used to store the HostedByMap",
"paramRequired": true
}
]

View File

@ -12,12 +12,19 @@
<value>true</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>spark2</value>
<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>
</configuration>

View File

@ -0,0 +1,174 @@
<workflow-app name="UnresolvedEntities" 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>bipScorePath</name>
<description>the path where to find the bipFinder scores</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="prepareInfo"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<fork name="prepareInfo">
<path start="prepareBip"/>
<path start="getFOS"/>
</fork>
<action name="prepareBip">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Produces the unresolved from bip finder!</name>
<class>eu.dnetlib.dhp.actionmanager.createunresolvedentities.PrepareBipFinder</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>${bipScorePath}</arg>
<arg>--outputPath</arg><arg>${workingDir}/prepared</arg>
</spark>
<ok to="join"/>
<error to="Kill"/>
</action>
<action name="getFOS">
<java>
<main-class>eu.dnetlib.dhp.actionmanager.createunresolvedentities.GetFOSData</main-class>
<arg>--hdfsNameNode</arg><arg>${nameNode}</arg>
<arg>--sourcePath</arg><arg>${fosPath}</arg>
<arg>--outputPath</arg><arg>${workingDir}/input/fos</arg>
<arg>--classForName</arg><arg>eu.dnetlib.dhp.actionmanager.createunresolvedentities.model.FOSDataModel</arg>
</java>
<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 unresolved 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>
</spark>
<ok to="join"/>
<error to="Kill"/>
</action>
<join name="join" to="produceUnresolved"/>
<action name="produceUnresolved">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Saves the result produced for bip and fos by grouping results with the same id</name>
<class>eu.dnetlib.dhp.actionmanager.createunresolvedentities.SparkSaveUnresolved</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</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,20 @@
[
{
"paramName": "issm",
"paramLongName": "isSparkSessionManaged",
"paramDescription": "when true will stop SparkSession after job execution",
"paramRequired": false
},
{
"paramName": "sp",
"paramLongName": "sourcePath",
"paramDescription": "the URL from where to get the programme file",
"paramRequired": true
},
{
"paramName": "o",
"paramLongName": "outputPath",
"paramDescription": "the path of the new ActionSet",
"paramRequired": true
}
]

View File

@ -0,0 +1,20 @@
[
{
"paramName": "issm",
"paramLongName": "isSparkSessionManaged",
"paramDescription": "when true will stop SparkSession after job execution",
"paramRequired": false
},
{
"paramName": "sp",
"paramLongName": "sourcePath",
"paramDescription": "the URL from where to get the programme file",
"paramRequired": true
},
{
"paramName": "o",
"paramLongName": "outputPath",
"paramDescription": "the path of the new ActionSet",
"paramRequired": true
}
]

View File

@ -1,81 +0,0 @@
<workflow-app name="Import_Datacite_and_transform_to_OAF" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>mainPath</name>
<description>the working path of Datacite stores</description>
</property>
<property>
<name>isLookupUrl</name>
<description>The IS lookUp service endopoint</description>
</property>
<property>
<name>blocksize</name>
<value>100</value>
<description>The request block size</description>
</property>
</parameters>
<start to="ImportDatacite"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="ImportDatacite">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>ImportDatacite</name>
<class>eu.dnetlib.dhp.actionmanager.datacite.ImportDatacite</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}
</spark-opts>
<arg>--targetPath</arg><arg>${mainPath}/datacite_update</arg>
<arg>--dataciteDumpPath</arg><arg>${mainPath}/datacite_dump</arg>
<arg>--namenode</arg><arg>${nameNode}</arg>
<arg>--master</arg><arg>yarn-cluster</arg>
<arg>--blocksize</arg><arg>${blocksize}</arg>
</spark>
<ok to="TransformJob"/>
<error to="Kill"/>
</action>
<action name="TransformJob">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>TransformJob</name>
<class>eu.dnetlib.dhp.actionmanager.datacite.GenerateDataciteDatasetSpark</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.sql.shuffle.partitions=3840
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${mainPath}/datacite_dump</arg>
<arg>--targetPath</arg><arg>${mainPath}/datacite_oaf</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--exportLinks</arg><arg>false</arg>
<arg>--master</arg><arg>yarn-cluster</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -1,84 +0,0 @@
<workflow-app name="Generate_Datacite_and_Crossref_dump_for_Scholexplorer" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>datacitePath</name>
<description>the path of Datacite spark dataset</description>
</property>
<property>
<name>isLookupUrl</name>
<description>The IS lookUp service endopoint</description>
</property>
<property>
<name>crossrefPath</name>
<description>the path of Crossref spark dataset</description>
</property>
<property>
<name>targetPath</name>
<description>the path of Crossref spark dataset</description>
</property>
</parameters>
<start to="ImportDatacite"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="ImportDatacite">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>ImportDatacite</name>
<class>eu.dnetlib.dhp.actionmanager.datacite.GenerateDataciteDatasetSpark</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.sql.shuffle.partitions=3840
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${datacitePath}</arg>
<arg>--targetPath</arg><arg>${targetPath}/datacite_oaf</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--exportLinks</arg><arg>true</arg>
<arg>--master</arg><arg>yarn-cluster</arg>
</spark>
<ok to="FilterCrossrefEntities"/>
<error to="Kill"/>
</action>
<action name="FilterCrossrefEntities">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>FilterCrossrefEntities</name>
<class>eu.dnetlib.dhp.actionmanager.datacite.FilterCrossrefEntitiesSpark</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.sql.shuffle.partitions=3840
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${crossrefPath}</arg>
<arg>--targetPath</arg><arg>${targetPath}/crossref_oaf</arg>
<arg>--master</arg><arg>yarn-cluster</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,25 @@
[
{
"paramName": "ip",
"paramLongName": "inputPath",
"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
}, {
"paramName": "sdr",
"paramLongName": "shouldDuplicateRels",
"paramDescription": "the hdfs name node",
"paramRequired": false
}
]

View File

@ -0,0 +1,20 @@
[
{
"paramName": "if",
"paramLongName": "inputFile",
"paramDescription": "the zipped opencitations file",
"paramRequired": true
},
{
"paramName": "wp",
"paramLongName": "workingPath",
"paramDescription": "the working path",
"paramRequired": true
},
{
"paramName": "hnn",
"paramLongName": "hdfsNameNode",
"paramDescription": "the hdfs name node",
"paramRequired": true
}
]

View File

@ -0,0 +1,58 @@
<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>oozie.action.sharelib.for.spark</name>
<value>spark2</value>
</property>
<property>
<name>hive_metastore_uris</name>
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
</property>
<property>
<name>spark2YarnHistoryServerAddress</name>
<value>http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089</value>
</property>
<property>
<name>spark2ExtraListeners</name>
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
</property>
<property>
<name>spark2SqlQueryExecutionListeners</name>
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
</property>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
<property>
<name>sparkExecutorNumber</name>
<value>4</value>
</property>
<property>
<name>spark2EventLogDir</name>
<value>/user/spark/spark2ApplicationHistory</value>
</property>
<property>
<name>sparkDriverMemory</name>
<value>15G</value>
</property>
<property>
<name>sparkExecutorMemory</name>
<value>6G</value>
</property>
<property>
<name>sparkExecutorCores</name>
<value>1</value>
</property>
</configuration>

View File

@ -0,0 +1,2 @@
#!/bin/bash
for file in $(echo $1 | tr ";" "\n"); do curl -L $(echo $file | cut -d '@' -f 1 ) | hdfs dfs -put - $2/$(echo $file | cut -d '@' -f 2) ; done;

View File

@ -0,0 +1,91 @@
<workflow-app name="OpenCitations Integration" xmlns="uri:oozie:workflow:0.5">
<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="resume_from"/>
<decision name="resume_from">
<switch>
<case to="download">${wf:conf('resumeFrom') eq 'DownloadDump'}</case>
<case to="extract">${wf:conf('resumeFrom') eq 'ExtractContent'}</case>
<default to="create_actionset"/> <!-- first action to be done when downloadDump is to be performed -->
</switch>
</decision>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="download">
<shell xmlns="uri:oozie:shell-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<configuration>
<property>
<name>mapred.job.queue.name</name>
<value>${queueName}</value>
</property>
</configuration>
<exec>download.sh</exec>
<argument>${filelist}</argument>
<argument>${workingPath}/Original</argument>
<env-var>HADOOP_USER_NAME=${wf:user()}</env-var>
<file>download.sh</file>
<capture-output/>
</shell>
<ok to="extract"/>
<error to="Kill"/>
</action>
<action name="extract">
<java>
<main-class>eu.dnetlib.dhp.actionmanager.opencitations.GetOpenCitationsRefs</main-class>
<arg>--hdfsNameNode</arg><arg>${nameNode}</arg>
<arg>--inputFile</arg><arg>${inputFile}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
</java>
<ok to="create_actionset"/>
<error to="Kill"/>
</action>
<action name="create_actionset">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Produces the AS for OC</name>
<class>eu.dnetlib.dhp.actionmanager.opencitations.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>--inputPath</arg><arg>${workingPath}/COCI</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,8 @@
[
{"paramName":"n", "paramLongName":"hdfsServerUri", "paramDescription": "the server uri", "paramRequired": true},
{"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the default work path", "paramRequired": true},
{"paramName":"f", "paramLongName":"opencitationFile", "paramDescription": "the name of the file", "paramRequired": true},
{"paramName":"issm", "paramLongName":"isSparkSessionManaged", "paramDescription": "the name of the activities orcid file", "paramRequired": false},
{"paramName":"o", "paramLongName":"outputPath", "paramDescription": "the name of the activities orcid file", "paramRequired": true}
]

View File

@ -1,46 +1,52 @@
<workflow-app name="Datacite_to_ActionSet_Workflow" xmlns="uri:oozie:workflow:0.5">
<workflow-app name="Collect_Datacite" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>sourcePath</name>
<name>mainPath</name>
<description>the working path of Datacite stores</description>
</property>
<property>
<name>outputPath</name>
<description>the path of Datacite ActionSet</description>
<name>isLookupUrl</name>
<description>The IS lookUp service endopoint</description>
</property>
<property>
<name>blocksize</name>
<value>100</value>
<description>The request block size</description>
</property>
</parameters>
<start to="ExportDataset"/>
<start to="ImportDatacite"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="ExportDataset">
<action name="ImportDatacite">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>ExportDataset</name>
<class>eu.dnetlib.dhp.actionmanager.datacite.ExportActionSetJobNode</class>
<name>ImportDatacite</name>
<class>eu.dnetlib.dhp.datacite.ImportDatacite</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.sql.shuffle.partitions=3840
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--targetPath</arg><arg>${outputPath}</arg>
<arg>--targetPath</arg><arg>${mainPath}/datacite_update</arg>
<arg>--dataciteDumpPath</arg><arg>${mainPath}/datacite_dump</arg>
<arg>--namenode</arg><arg>${nameNode}</arg>
<arg>--master</arg><arg>yarn-cluster</arg>
<arg>--blocksize</arg><arg>${blocksize}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -7,8 +7,8 @@
},
{
"paramName": "t",
"paramLongName": "targetPath",
"paramName": "mo",
"paramLongName": "mdstoreOutputVersion",
"paramDescription": "the target mdstore path",
"paramRequired": true
},

View File

@ -0,0 +1,126 @@
<workflow-app name="transform_Datacite" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>mainPath</name>
<description>the working path of Datacite stores</description>
</property>
<property>
<name>isLookupUrl</name>
<description>The IS lookUp service endopoint</description>
</property>
<property>
<name>mdStoreOutputId</name>
<description>the identifier of the cleaned MDStore</description>
</property>
<property>
<name>mdStoreManagerURI</name>
<description>the path of the cleaned mdstore</description>
</property>
</parameters>
<start to="StartTransaction"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="StartTransaction">
<java>
<configuration>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
<arg>--action</arg><arg>NEW_VERSION</arg>
<arg>--mdStoreID</arg><arg>${mdStoreOutputId}</arg>
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
<capture-output/>
</java>
<ok to="TransformJob"/>
<error to="EndReadRollBack"/>
</action>
<action name="TransformJob">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>TransformJob</name>
<class>eu.dnetlib.dhp.datacite.GenerateDataciteDatasetSpark</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.sql.shuffle.partitions=3840
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${mainPath}/datacite_dump</arg>
<arg>--mdstoreOutputVersion</arg><arg>${wf:actionData('StartTransaction')['mdStoreVersion']}</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--exportLinks</arg><arg>true</arg>
<arg>--master</arg><arg>yarn-cluster</arg>
</spark>
<ok to="CommitVersion"/>
<error to="Kill"/>
</action>
<action name="CommitVersion">
<java>
<configuration>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
<arg>--action</arg><arg>COMMIT</arg>
<arg>--namenode</arg><arg>${nameNode}</arg>
<arg>--mdStoreVersion</arg><arg>${wf:actionData('StartTransaction')['mdStoreVersion']}</arg>
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
</java>
<ok to="End"/>
<error to="Kill"/>
</action>
<action name="EndReadRollBack">
<java>
<configuration>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
<arg>--action</arg><arg>READ_UNLOCK</arg>
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
<arg>--readMDStoreId</arg><arg>${wf:actionData('BeginRead')['mdStoreReadLockVersion']}</arg>
<capture-output/>
</java>
<ok to="RollBack"/>
<error to="Kill"/>
</action>
<action name="RollBack">
<java>
<configuration>
<property>
<name>oozie.launcher.mapreduce.user.classpath.first</name>
<value>true</value>
</property>
</configuration>
<main-class>eu.dnetlib.dhp.aggregation.mdstore.MDStoreActionNode</main-class>
<arg>--action</arg><arg>ROLLBACK</arg>
<arg>--mdStoreVersion</arg><arg>${wf:actionData('StartTransaction')['mdStoreVersion']}</arg>
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
</java>
<ok to="Kill"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -52,7 +52,7 @@
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>Incremental Download EBI Links</name>
<class>eu.dnetllib.dhp.sx.bio.ebi.SparkDownloadEBILinks</class>
<class>eu.dnetlib.dhp.sx.bio.ebi.SparkDownloadEBILinks</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
@ -85,7 +85,7 @@
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>Create OAF DataSet</name>
<class>eu.dnetllib.dhp.sx.bio.ebi.SparkEBILinksToOaf</class>
<class>eu.dnetlib.dhp.sx.bio.ebi.SparkEBILinksToOaf</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}

View File

@ -30,7 +30,7 @@
<master>yarn</master>
<mode>cluster</mode>
<name>Convert Baseline to OAF Dataset</name>
<class>eu.dnetllib.dhp.sx.bio.ebi.SparkCreateBaselineDataFrame</class>
<class>eu.dnetlib.dhp.sx.bio.ebi.SparkCreateBaselineDataFrame</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}

View File

@ -0,0 +1,9 @@
##DHP-Aggregation
This module defines a set of oozie workflows for the **collection** and **transformation** of metadata records.
Both workflows interact with the Metadata Store Manager (MdSM) to handle the logical transactions required to ensure
the consistency of the read/write operations on the data as the MdSM in fact keeps track of the logical-physical mapping
of each MDStore.
It defines [mappings](mappings.md) for transformation of different datasource (See mapping section).

View File

@ -0,0 +1,7 @@
##DHP-Aggregation
This module defines a set of oozie workflows for the **collection** and **transformation** of metadata records.
Both workflows interact with the Metadata Store Manager (MdSM) to handle the logical transactions required to ensure
the consistency of the read/write operations on the data as the MdSM in fact keeps track of the logical-physical mapping
of each MDStore.

View File

@ -0,0 +1,18 @@
DHP Aggregation
===============
DHP-Aggregations contains different mappings from original data format into OAF Data Format,
which converge in the graph in different ways:
- Via Action Manager
- Direct in the MdStore on Hadoop
Below the list of the implemented mapping
Mappings
=======
1. [PubMed](pubmed.md)
2. [Datacite](datacite.md)

View File

@ -0,0 +1,62 @@
#Pubmed Mapping
This section describes the mapping implemented for [MEDLINE/PubMed](https://pubmed.ncbi.nlm.nih.gov/).
Collection
---------
The native data is collected from [ftp baseline](https://ftp.ncbi.nlm.nih.gov/pubmed/baseline/) containing XML with
the following [shcema](https://www.nlm.nih.gov/bsd/licensee/elements_descriptions.html)
Parsing
-------
The resposible class of parsing is [PMParser](./scaladocs/#eu.dnetlib.dhp.sx.bio.pubmed.PMParser) that generates
an intermediate mapping of PubMed Article defined [here](/apidocs/eu/dnetlib/dhp/sx/bio/pubmed/package-summary.html)
Mapping
-------
The table below describes the mapping from the XML Native to the OAF mapping
| Xpath Source | Oaf Field | Notes |
| ----------- | ----------- | ----------- |
| //PMID | pid | classid = classname = pmid
| | **Instance Mapping** | |
|//PublicationType | InstanceType | If the article contains the typology **Journal Article** then we apply this type else We have to find a terms that match the vocabulary otherwise we discard it
|//PMID | instance/PID | Map the pmid also in the pid in the instance |
| //ArticleId[./@IdType="doi" | instance/alternateIdentifier |classid = classname = doi
|//PMID | instance/URL | prepend to the PMId the base url https://pubmed.ncbi.nlm.nih.gov/
| //PubmedPubDate | instance/Dateofacceptance | apply the function GraphCleaningFunctions.cleanDate before assign it
| FOR ALL INSTANCE | CollectedFrom | datasourceName: *Europe PubMed Central* DatasourceId:
| | **Journal Mapping** | |
|//Journal/PubDate| Journal/Conferencedate | map the date of the Journal
|//Journal/Title| Journal/Name | |
|//Journal/Volume| Journal/Vol | |
|//Journal/ISSN| Journal/issPrinted | |
|//Journal/Issue| Journal/Iss | |
| | **Publication Mapping** | |
| //PubmedPubDate | Dateofacceptance | apply the function GraphCleaningFunctions.cleanDate before assign it
| //Title | title | with qualifier ModelConstants.MAIN_TITLE_QUALIFIER
| //AbstractText | Description ||
|//Language| Language| cleaning vocabulary -> dnet:languages
|//DescriptorName| Subject | classId, className = keyword
| | **Author Mapping** | |
|//Author/LastName| author.Surname| |
|//Author/ForeName| author.Forename| |
|//Author/FullName| author.Forename| Concatenation of forname + lastName if exist |
|FOR ALL AUTHOR | author.rank| sequential number starting from 1|

Binary file not shown.

After

Width:  |  Height:  |  Size: 21 KiB

View File

@ -0,0 +1,32 @@
<?xml version="1.0" encoding="ISO-8859-1"?>
<project xmlns="http://maven.apache.org/DECORATION/1.8.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/DECORATION/1.8.0 https://maven.apache.org/xsd/decoration-1.8.0.xsd"
name="DHP-Aggregation">
<skin>
<groupId>org.apache.maven.skins</groupId>
<artifactId>maven-fluido-skin</artifactId>
<version>1.8</version>
</skin>
<poweredBy>
<logo name="OpenAIRE Research Graph" href="https://graph.openaire.eu/"
img="https://graph.openaire.eu/assets/common-assets/logo-large-graph.png"/>
</poweredBy>
<body>
<links>
<item name="Code" href="https://code-repo.d4science.org/" />
</links>
<menu name="Documentation">
<item name="Mappings" href="mappings.html" collapse="true">
<item name="Pubmed" href="pubmed.html"/>
<item name="Datacite" href="datacite.html"/>
</item>
<item name="Release Notes" href="release-notes.html" />
<item name="General Information" href="about.html"/>
<item name="JavaDoc" href="apidocs/" />
<item name="ScalaDoc" href="scaladocs/" />
</menu>
<menu ref="reports"/>
</body>
</project>

View File

@ -0,0 +1,250 @@
package eu.dnetlib.dhp.actionmanager.createunresolvedentities;
import static org.junit.jupiter.api.Assertions.*;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.stream.Collectors;
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.model.FOSDataModel;
import eu.dnetlib.dhp.common.collection.CollectorException;
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 bipPrepareTest() throws Exception {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/actionmanager/createunresolvedentities/bip/bip.json")
.getPath();
PrepareBipFinder
.main(
new String[] {
"--isSparkSessionManaged", Boolean.FALSE.toString(),
"--sourcePath", sourcePath,
"--outputPath", workingDir.toString() + "/work"
});
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<Result> tmp = sc
.textFile(workingDir.toString() + "/work/bip")
.map(item -> OBJECT_MAPPER.readValue(item, Result.class));
Assertions.assertEquals(86, tmp.count());
String doi1 = "unresolved::10.0000/096020199389707::doi";
Assertions.assertEquals(1, tmp.filter(r -> r.getId().equals(doi1)).count());
Assertions.assertEquals(3, tmp.filter(r -> r.getId().equals(doi1)).collect().get(0).getMeasures().size());
Assertions
.assertEquals(
"6.34596412687e-09", tmp
.filter(r -> r.getId().equals(doi1))
.collect()
.get(0)
.getMeasures()
.stream()
.filter(sl -> sl.getId().equals("influence"))
.collect(Collectors.toList())
.get(0)
.getUnit()
.get(0)
.getValue());
Assertions
.assertEquals(
"0.641151896994", tmp
.filter(r -> r.getId().equals(doi1))
.collect()
.get(0)
.getMeasures()
.stream()
.filter(sl -> sl.getId().equals("popularity_alt"))
.collect(Collectors.toList())
.get(0)
.getUnit()
.get(0)
.getValue());
Assertions
.assertEquals(
"2.33375102921e-09", tmp
.filter(r -> r.getId().equals(doi1))
.collect()
.get(0)
.getMeasures()
.stream()
.filter(sl -> sl.getId().equals("popularity"))
.collect(Collectors.toList())
.get(0)
.getUnit()
.get(0)
.getValue());
}
@Test
void getFOSFileTest() throws IOException, ClassNotFoundException {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/actionmanager/createunresolvedentities/fos/h2020_fos_sbs.csv")
.getPath();
final String outputPath = workingDir.toString() + "/fos.json";
new GetFOSData()
.doRewrite(
sourcePath, outputPath, "eu.dnetlib.dhp.actionmanager.createunresolvedentities.model.FOSDataModel",
'\t', fs);
BufferedReader in = new BufferedReader(
new InputStreamReader(fs.open(new org.apache.hadoop.fs.Path(outputPath))));
String line;
int count = 0;
while ((line = in.readLine()) != null) {
FOSDataModel fos = new ObjectMapper().readValue(line, FOSDataModel.class);
System.out.println(new ObjectMapper().writeValueAsString(fos));
count += 1;
}
assertEquals(38, count);
}
@Test
void fosPrepareTest() throws Exception {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/actionmanager/createunresolvedentities/fos/fos.json")
.getPath();
PrepareFOSSparkJob
.main(
new String[] {
"--isSparkSessionManaged", Boolean.FALSE.toString(),
"--sourcePath", sourcePath,
"-outputPath", workingDir.toString() + "/work"
});
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<Result> tmp = sc
.textFile(workingDir.toString() + "/work/fos")
.map(item -> OBJECT_MAPPER.readValue(item, Result.class));
String doi1 = "unresolved::10.3390/s18072310::doi";
assertEquals(50, tmp.count());
assertEquals(1, tmp.filter(row -> row.getId().equals(doi1)).count());
assertTrue(
tmp
.filter(r -> r.getId().equals(doi1))
.flatMap(r -> r.getSubject().iterator())
.map(sbj -> sbj.getValue())
.collect()
.contains("engineering and technology"));
assertTrue(
tmp
.filter(r -> r.getId().equals(doi1))
.flatMap(r -> r.getSubject().iterator())
.map(sbj -> sbj.getValue())
.collect()
.contains("nano-technology"));
assertTrue(
tmp
.filter(r -> r.getId().equals(doi1))
.flatMap(r -> r.getSubject().iterator())
.map(sbj -> sbj.getValue())
.collect()
.contains("nanoscience & nanotechnology"));
String doi = "unresolved::10.1111/1365-2656.12831::doi";
assertEquals(1, tmp.filter(row -> row.getId().equals(doi)).count());
assertTrue(
tmp
.filter(r -> r.getId().equals(doi))
.flatMap(r -> r.getSubject().iterator())
.map(sbj -> sbj.getValue())
.collect()
.contains("psychology and cognitive sciences"));
assertTrue(
tmp
.filter(r -> r.getId().equals(doi))
.flatMap(r -> r.getSubject().iterator())
.map(sbj -> sbj.getValue())
.collect()
.contains("social sciences"));
assertFalse(
tmp
.filter(r -> r.getId().equals(doi))
.flatMap(r -> r.getSubject().iterator())
.map(sbj -> sbj.getValue())
.collect()
.contains("NULL"));
}
}

View File

@ -0,0 +1,234 @@
package eu.dnetlib.dhp.actionmanager.createunresolvedentities;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.List;
import java.util.stream.Collectors;
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.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.*;
public class ProduceTest {
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();
private static final String ID_PREFIX = "50|doi_________";
@BeforeAll
public static void beforeAll() throws IOException {
workingDir = Files.createTempDirectory(ProduceTest.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(ProduceTest.class.getSimpleName())
.config(conf)
.getOrCreate();
}
@AfterAll
public static void afterAll() throws IOException {
FileUtils.deleteDirectory(workingDir.toFile());
spark.stop();
}
@Test
void produceTest() throws Exception {
final String bipPath = getClass()
.getResource("/eu/dnetlib/dhp/actionmanager/createunresolvedentities/bip/bip.json")
.getPath();
PrepareBipFinder
.main(
new String[] {
"--isSparkSessionManaged", Boolean.FALSE.toString(),
"--sourcePath", bipPath,
"--outputPath", workingDir.toString() + "/work"
});
final String fosPath = getClass()
.getResource("/eu/dnetlib/dhp/actionmanager/createunresolvedentities/fos/fos.json")
.getPath();
PrepareFOSSparkJob
.main(
new String[] {
"--isSparkSessionManaged", Boolean.FALSE.toString(),
"--sourcePath", fosPath,
"-outputPath", workingDir.toString() + "/work"
});
SparkSaveUnresolved.main(new String[] {
"--isSparkSessionManaged", Boolean.FALSE.toString(),
"--sourcePath", workingDir.toString() + "/work",
"-outputPath", workingDir.toString() + "/unresolved"
});
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<Result> tmp = sc
.textFile(workingDir.toString() + "/unresolved")
.map(item -> OBJECT_MAPPER.readValue(item, Result.class));
Assertions.assertEquals(135, tmp.count());
Assertions.assertEquals(1, tmp.filter(row -> row.getId().equals("unresolved::10.3390/s18072310::doi")).count());
Assertions
.assertEquals(
3, tmp
.filter(row -> row.getId().equals("unresolved::10.3390/s18072310::doi"))
.collect()
.get(0)
.getSubject()
.size());
Assertions
.assertEquals(
3, tmp
.filter(row -> row.getId().equals("unresolved::10.3390/s18072310::doi"))
.collect()
.get(0)
.getMeasures()
.size());
List<StructuredProperty> sbjs = tmp
.filter(row -> row.getId().equals("unresolved::10.3390/s18072310::doi"))
.flatMap(row -> row.getSubject().iterator())
.collect();
sbjs.forEach(sbj -> Assertions.assertEquals("FOS", sbj.getQualifier().getClassid()));
sbjs
.forEach(
sbj -> Assertions
.assertEquals(
"Fields of Science and Technology classification", sbj.getQualifier().getClassname()));
sbjs
.forEach(
sbj -> Assertions
.assertEquals(ModelConstants.DNET_SUBJECT_TYPOLOGIES, sbj.getQualifier().getSchemeid()));
sbjs
.forEach(
sbj -> Assertions
.assertEquals(ModelConstants.DNET_SUBJECT_TYPOLOGIES, sbj.getQualifier().getSchemename()));
sbjs.forEach(sbj -> Assertions.assertEquals(false, sbj.getDataInfo().getDeletedbyinference()));
sbjs.forEach(sbj -> Assertions.assertEquals(true, sbj.getDataInfo().getInferred()));
sbjs.forEach(sbj -> Assertions.assertEquals(false, sbj.getDataInfo().getInvisible()));
sbjs.forEach(sbj -> Assertions.assertEquals("", sbj.getDataInfo().getTrust()));
sbjs.forEach(sbj -> Assertions.assertEquals("update", sbj.getDataInfo().getInferenceprovenance()));
sbjs
.forEach(
sbj -> Assertions.assertEquals("subject:fos", sbj.getDataInfo().getProvenanceaction().getClassid()));
sbjs
.forEach(
sbj -> Assertions
.assertEquals("Inferred by OpenAIRE", sbj.getDataInfo().getProvenanceaction().getClassname()));
sbjs
.forEach(
sbj -> Assertions
.assertEquals(
ModelConstants.DNET_PROVENANCE_ACTIONS, sbj.getDataInfo().getProvenanceaction().getSchemeid()));
sbjs
.forEach(
sbj -> Assertions
.assertEquals(
ModelConstants.DNET_PROVENANCE_ACTIONS,
sbj.getDataInfo().getProvenanceaction().getSchemename()));
sbjs.stream().anyMatch(sbj -> sbj.getValue().equals("engineering and technology"));
sbjs.stream().anyMatch(sbj -> sbj.getValue().equals("nano-technology"));
sbjs.stream().anyMatch(sbj -> sbj.getValue().equals("nanoscience & nanotechnology"));
List<Measure> measures = tmp
.filter(row -> row.getId().equals("unresolved::10.3390/s18072310::doi"))
.flatMap(row -> row.getMeasures().iterator())
.collect();
Assertions
.assertEquals(
"7.5597134689e-09", measures
.stream()
.filter(mes -> mes.getId().equals("influence"))
.collect(Collectors.toList())
.get(0)
.getUnit()
.get(0)
.getValue());
Assertions
.assertEquals(
"4.903880192", measures
.stream()
.filter(mes -> mes.getId().equals("popularity_alt"))
.collect(Collectors.toList())
.get(0)
.getUnit()
.get(0)
.getValue());
Assertions
.assertEquals(
"1.17977512835e-08", measures
.stream()
.filter(mes -> mes.getId().equals("popularity"))
.collect(Collectors.toList())
.get(0)
.getUnit()
.get(0)
.getValue());
Assertions
.assertEquals(
49, tmp
.filter(row -> !row.getId().equals("unresolved::10.3390/s18072310::doi"))
.filter(row -> row.getSubject() != null)
.count());
Assertions
.assertEquals(
85,
tmp
.filter(row -> !row.getId().equals("unresolved::10.3390/s18072310::doi"))
.filter(r -> r.getMeasures() != null)
.count());
}
}

View File

@ -0,0 +1,335 @@
package eu.dnetlib.dhp.actionmanager.opencitations;
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.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.Publication;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions;
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory;
public class CreateOpenCitationsASTest {
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static SparkSession spark;
private static Path workingDir;
private static final Logger log = LoggerFactory
.getLogger(CreateOpenCitationsASTest.class);
@BeforeAll
public static void beforeAll() throws IOException {
workingDir = Files
.createTempDirectory(CreateOpenCitationsASTest.class.getSimpleName());
log.info("using work dir {}", workingDir);
SparkConf conf = new SparkConf();
conf.setAppName(CreateOpenCitationsASTest.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(CreateOpenCitationsASTest.class.getSimpleName())
.config(conf)
.getOrCreate();
}
@AfterAll
public static void afterAll() throws IOException {
FileUtils.deleteDirectory(workingDir.toFile());
spark.stop();
}
@Test
void testNumberofRelations() throws Exception {
String inputPath = getClass()
.getResource(
"/eu/dnetlib/dhp/actionmanager/opencitations/inputFiles")
.getPath();
CreateActionSetSparkJob
.main(
new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-shouldDuplicateRels",
Boolean.TRUE.toString(),
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Relation) aa.getPayload()));
assertEquals(60, tmp.count());
// tmp.foreach(r -> System.out.println(OBJECT_MAPPER.writeValueAsString(r)));
}
@Test
void testNumberofRelations2() throws Exception {
String inputPath = getClass()
.getResource(
"/eu/dnetlib/dhp/actionmanager/opencitations/inputFiles")
.getPath();
CreateActionSetSparkJob
.main(
new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Relation) aa.getPayload()));
assertEquals(44, tmp.count());
// tmp.foreach(r -> System.out.println(OBJECT_MAPPER.writeValueAsString(r)));
}
@Test
void testRelationsCollectedFrom() throws Exception {
String inputPath = getClass()
.getResource(
"/eu/dnetlib/dhp/actionmanager/opencitations/inputFiles")
.getPath();
CreateActionSetSparkJob
.main(
new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Relation) aa.getPayload()));
tmp.foreach(r -> {
assertEquals(ModelConstants.OPENOCITATIONS_NAME, r.getCollectedfrom().get(0).getValue());
assertEquals(ModelConstants.OPENOCITATIONS_ID, r.getCollectedfrom().get(0).getKey());
});
}
@Test
void testRelationsDataInfo() throws Exception {
String inputPath = getClass()
.getResource(
"/eu/dnetlib/dhp/actionmanager/opencitations/inputFiles")
.getPath();
CreateActionSetSparkJob
.main(
new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Relation) aa.getPayload()));
tmp.foreach(r -> {
assertEquals(false, r.getDataInfo().getInferred());
assertEquals(false, r.getDataInfo().getDeletedbyinference());
assertEquals("0.91", r.getDataInfo().getTrust());
assertEquals(
CreateActionSetSparkJob.OPENCITATIONS_CLASSID, r.getDataInfo().getProvenanceaction().getClassid());
assertEquals(
CreateActionSetSparkJob.OPENCITATIONS_CLASSNAME, r.getDataInfo().getProvenanceaction().getClassname());
assertEquals(ModelConstants.DNET_PROVENANCE_ACTIONS, r.getDataInfo().getProvenanceaction().getSchemeid());
assertEquals(ModelConstants.DNET_PROVENANCE_ACTIONS, r.getDataInfo().getProvenanceaction().getSchemename());
});
}
@Test
void testRelationsSemantics() throws Exception {
String inputPath = getClass()
.getResource(
"/eu/dnetlib/dhp/actionmanager/opencitations/inputFiles")
.getPath();
CreateActionSetSparkJob
.main(
new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Relation) aa.getPayload()));
tmp.foreach(r -> {
assertEquals("citation", r.getSubRelType());
assertEquals("resultResult", r.getRelType());
});
assertEquals(22, tmp.filter(r -> r.getRelClass().equals("Cites")).count());
assertEquals(22, tmp.filter(r -> r.getRelClass().equals("IsCitedBy")).count());
}
@Test
void testRelationsSourceTargetPrefix() throws Exception {
String inputPath = getClass()
.getResource(
"/eu/dnetlib/dhp/actionmanager/opencitations/inputFiles")
.getPath();
CreateActionSetSparkJob
.main(
new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Relation) aa.getPayload()));
tmp.foreach(r -> {
assertEquals("50|doi_________::", r.getSource().substring(0, 17));
assertEquals("50|doi_________::", r.getTarget().substring(0, 17));
});
}
@Test
void testRelationsSourceTargetCouple() throws Exception {
final String doi1 = "50|doi_________::"
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1007/s10854-015-3684-x"));
final String doi2 = "50|doi_________::"
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1111/j.1551-2916.2008.02408.x"));
final String doi3 = "50|doi_________::"
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1007/s10854-014-2114-9"));
final String doi4 = "50|doi_________::"
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1016/j.ceramint.2013.09.069"));
final String doi5 = "50|doi_________::"
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1007/s10854-009-9913-4"));
final String doi6 = "50|doi_________::"
+ IdentifierFactory.md5(CleaningFunctions.normalizePidValue("doi", "10.1016/0038-1098(72)90370-5"));
String inputPath = getClass()
.getResource(
"/eu/dnetlib/dhp/actionmanager/opencitations/inputFiles")
.getPath();
CreateActionSetSparkJob
.main(
new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Relation) aa.getPayload()));
JavaRDD<Relation> check = tmp.filter(r -> r.getSource().equals(doi1) || r.getTarget().equals(doi1));
assertEquals(10, check.count());
check.foreach(r -> {
if (r.getSource().equals(doi2) || r.getSource().equals(doi3) || r.getSource().equals(doi4) ||
r.getSource().equals(doi5) || r.getSource().equals(doi6)) {
assertEquals(ModelConstants.IS_CITED_BY, r.getRelClass());
assertEquals(doi1, r.getTarget());
}
});
assertEquals(5, check.filter(r -> r.getSource().equals(doi1)).count());
check.filter(r -> r.getSource().equals(doi1)).foreach(r -> assertEquals(ModelConstants.CITES, r.getRelClass()));
}
}

View File

@ -1,8 +1,7 @@
package eu.dnetlib.dhp.actionmanager.datacite
package eu.dnetlib.dhp.datacite
import com.fasterxml.jackson.databind.ObjectMapper
import com.fasterxml.jackson.databind.SerializationFeature
import com.fasterxml.jackson.databind.{ObjectMapper, SerializationFeature}
import eu.dnetlib.dhp.aggregation.AbstractVocabularyTest
import eu.dnetlib.dhp.schema.oaf.Oaf
import org.junit.jupiter.api.extension.ExtendWith

View File

@ -0,0 +1,86 @@
{"10.3390/s18072310": [{"id": "influence", "unit": [{"value": "7.5597134689e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "4.903880192", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "1.17977512835e-08", "key": "score"}]}]}
{"10.0000/096020199389707": [{"id": "influence", "unit": [{"value": "6.34596412687e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.641151896994", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "2.33375102921e-09", "key": "score"}]}]}
{"10.00000/jpmc.2017.106": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "5.39172290649e-09", "key": "score"}]}]}
{"10.0000/9781845416881": [{"id": "influence", "unit": [{"value": "5.96492048955e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "1.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "1.12641925838e-08", "key": "score"}]}]}
{"10.0000/anziamj.v0i0.266": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "3.76260934675e-10", "key": "score"}]}]}
{"10.0000/anziamj.v48i0.79": [{"id": "influence", "unit": [{"value": "6.93311506443e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.002176782336", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "1.7668105708e-09", "key": "score"}]}]}
{"10.0000/anziamj.v50i0.1472": [{"id": "influence", "unit": [{"value": "6.26777280882e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.406656", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "3.39745193285e-09", "key": "score"}]}]}
{"10.0000/cja5553": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "8.48190886761e-09", "key": "score"}]}]}
{"10.0000/czastest.16": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.01810569717e-09", "key": "score"}]}]}
{"10.0000/czastest.17": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "3.47956715615e-09", "key": "score"}]}]}
{"10.0000/czastest.18": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "3.47956715615e-09", "key": "score"}]}]}
{"10.0000/czastest.20": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.01810569717e-09", "key": "score"}]}]}
{"10.0000/czastest.21": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "3.47956715615e-09", "key": "score"}]}]}
{"10.0000/czastest.28": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "3.47956715615e-09", "key": "score"}]}]}
{"10.0000/czastest.60": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.65008652949e-09", "key": "score"}]}]}
{"10.0000/czt.2019.1.2.15": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "7.28336930301e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v4i02.36": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.01810569717e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v4i02.37": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.01810569717e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v4i02.38": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.01810569717e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v5i01.32": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.01810569717e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v6i01.24": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.01810569717e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v6i01.27": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.01810569717e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v6i02.41": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.01810569717e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v6i02.44": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.01810569717e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v7i01.40": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.01810569717e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v7i01.42": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.65008652949e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v7i01.47": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.65008652949e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v7i01.51": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.65008652949e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v7i01.52": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.65008652949e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v7i02.86": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.65008652949e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v7i02.88": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.65008652949e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v7i02.91": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.65008652949e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v8i01.129": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.65008652949e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v8i01.180": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "5.39172290649e-09", "key": "score"}]}]}
{"10.0000/geoekonomi.v8i01.87": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "4.65008652949e-09", "key": "score"}]}]}
{"10.0000/hbv2004w010": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "8.48190886761e-09", "key": "score"}]}]}
{"10.0000/hbv2101w001": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "9.88840807598e-09", "key": "score"}]}]}
{"10.0000/hbv2101w002": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "9.88840807598e-09", "key": "score"}]}]}
{"10.0000/hbv2101w003": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "9.88840807598e-09", "key": "score"}]}]}
{"10.0000/hbv2101w004": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "9.88840807598e-09", "key": "score"}]}]}
{"10.0000/hbv2101w005": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "9.88840807598e-09", "key": "score"}]}]}
{"10.0000/hbv2101w006": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "9.88840807598e-09", "key": "score"}]}]}
{"10.0000/hbv2101w007": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "9.88840807598e-09", "key": "score"}]}]}
{"10.0000/hbv2102w001": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "9.88840807598e-09", "key": "score"}]}]}
{"10.0000/hbv2102w010": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "9.88840807598e-09", "key": "score"}]}]}
{"10.0000/hoplos.v1i1.13207": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "6.26204125721e-09", "key": "score"}]}]}
{"10.0000/hoplos.v1i1.13208": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "5.39172290649e-09", "key": "score"}]}]}
{"10.0000/hoplos.v1i1.13209": [{"id": "influence", "unit": [{"value": "6.32078461509e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "1.6", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "8.3168486939e-09", "key": "score"}]}]}
{"10.0000/hoplos.v1i1.13210": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "6.26204125721e-09", "key": "score"}]}]}
{"10.0000/hoplos.v1i1.13211": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "5.39172290649e-09", "key": "score"}]}]}
{"10.0000/hoplos.v1i1.13212": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "5.39172290649e-09", "key": "score"}]}]}
{"10.0000/hoplos.v1i2.13231": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "6.26204125721e-09", "key": "score"}]}]}
{"10.0000/hoplos.v2i2.28782": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "6.26204125721e-09", "key": "score"}]}]}
{"10.0000/hoplos.v2i2.28783": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "6.26204125721e-09", "key": "score"}]}]}
{"10.0000/hoplos.v2i2.28784": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "6.26204125721e-09", "key": "score"}]}]}
{"10.0000/hoplos.v2i2.28786": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "6.26204125721e-09", "key": "score"}]}]}
{"10.0000/hoplos.v2i2.28787": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "6.26204125721e-09", "key": "score"}]}]}
{"10.0000/hoplos.v2i2.28788": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "6.26204125721e-09", "key": "score"}]}]}
{"10.0000/hoplos.v2i3.28234": [{"id": "influence", "unit": [{"value": "6.40470414877e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.6", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "7.89465099068e-09", "key": "score"}]}]}
{"10.0000/hoplos.v2i3.28236": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "6.26204125721e-09", "key": "score"}]}]}
{"10.0000/hoplos.v2i3.28238": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "6.26204125721e-09", "key": "score"}]}]}
{"10.0000/hoplos.v2i3.28239": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "6.26204125721e-09", "key": "score"}]}]}
{"10.0000/hoplos.v2i3.28242": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "6.26204125721e-09", "key": "score"}]}]}
{"10.0000/hoplos.v2i3.28243": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "6.26204125721e-09", "key": "score"}]}]}
{"10.0000/hoplos.v3i4.38186": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "7.28336930301e-09", "key": "score"}]}]}
{"10.0000/hoplos.v3i4.38187": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "7.28336930301e-09", "key": "score"}]}]}
{"10.0000/hoplos.v3i4.38190": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "7.28336930301e-09", "key": "score"}]}]}
{"10.0000/hoplos.v3i4.38207": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "7.28336930301e-09", "key": "score"}]}]}
{"10.0000/hoplos.v3i4.38209": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "7.28336930301e-09", "key": "score"}]}]}
{"10.0000/hoplos.v3i5.41163": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "7.28336930301e-09", "key": "score"}]}]}
{"10.0000/hoplos.v3i5.41166": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "7.28336930301e-09", "key": "score"}]}]}
{"10.0000/hoplos.v3i5.41167": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "7.28336930301e-09", "key": "score"}]}]}
{"10.0000/hoplos.v3i5.41168": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "7.28336930301e-09", "key": "score"}]}]}
{"10.0000/hoplos.v3i5.41229": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "8.48190886761e-09", "key": "score"}]}]}
{"10.0000/hoplos.v4i6.36360": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "8.48190886761e-09", "key": "score"}]}]}
{"10.0000/hoplos.v4i6.40796": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "8.48190886761e-09", "key": "score"}]}]}
{"10.0000/hoplos.v4i6.41153": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "8.48190886761e-09", "key": "score"}]}]}
{"10.0000/hoplos.v4i6.42511": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "8.48190886761e-09", "key": "score"}]}]}
{"10.0000/hoplos.v4i6.42555": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "8.48190886761e-09", "key": "score"}]}]}
{"10.0000/hoplos.v4i6.42752": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "8.48190886761e-09", "key": "score"}]}]}
{"10.0000/hoplos.v4i6.42768": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "8.48190886761e-09", "key": "score"}]}]}
{"10.0000/hoplos.v4i6.42795": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "8.48190886761e-09", "key": "score"}]}]}
{"10.0000/hoplos.v4i7.41295": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "8.48190886761e-09", "key": "score"}]}]}
{"10.0000/hoplos.v4i7.42830": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "8.48190886761e-09", "key": "score"}]}]}
{"10.0000/hoplos.v4i7.42861": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "8.48190886761e-09", "key": "score"}]}]}
{"10.0000/hoplos.v4i7.43096": [{"id": "influence", "unit": [{"value": "5.91019644836e-09", "key": "score"}]}, {"id": "popularity_alt", "unit": [{"value": "0.0", "key": "score"}]}, {"id": "popularity", "unit": [{"value": "8.48190886761e-09", "key": "score"}]}]}

View File

@ -0,0 +1,38 @@
{"doi":"10.3390/s18072310","level1":"engineering and technology","level2":"nano-technology","level3":"nanoscience & nanotechnology"}
{"doi":"10.1111/1365-2656.12831\u000210.17863/cam.24369","level1":"social sciences","level2":"psychology and cognitive sciences","level3":"NULL"}
{"doi":"10.3929/ethz-b-000187584\u000210.1002/chem.201701644","level1":"natural sciences","level2":"NULL","level3":"NULL"}
{"doi":"10.1080/01913123.2017.1367361","level1":"medical and health sciences","level2":"clinical medicine","level3":"oncology & carcinogenesis"}
{"doi":"10.1051/e3sconf/20199207011","level1":"natural sciences","level2":"earth and related environmental sciences","level3":"environmental sciences"}
{"doi":"10.1038/onc.2015.333","level1":"medical and health sciences","level2":"clinical medicine","level3":"oncology & carcinogenesis"}
{"doi":"10.1093/mnras/staa256","level1":"natural sciences","level2":"physical sciences","level3":"NULL"}
{"doi":"10.1016/j.jclepro.2018.07.166","level1":"engineering and technology","level2":"other engineering and technologies","level3":"building & construction"}
{"doi":"10.1103/physrevlett.125.037403","level1":"natural sciences","level2":"physical sciences","level3":"nuclear & particles physics"}
{"doi":"10.1080/03602532.2017.1316285","level1":"natural sciences","level2":"NULL","level3":"NULL"}
{"doi":"10.1001/jamanetworkopen.2019.1868","level1":"medical and health sciences","level2":"other medical science","level3":"health policy & services"}
{"doi":"10.1128/mra.00874-18","level1":"natural sciences","level2":"biological sciences","level3":"plant biology & botany"}
{"doi":"10.1016/j.nancom.2018.03.001","level1":"engineering and technology","level2":"NULL","level3":"NULL"}
{"doi":"10.1112/topo.12174","level1":"natural sciences","level2":"NULL","level3":"NULL"}
{"doi":"10.12688/wellcomeopenres.15846.1","level1":"medical and health sciences","level2":"health sciences","level3":"NULL"}
{"doi":"10.21468/scipostphys.3.1.001","level1":"natural sciences","level2":"physical sciences","level3":"NULL"}
{"doi":"10.1088/1741-4326/ab6c77","level1":"natural sciences","level2":"physical sciences","level3":"nuclear & particles physics"}
{"doi":"10.1109/tpwrs.2019.2944747","level1":"engineering and technology","level2":"electrical engineering, electronic engineering, information engineering","level3":"electrical & electronic engineering"}
{"doi":"10.1016/j.expthermflusci.2019.109994\u000210.17863/cam.46212","level1":"engineering and technology","level2":"mechanical engineering","level3":"mechanical engineering & transports"}
{"doi":"10.1109/tc.2018.2860012","level1":"engineering and technology","level2":"electrical engineering, electronic engineering, information engineering","level3":"computer hardware & architecture"}
{"doi":"10.1002/mma.6622","level1":"natural sciences","level2":"mathematics","level3":"numerical & computational mathematics"}
{"doi":"10.1051/radiopro/2020020","level1":"natural sciences","level2":"chemical sciences","level3":"NULL"}
{"doi":"10.1007/s12268-019-1003-4","level1":"medical and health sciences","level2":"basic medicine","level3":"NULL"}
{"doi":"10.3390/cancers12010236","level1":"medical and health sciences","level2":"health sciences","level3":"biochemistry & molecular biology"}
{"doi":"10.6084/m9.figshare.9912614\u000210.6084/m9.figshare.9912614.v1\u000210.1080/00268976.2019.1665199","level1":"natural sciences","level2":"chemical sciences","level3":"physical chemistry"}
{"doi":"10.1175/jpo-d-17-0239.1","level1":"natural sciences","level2":"biological sciences","level3":"marine biology & hydrobiology"}
{"doi":"10.1007/s13218-020-00674-7","level1":"engineering and technology","level2":"industrial biotechnology","level3":"industrial engineering & automation"}
{"doi":"10.1016/j.psyneuen.2016.02.003\u000210.1016/j.psyneuen.2016.02.00310.7892/boris.78886\u000210.7892/boris.78886","level1":"medical and health sciences","level2":"basic medicine","level3":"NULL"}
{"doi":"10.1109/ted.2018.2813542","level1":"engineering and technology","level2":"electrical engineering, electronic engineering, information engineering","level3":"electrical & electronic engineering"}
{"doi":"10.3989/scimar.04739.25a","level1":"natural sciences","level2":"biological sciences","level3":"NULL"}
{"doi":"10.3390/su12187503","level1":"natural sciences","level2":"earth and related environmental sciences","level3":"NULL"}
{"doi":"10.1016/j.ccell.2018.08.017","level1":"medical and health sciences","level2":"basic medicine","level3":"biochemistry & molecular biology"}
{"doi":"10.1103/physrevresearch.2.023322","level1":"natural sciences","level2":"physical sciences","level3":"nuclear & particles physics"}
{"doi":"10.1039/c8cp03234c","level1":"natural sciences","level2":"NULL","level3":"NULL"}
{"doi":"10.5281/zenodo.3696557\u000210.5281/zenodo.3696556\u000210.1109/jsac.2016.2545384","level1":"engineering and technology","level2":"electrical engineering, electronic engineering, information engineering","level3":"networking & telecommunications"}
{"doi":"10.1038/ng.3667\u000210.1038/ng.3667.\u000210.17615/tct6-4m26\u000210.17863/cam.15649","level1":"medical and health sciences","level2":"health sciences","level3":"genetics & heredity"}
{"doi":"10.1016/j.jclepro.2019.119065","level1":"engineering and technology","level2":"other engineering and technologies","level3":"building & construction"}
{"doi":"10.1111/pce.13392","level1":"agricultural and veterinary sciences","level2":"agriculture, forestry, and fisheries","level3":"agronomy & agriculture"}

View File

@ -0,0 +1,38 @@
dedup_wf_001::ddcc7a56fa13e49bcc59c6bdd19ad26c 10.3390/s18072310 engineering and technology nano-technology nanoscience & nanotechnology
dedup_wf_001::b76062d56e28224eac56111a4e1e5ecf 10.1111/1365-2656.1283110.17863/cam.24369 social sciences psychology and cognitive sciences NULL
dedup_wf_001::bb752acb8f403a25fa7851a302f7b7ac 10.3929/ethz-b-00018758410.1002/chem.201701644 natural sciences NULL NULL
dedup_wf_001::2f1435a9201ecf5cbbcb12c9b2d971cd 10.1080/01913123.2017.1367361 medical and health sciences clinical medicine oncology & carcinogenesis
dedup_wf_001::fc9e47ec16c67b101724320d4b030514 10.1051/e3sconf/20199207011 natural sciences earth and related environmental sciences environmental sciences
dedup_wf_001::caa1e5b4de387cb31751552f4f0f5d72 10.1038/onc.2015.333 medical and health sciences clinical medicine oncology & carcinogenesis
dedup_wf_001::c2a98df5637d69bf0524eaf40fe6bf11 10.1093/mnras/staa256 natural sciences physical sciences NULL
dedup_wf_001::c221262bdc77cbfd59859a402f0e3991 10.1016/j.jclepro.2018.07.166 engineering and technology other engineering and technologies building & construction
doiboost____::d56d9dc21f317b3e009d5b6c8ea87212 10.1103/physrevlett.125.037403 natural sciences physical sciences nuclear & particles physics
dedup_wf_001::8a7269c8ee6470b2fb4fd384bc389e08 10.1080/03602532.2017.1316285 natural sciences NULL NULL
dedup_wf_001::28342ebbc19833e4e1f4a2b23cf5ee20 10.1001/jamanetworkopen.2019.1868 medical and health sciences other medical science health policy & services
dedup_wf_001::c1e1daf2b55dd9ec8e1c7c7458bbc7bc 10.1128/mra.00874-18 natural sciences biological sciences plant biology & botany
dedup_wf_001::a2ef4a2720c71907180750e5871298ef 10.1016/j.nancom.2018.03.001 engineering and technology NULL NULL
dedup_wf_001::676f46a31519e83a89efcb1c626286fb 10.1112/topo.12174 natural sciences NULL NULL
dedup_wf_001::6f2761642f1e39313388e2c4060657dd 10.12688/wellcomeopenres.15846.1 medical and health sciences health sciences NULL
dedup_wf_001::e414c1dec599521a9635a60de0f6755b 10.21468/scipostphys.3.1.001 natural sciences physical sciences NULL
dedup_wf_001::f3395fe0f330164ea424dc61c86c9a3d 10.1088/1741-4326/ab6c77 natural sciences physical sciences nuclear & particles physics
dedup_wf_001::a4f32a97a783117012f1de11797e73f2 10.1109/tpwrs.2019.2944747 engineering and technology electrical engineering, electronic engineering, information engineering electrical & electronic engineering
dedup_wf_001::313ae1cd083ae1696d12dd1909f97df8 10.1016/j.expthermflusci.2019.10999410.17863/cam.46212 engineering and technology mechanical engineering mechanical engineering & transports
dedup_wf_001::2a300a7d3ca7347791ebcef986bc0682 10.1109/tc.2018.2860012 engineering and technology electrical engineering, electronic engineering, information engineering computer hardware & architecture
doiboost____::5b79bd7bd9f87361b4a4abc3cbb2df75 10.1002/mma.6622 natural sciences mathematics numerical & computational mathematics
dedup_wf_001::6a3f61f217a2519fbaddea1094e3bfc2 10.1051/radiopro/2020020 natural sciences chemical sciences NULL
dedup_wf_001::a3f0430309a639f4234a0e57b10f2dee 10.1007/s12268-019-1003-4 medical and health sciences basic medicine NULL
dedup_wf_001::b6b8a3a1cccbee459cf3343485efdb12 10.3390/cancers12010236 medical and health sciences health sciences biochemistry & molecular biology
dedup_wf_001::dd06ee7974730e7b09a4f03c83b3f9bd 10.6084/m9.figshare.991261410.6084/m9.figshare.9912614.v110.1080/00268976.2019.1665199 natural sciences chemical sciences physical chemistry
dedup_wf_001::027c78bef6f972b5e26dfea55d30fbe3 10.1175/jpo-d-17-0239.1 natural sciences biological sciences marine biology & hydrobiology
dedup_wf_001::43edc179aa9e1fbaf582c5203b18b519 10.1007/s13218-020-00674-7 engineering and technology industrial biotechnology industrial engineering & automation
dedup_wf_001::e7770e11cd6eb514bb52c07b5a8a80f0 10.1016/j.psyneuen.2016.02.00310.1016/j.psyneuen.2016.02.00310.7892/boris.7888610.7892/boris.78886 medical and health sciences basic medicine NULL
dedup_wf_001::80bc15d69bdc589149631f3439dde5aa 10.1109/ted.2018.2813542 engineering and technology electrical engineering, electronic engineering, information engineering electrical & electronic engineering
dedup_wf_001::42c1cfa33e7872944b920cff90f4d99e 10.3989/scimar.04739.25a natural sciences biological sciences NULL
dedup_wf_001::9bacdbbaa9da3658b7243d5de8e3ce14 10.3390/su12187503 natural sciences earth and related environmental sciences NULL
dedup_wf_001::59e43d3527dcfecb6097fbd5740c8950 10.1016/j.ccell.2018.08.017 medical and health sciences basic medicine biochemistry & molecular biology
doiboost____::e024d1b738df3b24bc58fa0228542571 10.1103/physrevresearch.2.023322 natural sciences physical sciences nuclear & particles physics
dedup_wf_001::66e9a3237fa8178886d26d3c2d5b9e66 10.1039/c8cp03234c natural sciences NULL NULL
dedup_wf_001::83737ab4205bae751571bb3b166efa18 10.5281/zenodo.369655710.5281/zenodo.369655610.1109/jsac.2016.2545384 engineering and technology electrical engineering, electronic engineering, information engineering networking & telecommunications
dedup_wf_001::e3f892db413a689e572dd256acad55fe 10.1038/ng.366710.1038/ng.3667.10.17615/tct6-4m2610.17863/cam.15649 medical and health sciences health sciences genetics & heredity
dedup_wf_001::14ba594e8fd081847bc3f50f56335003 10.1016/j.jclepro.2019.119065 engineering and technology other engineering and technologies building & construction
dedup_wf_001::08ac7b33a41bcea2d055ecd8585d632e 10.1111/pce.13392 agricultural and veterinary sciences agriculture, forestry, and fisheries agronomy & agriculture
1 dedup_wf_001::ddcc7a56fa13e49bcc59c6bdd19ad26c 10.3390/s18072310 engineering and technology nano-technology nanoscience & nanotechnology
2 dedup_wf_001::b76062d56e28224eac56111a4e1e5ecf 10.1111/1365-2656.1283110.17863/cam.24369 social sciences psychology and cognitive sciences NULL
3 dedup_wf_001::bb752acb8f403a25fa7851a302f7b7ac 10.3929/ethz-b-00018758410.1002/chem.201701644 natural sciences NULL NULL
4 dedup_wf_001::2f1435a9201ecf5cbbcb12c9b2d971cd 10.1080/01913123.2017.1367361 medical and health sciences clinical medicine oncology & carcinogenesis
5 dedup_wf_001::fc9e47ec16c67b101724320d4b030514 10.1051/e3sconf/20199207011 natural sciences earth and related environmental sciences environmental sciences
6 dedup_wf_001::caa1e5b4de387cb31751552f4f0f5d72 10.1038/onc.2015.333 medical and health sciences clinical medicine oncology & carcinogenesis
7 dedup_wf_001::c2a98df5637d69bf0524eaf40fe6bf11 10.1093/mnras/staa256 natural sciences physical sciences NULL
8 dedup_wf_001::c221262bdc77cbfd59859a402f0e3991 10.1016/j.jclepro.2018.07.166 engineering and technology other engineering and technologies building & construction
9 doiboost____::d56d9dc21f317b3e009d5b6c8ea87212 10.1103/physrevlett.125.037403 natural sciences physical sciences nuclear & particles physics
10 dedup_wf_001::8a7269c8ee6470b2fb4fd384bc389e08 10.1080/03602532.2017.1316285 natural sciences NULL NULL
11 dedup_wf_001::28342ebbc19833e4e1f4a2b23cf5ee20 10.1001/jamanetworkopen.2019.1868 medical and health sciences other medical science health policy & services
12 dedup_wf_001::c1e1daf2b55dd9ec8e1c7c7458bbc7bc 10.1128/mra.00874-18 natural sciences biological sciences plant biology & botany
13 dedup_wf_001::a2ef4a2720c71907180750e5871298ef 10.1016/j.nancom.2018.03.001 engineering and technology NULL NULL
14 dedup_wf_001::676f46a31519e83a89efcb1c626286fb 10.1112/topo.12174 natural sciences NULL NULL
15 dedup_wf_001::6f2761642f1e39313388e2c4060657dd 10.12688/wellcomeopenres.15846.1 medical and health sciences health sciences NULL
16 dedup_wf_001::e414c1dec599521a9635a60de0f6755b 10.21468/scipostphys.3.1.001 natural sciences physical sciences NULL
17 dedup_wf_001::f3395fe0f330164ea424dc61c86c9a3d 10.1088/1741-4326/ab6c77 natural sciences physical sciences nuclear & particles physics
18 dedup_wf_001::a4f32a97a783117012f1de11797e73f2 10.1109/tpwrs.2019.2944747 engineering and technology electrical engineering, electronic engineering, information engineering electrical & electronic engineering
19 dedup_wf_001::313ae1cd083ae1696d12dd1909f97df8 10.1016/j.expthermflusci.2019.10999410.17863/cam.46212 engineering and technology mechanical engineering mechanical engineering & transports
20 dedup_wf_001::2a300a7d3ca7347791ebcef986bc0682 10.1109/tc.2018.2860012 engineering and technology electrical engineering, electronic engineering, information engineering computer hardware & architecture
21 doiboost____::5b79bd7bd9f87361b4a4abc3cbb2df75 10.1002/mma.6622 natural sciences mathematics numerical & computational mathematics
22 dedup_wf_001::6a3f61f217a2519fbaddea1094e3bfc2 10.1051/radiopro/2020020 natural sciences chemical sciences NULL
23 dedup_wf_001::a3f0430309a639f4234a0e57b10f2dee 10.1007/s12268-019-1003-4 medical and health sciences basic medicine NULL
24 dedup_wf_001::b6b8a3a1cccbee459cf3343485efdb12 10.3390/cancers12010236 medical and health sciences health sciences biochemistry & molecular biology
25 dedup_wf_001::dd06ee7974730e7b09a4f03c83b3f9bd 10.6084/m9.figshare.991261410.6084/m9.figshare.9912614.v110.1080/00268976.2019.1665199 natural sciences chemical sciences physical chemistry
26 dedup_wf_001::027c78bef6f972b5e26dfea55d30fbe3 10.1175/jpo-d-17-0239.1 natural sciences biological sciences marine biology & hydrobiology
27 dedup_wf_001::43edc179aa9e1fbaf582c5203b18b519 10.1007/s13218-020-00674-7 engineering and technology industrial biotechnology industrial engineering & automation
28 dedup_wf_001::e7770e11cd6eb514bb52c07b5a8a80f0 10.1016/j.psyneuen.2016.02.00310.1016/j.psyneuen.2016.02.00310.7892/boris.7888610.7892/boris.78886 medical and health sciences basic medicine NULL
29 dedup_wf_001::80bc15d69bdc589149631f3439dde5aa 10.1109/ted.2018.2813542 engineering and technology electrical engineering, electronic engineering, information engineering electrical & electronic engineering
30 dedup_wf_001::42c1cfa33e7872944b920cff90f4d99e 10.3989/scimar.04739.25a natural sciences biological sciences NULL
31 dedup_wf_001::9bacdbbaa9da3658b7243d5de8e3ce14 10.3390/su12187503 natural sciences earth and related environmental sciences NULL
32 dedup_wf_001::59e43d3527dcfecb6097fbd5740c8950 10.1016/j.ccell.2018.08.017 medical and health sciences basic medicine biochemistry & molecular biology
33 doiboost____::e024d1b738df3b24bc58fa0228542571 10.1103/physrevresearch.2.023322 natural sciences physical sciences nuclear & particles physics
34 dedup_wf_001::66e9a3237fa8178886d26d3c2d5b9e66 10.1039/c8cp03234c natural sciences NULL NULL
35 dedup_wf_001::83737ab4205bae751571bb3b166efa18 10.5281/zenodo.369655710.5281/zenodo.369655610.1109/jsac.2016.2545384 engineering and technology electrical engineering, electronic engineering, information engineering networking & telecommunications
36 dedup_wf_001::e3f892db413a689e572dd256acad55fe 10.1038/ng.366710.1038/ng.3667.10.17615/tct6-4m2610.17863/cam.15649 medical and health sciences health sciences genetics & heredity
37 dedup_wf_001::14ba594e8fd081847bc3f50f56335003 10.1016/j.jclepro.2019.119065 engineering and technology other engineering and technologies building & construction
38 dedup_wf_001::08ac7b33a41bcea2d055ecd8585d632e 10.1111/pce.13392 agricultural and veterinary sciences agriculture, forestry, and fisheries agronomy & agriculture

View File

@ -0,0 +1,8 @@
oci,citing,cited,creation,timespan,journal_sc,author_sc
02001000007362801000805046300010563030608046333-0200101010136193701050501630209010637020000083700020400083733,10.1007/s10854-015-3684-x,10.1111/j.1551-2916.2008.02408.x,2015-09-01,P7Y2M,no,no
02001000007362801000805046300010563030608046333-02001000007362801000805046300010463020101046309,10.1007/s10854-015-3684-x,10.1007/s10854-014-2114-9,2015-09-01,P1Y2M4D,yes,no
02001000007362801000805046300010563030608046333-020010001063619371214271022182329370200010337000937000609,10.1007/s10854-015-3684-x,10.1016/j.ceramint.2013.09.069,2015-09-01,P1Y6M,no,no
02001000007362801000805046300010563030608046333-02001000007362801000805046300000963090901036304,10.1007/s10854-015-3684-x,10.1007/s10854-009-9913-4,2015-09-01,P6Y3M10D,yes,no
02001000007362801000805046300010563030608046333-02001000106360000030863010009085807025909000307006305,10.1007/s10854-015-3684-x,10.1016/0038-1098(72)90370-5,2015-09-01,P43Y8M,no,no
02001000007362801000805046300010563030608056309-02001000106361937281010370200010437000937000308,10.1007/s10854-015-3685-9,10.1016/j.saa.2014.09.038,2015-09-03,P0Y7M,no,no
02001000007362801000805046300010563030608056309-0200100010636193722102912171027370200010537000437000106,10.1007/s10854-015-3685-9,10.1016/j.matchar.2015.04.016,2015-09-03,P0Y2M,no,no

View File

@ -0,0 +1,8 @@
oci,citing,cited,creation,timespan,journal_sc,author_sc
02001000308362804010509076300010963000003086301-0200100020936020001003227000009010004,10.1038/s41597-019-0038-1,10.1029/2010wr009104,2019-04-15,P8Y1M,no,no
02001000308362804010509076300010963000003086301-0200100010636280103060463080105025800015900000006006303,10.1038/s41597-019-0038-1,10.1016/s1364-8152(01)00060-3,2019-04-15,P17Y3M,no,no
02001000308362804010509076300010963000003086301-02001000007362800000407076300010063000401066333,10.1038/s41597-019-0038-1,10.1007/s00477-010-0416-x,2019-04-15,P8Y9M6D,no,no
02001000308362804010509076300010963000003086301-02001000007362800000700046300010363000905016308,10.1038/s41597-019-0038-1,10.1007/s00704-013-0951-8,2019-04-15,P5Y9M23D,no,no
02001000308362804010509076300010963000003086301-02001000002361924123705070707,10.1038/s41597-019-0038-1,10.1002/joc.5777,2019-04-15,P0Y8M1D,no,no
02001000308362804010509076300010963000003086301-02005010904361714282863020263040504076302000108,10.1038/s41597-019-0038-1,10.5194/hess-22-4547-2018,2019-04-15,P0Y7M18D,no,no
02001000308362804010509076300010963000003086301-02001000002361924123703050404,10.1038/s41597-019-0038-1,10.1002/joc.3544,2019-04-15,P6Y9M6D,no,no

View File

@ -0,0 +1,9 @@
oci,citing,cited,creation,timespan,journal_sc,author_sc
0200100000236090708010101090307000202023727141528-020050302063600040000010307,10.1002/9781119370222.refs,10.5326/0400137,2020-06-22,P16Y3M,no,no
0200100000236090708010101090307000202023727141528-0200101010136193701050302630905003337020000073700000301093733,10.1002/9781119370222.refs,10.1111/j.1532-950x.2007.00319.x,2020-06-22,P12Y8M,no,no
0200100000236090708010101090307000202023727141528-0200101010136312830370102030509,10.1002/9781119370222.refs,10.1111/vsu.12359,2020-06-22,P4Y10M29D,no,no
0200100000236090708010101090307000202023727141528-020050302063600030900020904,10.1002/9781119370222.refs,10.5326/0390294,2020-06-22,P17Y1M,no,no
0200100000236090708010101090307000202023727141528-020050302063600040200030701,10.1002/9781119370222.refs,10.5326/0420371,2020-06-22,P13Y9M,no,no
0200100000236090708010101090307000202023727141528-0200101010136193701050302630905003337020001033701020000003733,10.1002/9781119370222.refs,10.1111/j.1532-950x.2013.12000.x,2020-06-22,P7Y2M,no,no
0200100000236090708010101090307000202023727141528-020010008003600000408000106093702000006370306070200,10.1002/9781119370222.refs,10.1080/00480169.2006.36720,2020-06-22,P13Y6M,no,no
0200100000236090708010101090307000202023727141528-0200101010136193701070501630008010337020000063700000003033733,10.1002/9781119370222.refs,10.1111/j.1751-0813.2006.00033.x,2020-06-22,P13Y8M,no,no

View File

@ -89,7 +89,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=7680
--conf spark.sql.shuffle.partitions=15000
</spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--o</arg><arg>${graphOutputPath}</arg>
@ -114,7 +114,7 @@
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=7680
--conf spark.sql.shuffle.partitions=15000
</spark-opts>
<arg>--graphInputPath</arg><arg>${graphBasePath}</arg>
<arg>--outputPath</arg><arg>${workingPath}/grouped_entities</arg>

View File

@ -70,7 +70,7 @@ case object Crossref2Oaf {
"reference-book" -> "0002 Book",
"monograph" -> "0002 Book",
"journal-article" -> "0001 Article",
"dissertation" -> "0006 Doctoral thesis",
"dissertation" -> "0044 Thesis",
"other" -> "0038 Other literature type",
"peer-review" -> "0015 Review",
"proceedings" -> "0004 Conference object",
@ -206,11 +206,16 @@ case object Crossref2Oaf {
else {
instance.setDateofacceptance(asField(createdDate.getValue))
}
val s: String = (json \ "URL").extract[String]
val links: List[String] = ((for {JString(url) <- json \ "link" \ "URL"} yield url) ::: List(s)).filter(p => p != null).distinct
if (links.nonEmpty) {
instance.setUrl(links.asJava)
}
val s: List[String] = List("https://doi.org/" + doi)
// val links: List[String] = ((for {JString(url) <- json \ "link" \ "URL"} yield url) ::: List(s)).filter(p => p != null && p.toLowerCase().contains(doi.toLowerCase())).distinct
// if (links.nonEmpty) {
// instance.setUrl(links.asJava)
// }
if(s.nonEmpty)
{
instance.setUrl(s.asJava)
}
result.setInstance(List(instance).asJava)
//IMPORTANT

View File

@ -111,26 +111,9 @@ object SparkProcessMAG {
.map(item => ConversionUtil.updatePubsWithConferenceInfo(item))
.write
.mode(SaveMode.Overwrite)
.save(s"$workingPath/merge_step_2_conference")
magPubs= spark.read.load(s"$workingPath/merge_step_2_conference").as[Publication]
.map(p => (ConversionUtil.extractMagIdentifier(p.getOriginalId.asScala), p)).as[(String, Publication)]
val paperUrlDataset = spark.read.load(s"$sourcePath/PaperUrls").as[MagPaperUrl].groupBy("PaperId").agg(collect_list(struct("sourceUrl")).as("instances")).as[MagUrl]
logger.info("Phase 5) enrich publication with URL and Instances")
magPubs.joinWith(paperUrlDataset, col("_1").equalTo(paperUrlDataset("PaperId")), "left")
.map { a: ((String, Publication), MagUrl) => ConversionUtil.addInstances((a._1._2, a._2)) }
.write.mode(SaveMode.Overwrite)
.save(s"$workingPath/merge_step_3")
// logger.info("Phase 6) Enrich Publication with description")
// val pa = spark.read.load(s"${parser.get("sourcePath")}/PaperAbstractsInvertedIndex").as[MagPaperAbstract]
// pa.map(ConversionUtil.transformPaperAbstract).write.mode(SaveMode.Overwrite).save(s"${parser.get("targetPath")}/PaperAbstract")
val paperAbstract = spark.read.load((s"$workingPath/PaperAbstract")).as[MagPaperAbstract]
@ -162,12 +145,14 @@ object SparkProcessMAG {
.write.mode(SaveMode.Overwrite)
.save(s"$workingPath/mag_publication")
spark.read.load(s"$workingPath/mag_publication").as[Publication]
.filter(p => p.getId == null)
.groupByKey(p => p.getId)
.reduceGroups((a:Publication, b:Publication) => ConversionUtil.mergePublication(a,b))
.map(_._2)
.write.mode(SaveMode.Overwrite).save(s"$targetPath/magPublication")
val s:RDD[Publication] = spark.read.load(s"$workingPath/mag_publication").as[Publication]
.map(p=>Tuple2(p.getId, p)).rdd.reduceByKey((a:Publication, b:Publication) => ConversionUtil.mergePublication(a,b))
.map(_._2)
spark.createDataset(s).as[Publication].write.mode(SaveMode.Overwrite).save(s"$targetPath/magPublication")
}
}

View File

@ -612,4 +612,26 @@ class CrossrefMappingTest {
}
@Test
def testMultipleURLs() :Unit = {
val json = Source.fromInputStream(getClass.getResourceAsStream("multiple_urls.json")).mkString
assertNotNull(json)
assertFalse(json.isEmpty);
val resultList: List[Oaf] = Crossref2Oaf.convert(json)
assertTrue(resultList.nonEmpty)
val item : Result = resultList.filter(p => p.isInstanceOf[Result]).head.asInstanceOf[Result]
assertEquals(1, item.getInstance().size())
assertEquals(1, item.getInstance().get(0).getUrl().size())
assertEquals("https://doi.org/10.1016/j.jas.2019.105013", item.getInstance().get(0).getUrl().get(0))
//println(mapper.writeValueAsString(item))
}
}

View File

@ -0,0 +1,614 @@
{
"indexed": {
"date-parts": [
[
2021,
10,
31
]
],
"date-time": "2021-10-31T15:48:01Z",
"timestamp": 1635695281393
},
"reference-count": 39,
"publisher": "Elsevier BV",
"license": [
{
"start": {
"date-parts": [
[
2019,
12,
1
]
],
"date-time": "2019-12-01T00:00:00Z",
"timestamp": 1575158400000
},
"content-version": "tdm",
"delay-in-days": 0,
"URL": "https://www.elsevier.com/tdm/userlicense/1.0/"
},
{
"start": {
"date-parts": [
[
2019,
9,
13
]
],
"date-time": "2019-09-13T00:00:00Z",
"timestamp": 1568332800000
},
"content-version": "vor",
"delay-in-days": 0,
"URL": "http://creativecommons.org/licenses/by/4.0/"
}
],
"funder": [
{
"DOI": "10.13039/100001182",
"name": "INSTAP",
"doi-asserted-by": "publisher"
},
{
"DOI": "10.13039/100014440",
"name": "Ministry of Science, Innovation and Universities",
"doi-asserted-by": "publisher",
"award": [
"RYC-2016-19637"
]
},
{
"DOI": "10.13039/100010661",
"name": "European Unions Horizon 2020",
"doi-asserted-by": "publisher",
"award": [
"746446"
]
}
],
"content-domain": {
"domain": [
"elsevier.com",
"sciencedirect.com"
],
"crossmark-restriction": true
},
"short-container-title": [
"Journal of Archaeological Science"
],
"published-print": {
"date-parts": [
[
2019,
12
]
]
},
"DOI": "10.1016/j.jas.2019.105013",
"type": "journal-article",
"created": {
"date-parts": [
[
2019,
9,
25
]
],
"date-time": "2019-09-25T20:05:08Z",
"timestamp": 1569441908000
},
"page": "105013",
"update-policy": "http://dx.doi.org/10.1016/elsevier_cm_policy",
"source": "Crossref",
"is-referenced-by-count": 21,
"title": [
"A brave new world for archaeological survey: Automated machine learning-based potsherd detection using high-resolution drone imagery"
],
"prefix": "10.1016",
"volume": "112",
"author": [
{
"given": "H.A.",
"family": "Orengo",
"sequence": "first",
"affiliation": [
]
},
{
"given": "A.",
"family": "Garcia-Molsosa",
"sequence": "additional",
"affiliation": [
]
}
],
"member": "78",
"reference": [
{
"key": "10.1016/j.jas.2019.105013_bib1",
"doi-asserted-by": "crossref",
"first-page": "85",
"DOI": "10.1080/17538947.2016.1250829",
"article-title": "Remote sensing heritage in a petabyte-scale: satellite data and heritage Earth Engine© applications",
"volume": "10",
"author": "Agapiou",
"year": "2017",
"journal-title": "Int. J. Digit. Earth"
},
{
"key": "10.1016/j.jas.2019.105013_bib2",
"series-title": "Extracting Meaning from Ploughsoil Assemblages",
"first-page": "1",
"article-title": "Extracting meaning from ploughsoil assemblages: assessments of the past, strategies for the future",
"author": "Alcock",
"year": "2000"
},
{
"key": "10.1016/j.jas.2019.105013_bib3",
"series-title": "Side-by-Side Survey. Comparative Regional Studies in the Mediterranean World",
"first-page": "1",
"article-title": "Introduction",
"author": "Alcock",
"year": "2004"
},
{
"key": "10.1016/j.jas.2019.105013_bib4",
"doi-asserted-by": "crossref",
"first-page": "93",
"DOI": "10.1111/j.1538-4632.1995.tb00338.x",
"article-title": "Local indicators of spatial association—LISA",
"volume": "27",
"author": "Anselin",
"year": "1995",
"journal-title": "Geogr. Anal."
},
{
"key": "10.1016/j.jas.2019.105013_bib5",
"series-title": "Archaeological Survey",
"author": "Banning",
"year": "2002"
},
{
"issue": "1/2",
"key": "10.1016/j.jas.2019.105013_bib6",
"doi-asserted-by": "crossref",
"first-page": "123",
"DOI": "10.2307/3181488",
"article-title": "GIS, archaeological survey and landscape archaeology on the island of Kythera, Greece",
"volume": "29",
"author": "Bevan",
"year": "2004",
"journal-title": "J. Field Archaeol."
},
{
"issue": "1",
"key": "10.1016/j.jas.2019.105013_bib8",
"doi-asserted-by": "crossref",
"first-page": "5",
"DOI": "10.1023/A:1010933404324",
"article-title": "Random forests",
"volume": "45",
"author": "Breiman",
"year": "2001",
"journal-title": "Mach. Learn."
},
{
"key": "10.1016/j.jas.2019.105013_bib9",
"series-title": "Sampling in Contemporary British Archaeology",
"author": "Cherry",
"year": "1978"
},
{
"issue": "3",
"key": "10.1016/j.jas.2019.105013_bib10",
"doi-asserted-by": "crossref",
"first-page": "273",
"DOI": "10.1016/0734-189X(84)90197-X",
"article-title": "Segmentation of a high-resolution urban scene using texture operators",
"volume": "25",
"author": "Conners",
"year": "1984",
"journal-title": "Comput. Vis. Graph Image Process"
},
{
"key": "10.1016/j.jas.2019.105013_bib11",
"first-page": "31",
"article-title": "Old land surfaces and modern ploughsoil: implications of recent work at Maxey, Cambridgeshire",
"volume": "2",
"author": "Crowther",
"year": "1983",
"journal-title": "Scott. Archaeol. Rev."
},
{
"key": "10.1016/j.jas.2019.105013_bib12",
"series-title": "Settlement Pattern Studies in the Americas: Fifty Years since Virú",
"first-page": "203",
"article-title": "Conclusions: the settlement pattern concept from an Americanist perspective",
"author": "Fish",
"year": "1999"
},
{
"key": "10.1016/j.jas.2019.105013_bib13",
"doi-asserted-by": "crossref",
"first-page": "21",
"DOI": "10.3390/geosciences9010021",
"article-title": "Remote sensing and historical morphodynamics of alluvial plains. The 1909 indus flood and the city of Dera Gazhi Khan (province of Punjab, Pakistan)",
"volume": "9",
"author": "Garcia",
"year": "2019",
"journal-title": "Geosciences"
},
{
"key": "10.1016/j.jas.2019.105013_bib14",
"unstructured": "Georgiadis, M.; Garcia-Molsosa, A.; Orengo, H.A.; Kefalidou, E. and Kallintzi, K. In Preparation. APAX Project 2015-2018: A Preliminary Report. (Hesperia)."
},
{
"key": "10.1016/j.jas.2019.105013_bib15",
"series-title": "Geographical Information Systems and Landscape Archaeology",
"first-page": "35",
"article-title": "Regional survey and GIS: the boeotia project",
"author": "Gillings",
"year": "1999"
},
{
"key": "10.1016/j.jas.2019.105013_bib16",
"doi-asserted-by": "crossref",
"first-page": "18",
"DOI": "10.1016/j.rse.2017.06.031",
"article-title": "Google Earth engine: planetary-scale geospatial analysis for everyone",
"volume": "202",
"author": "Gorelick",
"year": "2017",
"journal-title": "Remote Sens. Environ."
},
{
"issue": "107",
"key": "10.1016/j.jas.2019.105013_bib17",
"doi-asserted-by": "crossref",
"first-page": "177",
"DOI": "10.1111/j.0031-868X.2004.00278.x",
"article-title": "Photogrammetric reconstruction of the great buddha of Bamiyan, Afghanistan",
"volume": "19",
"author": "Grün",
"year": "2004",
"journal-title": "Photogramm. Rec."
},
{
"issue": "6",
"key": "10.1016/j.jas.2019.105013_bib18",
"doi-asserted-by": "crossref",
"first-page": "610",
"DOI": "10.1109/TSMC.1973.4309314",
"article-title": "Textural features for image classification",
"author": "Haralick",
"year": "1973",
"journal-title": "IEEE Trans. Syst., Man, Cybernet., SMC-3"
},
{
"key": "10.1016/j.jas.2019.105013_bib19",
"doi-asserted-by": "crossref",
"first-page": "76",
"DOI": "10.1558/jmea.v14i1.76",
"article-title": "Excavating to excess? Implications of the last decade of archaeology in Israel",
"volume": "14",
"author": "Kletter",
"year": "2001",
"journal-title": "J. Mediterr. Archaeol."
},
{
"key": "10.1016/j.jas.2019.105013_bib20",
"first-page": "299",
"article-title": "Testing Google Earth Engine for the automatic identification and vectorization of archaeological features: a case study from Faynan, Jordan",
"volume": "15",
"author": "Liss",
"year": "2017",
"journal-title": "J. Archaeol. Sci.: Report"
},
{
"key": "10.1016/j.jas.2019.105013_bib21",
"series-title": "Geographical Information Systems and Landscape Archaeology",
"first-page": "55",
"article-title": "Towards a methodology for modelling surface survey data: the sangro valley project",
"author": "Lock",
"year": "1999"
},
{
"key": "10.1016/j.jas.2019.105013_bib22",
"series-title": "Extracting Meaning from Ploughsoil Assemblages",
"first-page": "5",
"article-title": "Methods of collection recording and quantification",
"author": "Mattingly",
"year": "2000"
},
{
"issue": "14",
"key": "10.1016/j.jas.2019.105013_bib23",
"doi-asserted-by": "crossref",
"first-page": "E778",
"DOI": "10.1073/pnas.1115472109",
"article-title": "Mapping patterns of long-term settlement in Northern Mesopotamia at a large scale",
"volume": "109",
"author": "Menze",
"year": "2012",
"journal-title": "Proc. Natl. Acad. Sci."
},
{
"key": "10.1016/j.jas.2019.105013_bib24",
"doi-asserted-by": "crossref",
"first-page": "80",
"DOI": "10.1016/j.jas.2015.04.002",
"article-title": "A supervised machine-learning approach towards geochemical predictive modelling in archaeology",
"volume": "59",
"author": "Oonk",
"year": "2015",
"journal-title": "J. Archaeol. Sci."
},
{
"key": "10.1016/j.jas.2019.105013_bib25",
"doi-asserted-by": "crossref",
"first-page": "49",
"DOI": "10.1016/j.isprsjprs.2012.07.005",
"article-title": "Combining terrestrial stereophotogrammetry, DGPS and GIS-based 3D voxel modelling in the volumetric recording of archaeological features",
"volume": "76",
"author": "Orengo",
"year": "2013",
"journal-title": "ISPRS J. Photogrammetry Remote Sens."
},
{
"key": "10.1016/j.jas.2019.105013_bib26",
"doi-asserted-by": "crossref",
"first-page": "100",
"DOI": "10.1016/j.jas.2015.10.008",
"article-title": "Photogrammetric re-discovery of the Eastern Thessalian hidden long-term landscapes",
"volume": "64",
"author": "Orengo",
"year": "2015",
"journal-title": "J. Archaeol. Sci."
},
{
"issue": "3",
"key": "10.1016/j.jas.2019.105013_bib27",
"doi-asserted-by": "crossref",
"first-page": "479",
"DOI": "10.3764/aja.122.3.0479",
"article-title": "Towards a definition of Minoan agro-pastoral landscapes: results of the survey at Palaikastro (Crete)",
"volume": "122",
"author": "Orengo",
"year": "2018",
"journal-title": "Am. J. Archaeol."
},
{
"issue": "7",
"key": "10.1016/j.jas.2019.105013_bib28",
"doi-asserted-by": "crossref",
"first-page": "735",
"DOI": "10.3390/rs9070735",
"article-title": "Large-scale, multi-temporal remote sensing of palaeo-river networks: a case study from Northwest India and its implications for the Indus civilisation",
"volume": "9",
"author": "Orengo",
"year": "2017",
"journal-title": "Remote Sens."
},
{
"key": "10.1016/j.jas.2019.105013_bib29",
"doi-asserted-by": "crossref",
"first-page": "1361",
"DOI": "10.1002/esp.4317",
"article-title": "Multi-scale relief model (MSRM): a new algorithm for the visualization of subtle topographic change of variable size in digital elevation models",
"volume": "43",
"author": "Orengo",
"year": "2018",
"journal-title": "Earth Surf. Process. Landforms"
},
{
"key": "10.1016/j.jas.2019.105013_bib30",
"series-title": "Submitted to Proceedings of the National Academy of Sciences",
"article-title": "Living on the edge of the desert: automated detection of archaeological mounds in Cholistan (Pakistan) using machine learning classification of multi-sensor multi-temporal satellite data",
"author": "Orengo",
"year": "2019"
},
{
"key": "10.1016/j.jas.2019.105013_bib31",
"first-page": "154",
"article-title": "How many trees in a random forest?",
"volume": "vol. 7376",
"author": "Oshiro",
"year": "2012"
},
{
"key": "10.1016/j.jas.2019.105013_bib32",
"article-title": "Decision-making in modern surveys",
"volume": "ume 1",
"author": "Plog",
"year": "1978"
},
{
"issue": "4",
"key": "10.1016/j.jas.2019.105013_bib33",
"doi-asserted-by": "crossref",
"first-page": "100",
"DOI": "10.3390/geosciences7040100",
"article-title": "From above and on the ground: geospatial methods for recording endangered archaeology in the Middle East and north africa",
"volume": "7",
"author": "Rayne",
"year": "2017",
"journal-title": "Geosciences"
},
{
"issue": "1",
"key": "10.1016/j.jas.2019.105013_bib34",
"doi-asserted-by": "crossref",
"first-page": "1",
"DOI": "10.1080/00438243.1978.9979712",
"article-title": "The design of archaeological surveys",
"volume": "10",
"author": "Schiffer",
"year": "1978",
"journal-title": "World Archaeol."
},
{
"key": "10.1016/j.jas.2019.105013_bib35",
"series-title": "Experiments in the Collection and Analysis of Archaeological Survey Data: the East Hampshire Survey",
"author": "Shennan",
"year": "1985"
},
{
"key": "10.1016/j.jas.2019.105013_bib36",
"doi-asserted-by": "crossref",
"first-page": "1066",
"DOI": "10.1016/j.culher.2016.06.006",
"article-title": "Drones over Mediterranean landscapes. The potential of small UAV's (drones) for site detection and heritage management in archaeological survey projects: a case study from Le Pianelle in the Tappino Valley, Molise (Italy)",
"volume": "22",
"author": "Stek",
"year": "2016",
"journal-title": "J. Cult. Herit."
},
{
"key": "10.1016/j.jas.2019.105013_bib37",
"series-title": "Side-by-Side Survey. Comparative Regional Studies in the Mediterranean World",
"first-page": "65",
"article-title": "Side-by-side and back to front: exploring intra-regional latitudinal and longitudinal comparability in survey data. Three case studies from Metaponto, southern Italy",
"author": "Thomson",
"year": "2004"
},
{
"key": "10.1016/j.jas.2019.105013_bib38",
"series-title": "Digital Discovery. Exploring New Frontiers in Human Heritage. Computer Applications and Quantitative Methods in Archaeology",
"article-title": "Computer vision and machine learning for archaeology",
"author": "van der Maaten",
"year": "2007"
},
{
"key": "10.1016/j.jas.2019.105013_bib39",
"doi-asserted-by": "crossref",
"first-page": "1114",
"DOI": "10.1111/j.1475-4754.2012.00667.x",
"article-title": "Computer vision-based orthophoto mapping of complex archaeological sites: the ancient quarry of Pitaranha (Portugal-Spain)",
"volume": "54",
"author": "Verhoeven",
"year": "2012",
"journal-title": "Archaeometry"
},
{
"key": "10.1016/j.jas.2019.105013_bib40",
"series-title": "A Guide for Salvage Archeology",
"author": "Wendorf",
"year": "1962"
}
],
"container-title": [
"Journal of Archaeological Science"
],
"original-title": [
],
"language": "en",
"link": [
{
"URL": "https://api.elsevier.com/content/article/PII:S0305440319301001?httpAccept=text/xml",
"content-type": "text/xml",
"content-version": "vor",
"intended-application": "text-mining"
},
{
"URL": "https://api.elsevier.com/content/article/PII:S0305440319301001?httpAccept=text/plain",
"content-type": "text/plain",
"content-version": "vor",
"intended-application": "text-mining"
}
],
"deposited": {
"date-parts": [
[
2019,
11,
25
]
],
"date-time": "2019-11-25T06:46:34Z",
"timestamp": 1574664394000
},
"score": 1,
"subtitle": [
],
"short-title": [
],
"issued": {
"date-parts": [
[
2019,
12
]
]
},
"references-count": 39,
"alternative-id": [
"S0305440319301001"
],
"URL": "http://dx.doi.org/10.1016/j.jas.2019.105013",
"relation": {
},
"ISSN": [
"0305-4403"
],
"issn-type": [
{
"value": "0305-4403",
"type": "print"
}
],
"subject": [
"Archaeology",
"Archaeology"
],
"published": {
"date-parts": [
[
2019,
12
]
]
},
"assertion": [
{
"value": "Elsevier",
"name": "publisher",
"label": "This article is maintained by"
},
{
"value": "A brave new world for archaeological survey: Automated machine learning-based potsherd detection using high-resolution drone imagery",
"name": "articletitle",
"label": "Article Title"
},
{
"value": "Journal of Archaeological Science",
"name": "journaltitle",
"label": "Journal Title"
},
{
"value": "https://doi.org/10.1016/j.jas.2019.105013",
"name": "articlelink",
"label": "CrossRef DOI link to publisher maintained version"
},
{
"value": "article",
"name": "content_type",
"label": "Content Type"
},
{
"value": "© 2019 The Authors. Published by Elsevier Ltd.",
"name": "copyright",
"label": "Copyright"
}
],
"article-number": "105013"
}

View File

@ -25,6 +25,24 @@ public class PropagationConstant {
private PropagationConstant() {
}
public static final String DOI = "doi";
public static final String REF_DOI = ".refs";
public static final String UPDATE_DATA_INFO_TYPE = "update";
public static final String UPDATE_SUBJECT_FOS_CLASS_ID = "subject:fos";
public static final String UPDATE_CLASS_NAME = "Inferred by OpenAIRE";
public static final String UPDATE_MEASURE_BIP_CLASS_ID = "measure:bip";
public static final String FOS_CLASS_ID = "FOS";
public static final String FOS_CLASS_NAME = "Fields of Science and Technology classification";
public static final String OPENCITATIONS_CLASSID = "sysimport:crosswalk:opencitations";
public static final String OPENCITATIONS_CLASSNAME = "Imported from OpenCitations";
public static final String ID_PREFIX = "50|doi_________::";
public static final String OC_TRUST = "0.91";
public final static String NULL = "NULL";
public static final String INSTITUTIONAL_REPO_TYPE = "pubsrepository::institutional";
public static final String PROPAGATION_DATA_INFO_TYPE = "propagation";
@ -75,10 +93,25 @@ public class PropagationConstant {
public static DataInfo getDataInfo(
String inference_provenance, String inference_class_id, String inference_class_name, String qualifierSchema) {
return getDataInfo(inference_provenance, inference_class_id, inference_class_name, qualifierSchema, "0.85");
}
public static DataInfo getDataInfo(
String inference_provenance, String inference_class_id, String inference_class_name, String qualifierSchema,
String trust) {
return getDataInfo(
inference_provenance, inference_class_id, inference_class_name, qualifierSchema, trust, true);
}
public static DataInfo getDataInfo(
String inference_provenance, String inference_class_id, String inference_class_name, String qualifierSchema,
String trust, boolean inferred) {
DataInfo di = new DataInfo();
di.setInferred(true);
di.setInferred(inferred);
di.setDeletedbyinference(false);
di.setTrust("0.85");
di.setTrust(trust);
di.setInferenceprovenance(inference_provenance);
di.setProvenanceaction(getQualifier(inference_class_id, inference_class_name, qualifierSchema));
return di;

View File

@ -127,13 +127,6 @@ public class MergeGraphTableSparkJob {
}
}, Encoders.bean(p_clazz))
.filter((FilterFunction<P>) Objects::nonNull)
.filter((FilterFunction<P>) o -> {
HashSet<String> collectedFromNames = Optional
.ofNullable(o.getCollectedfrom())
.map(c -> c.stream().map(KeyValue::getValue).collect(Collectors.toCollection(HashSet::new)))
.orElse(new HashSet<>());
return !collectedFromNames.contains("Datacite");
})
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")

View File

@ -1,136 +0,0 @@
package eu.dnetlib.dhp.oa.graph.raw;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.*;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
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.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.clearspring.analytics.util.Lists;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
import eu.dnetlib.dhp.oa.graph.raw.common.AbstractMigrationApplication;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.Oaf;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
public class CopyHdfsOafApplication extends AbstractMigrationApplication {
private static final Logger log = LoggerFactory.getLogger(CopyHdfsOafApplication.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
public static void main(final String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
CopyHdfsOafApplication.class
.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/copy_hdfs_oaf_parameters.json")));
parser.parseArgument(args);
final Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String mdstoreManagerUrl = parser.get("mdstoreManagerUrl");
log.info("mdstoreManagerUrl: {}", mdstoreManagerUrl);
final String mdFormat = parser.get("mdFormat");
log.info("mdFormat: {}", mdFormat);
final String mdLayout = parser.get("mdLayout");
log.info("mdLayout: {}", mdLayout);
final String mdInterpretation = parser.get("mdInterpretation");
log.info("mdInterpretation: {}", mdInterpretation);
final String hdfsPath = parser.get("hdfsPath");
log.info("hdfsPath: {}", hdfsPath);
final String isLookupUrl = parser.get("isLookupUrl");
log.info("isLookupUrl: {}", isLookupUrl);
final ISLookUpService isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl);
final VocabularyGroup vocs = VocabularyGroup.loadVocsFromIS(isLookupService);
final Set<String> paths = mdstorePaths(mdstoreManagerUrl, mdFormat, mdLayout, mdInterpretation);
final SparkConf conf = new SparkConf();
runWithSparkSession(conf, isSparkSessionManaged, spark -> processPaths(spark, vocs, hdfsPath, paths));
}
public static void processPaths(final SparkSession spark,
final VocabularyGroup vocs,
final String outputPath,
final Set<String> paths) {
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
log.info("Found {} mdstores", paths.size());
paths.forEach(log::info);
final String[] validPaths = paths
.stream()
.filter(p -> HdfsSupport.exists(p, sc.hadoopConfiguration()))
.toArray(String[]::new);
log.info("Non empty mdstores {}", validPaths.length);
if (validPaths.length > 0) {
// load the dataset
Dataset<Oaf> oaf = spark
.read()
.load(validPaths)
.as(Encoders.kryo(Oaf.class));
// dispatch each entity type individually in the respective graph subdirectory in append mode
for (Map.Entry<String, Class> e : ModelSupport.oafTypes.entrySet()) {
oaf
.filter((FilterFunction<Oaf>) o -> o.getClass().getSimpleName().toLowerCase().equals(e.getKey()))
.map((MapFunction<Oaf, String>) OBJECT_MAPPER::writeValueAsString, Encoders.bean(e.getValue()))
.write()
.option("compression", "gzip")
.mode(SaveMode.Append)
.text(outputPath + "/" + e.getKey());
}
}
}
private static Relation getInverse(Relation rel, VocabularyGroup vocs) {
final Relation inverse = new Relation();
inverse.setProperties(rel.getProperties());
inverse.setValidated(rel.getValidated());
inverse.setValidationDate(rel.getValidationDate());
inverse.setCollectedfrom(rel.getCollectedfrom());
inverse.setDataInfo(rel.getDataInfo());
inverse.setLastupdatetimestamp(rel.getLastupdatetimestamp());
inverse.setSource(rel.getTarget());
inverse.setTarget(rel.getSource());
inverse.setRelType(rel.getRelType());
inverse.setSubRelType(rel.getSubRelType());
return inverse;
}
}

View File

@ -0,0 +1,74 @@
package eu.dnetlib.dhp.oa.graph.raw
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.common.HdfsSupport
import eu.dnetlib.dhp.schema.common.ModelSupport
import eu.dnetlib.dhp.schema.mdstore.MDStoreWithInfo
import eu.dnetlib.dhp.schema.oaf.Oaf
import eu.dnetlib.dhp.utils.DHPUtils
import org.apache.commons.io.IOUtils
import org.apache.commons.lang3.StringUtils
import org.apache.http.client.methods.HttpGet
import org.apache.http.impl.client.HttpClients
import org.apache.spark.sql.{Encoder, Encoders, SaveMode, SparkSession}
import org.apache.spark.{SparkConf, SparkContext}
import org.slf4j.LoggerFactory
import scala.collection.JavaConverters._
import scala.io.Source
object CopyHdfsOafSparkApplication {
def main(args: Array[String]): Unit = {
val log = LoggerFactory.getLogger(getClass)
val conf = new SparkConf()
val parser = new ArgumentApplicationParser(Source.fromInputStream(getClass.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/copy_hdfs_oaf_parameters.json")).mkString)
parser.parseArgument(args)
val spark =
SparkSession
.builder()
.config(conf)
.appName(getClass.getSimpleName)
.master(parser.get("master")).getOrCreate()
val sc: SparkContext = spark.sparkContext
val mdstoreManagerUrl = parser.get("mdstoreManagerUrl")
log.info("mdstoreManagerUrl: {}", mdstoreManagerUrl)
val mdFormat = parser.get("mdFormat")
log.info("mdFormat: {}", mdFormat)
val mdLayout = parser.get("mdLayout")
log.info("mdLayout: {}", mdLayout)
val mdInterpretation = parser.get("mdInterpretation")
log.info("mdInterpretation: {}", mdInterpretation)
val hdfsPath = parser.get("hdfsPath")
log.info("hdfsPath: {}", hdfsPath)
implicit val oafEncoder: Encoder[Oaf] = Encoders.kryo[Oaf]
val paths = DHPUtils.mdstorePaths(mdstoreManagerUrl, mdFormat, mdLayout, mdInterpretation, true).asScala
val validPaths: List[String] = paths.filter(p => HdfsSupport.exists(p, sc.hadoopConfiguration)).toList
if (validPaths.nonEmpty) {
val oaf = spark.read.load(validPaths: _*).as[Oaf]
val mapper = new ObjectMapper()
val l =ModelSupport.oafTypes.entrySet.asScala.map(e => e.getKey).toList
l.foreach(
e =>
oaf.filter(o => o.getClass.getSimpleName.equalsIgnoreCase(e))
.map(s => mapper.writeValueAsString(s))(Encoders.STRING)
.write
.option("compression", "gzip")
.mode(SaveMode.Append)
.text(s"$hdfsPath/${e}")
)
}
}
}

View File

@ -186,6 +186,9 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i
log.info("Processing Openorgs Merge Rels...");
smdbe.execute("queryOpenOrgsSimilarityForProvision.sql", smdbe::processOrgOrgMergeRels);
log.info("Processing Openorgs Parent/Child Rels...");
smdbe.execute("queryParentChildRelsOpenOrgs.sql", smdbe::processOrgOrgParentChildRels);
break;
case openaire_organizations:
@ -689,6 +692,35 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i
}
}
public List<Oaf> processOrgOrgParentChildRels(final ResultSet rs) {
try {
final DataInfo info = prepareDataInfo(rs); // TODO
final String orgId1 = createOpenaireId(20, rs.getString("source"), true);
final String orgId2 = createOpenaireId(20, rs.getString("target"), true);
final List<KeyValue> collectedFrom = listKeyValues(
createOpenaireId(10, rs.getString("collectedfromid"), true), rs.getString("collectedfromname"));
final Relation r = new Relation();
r.setRelType(ORG_ORG_RELTYPE);
r.setSubRelType(ModelConstants.RELATIONSHIP);
r
.setRelClass(
rs.getString("type").equalsIgnoreCase("parent") ? ModelConstants.IS_PARENT_OF
: ModelConstants.IS_CHILD_OF);
r.setSource(orgId1);
r.setTarget(orgId2);
r.setCollectedfrom(collectedFrom);
r.setDataInfo(info);
r.setLastupdatetimestamp(lastUpdateTimestamp);
return Arrays.asList(r);
} catch (final Exception e) {
throw new RuntimeException(e);
}
}
public List<Oaf> processOrgOrgSimRels(final ResultSet rs) {
try {
final DataInfo info = prepareDataInfo(rs); // TODO

View File

@ -26,6 +26,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.schema.mdstore.MDStoreWithInfo;
import eu.dnetlib.dhp.schema.oaf.Oaf;
import eu.dnetlib.dhp.utils.DHPUtils;
public class AbstractMigrationApplication implements Closeable {
@ -71,27 +72,7 @@ public class AbstractMigrationApplication implements Closeable {
final String format,
final String layout,
final String interpretation) throws IOException {
final String url = mdstoreManagerUrl + "/mdstores/";
final ObjectMapper objectMapper = new ObjectMapper();
final HttpGet req = new HttpGet(url);
try (final CloseableHttpClient client = HttpClients.createDefault()) {
try (final CloseableHttpResponse response = client.execute(req)) {
final String json = IOUtils.toString(response.getEntity().getContent());
final MDStoreWithInfo[] mdstores = objectMapper.readValue(json, MDStoreWithInfo[].class);
return Arrays
.stream(mdstores)
.filter(md -> md.getFormat().equalsIgnoreCase(format))
.filter(md -> md.getLayout().equalsIgnoreCase(layout))
.filter(md -> md.getInterpretation().equalsIgnoreCase(interpretation))
.filter(md -> StringUtils.isNotBlank(md.getHdfsPath()))
.filter(md -> StringUtils.isNotBlank(md.getCurrentVersion()))
.filter(md -> md.getSize() > 0)
.map(md -> md.getHdfsPath() + "/" + md.getCurrentVersion() + "/store")
.collect(Collectors.toSet());
}
}
return DHPUtils.mdstorePaths(mdstoreManagerUrl, format, layout, interpretation, false);
}
private Configuration getConf() {

View File

@ -0,0 +1,107 @@
package eu.dnetlib.dhp.oa.graph.resolution
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.common.HdfsSupport
import eu.dnetlib.dhp.schema.common.EntityType
import eu.dnetlib.dhp.schema.oaf.{OtherResearchProduct, Publication, Result, Software, Dataset => OafDataset}
import org.apache.commons.io.IOUtils
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.spark.SparkConf
import org.apache.spark.sql._
import org.slf4j.{Logger, LoggerFactory}
object SparkResolveEntities {
val mapper = new ObjectMapper()
val entities = List(EntityType.dataset,EntityType.publication, EntityType.software, EntityType.otherresearchproduct)
def main(args: Array[String]): Unit = {
val log: Logger = LoggerFactory.getLogger(getClass)
val conf: SparkConf = new SparkConf()
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/resolution/resolve_entities_params.json")))
parser.parseArgument(args)
val spark: SparkSession =
SparkSession
.builder()
.config(conf)
.appName(getClass.getSimpleName)
.master(parser.get("master")).getOrCreate()
val graphBasePath = parser.get("graphBasePath")
log.info(s"graphBasePath -> $graphBasePath")
val workingPath = parser.get("workingPath")
log.info(s"workingPath -> $workingPath")
val unresolvedPath = parser.get("unresolvedPath")
log.info(s"unresolvedPath -> $unresolvedPath")
val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration)
fs.mkdirs(new Path(workingPath))
resolveEntities(spark, workingPath, unresolvedPath)
generateResolvedEntities(spark, workingPath, graphBasePath)
// TO BE conservative we keep the original entities in the working dir
// and save the resolved entities on the graphBasePath
//In future these lines of code should be removed
entities.foreach {
e =>
fs.rename(new Path(s"$graphBasePath/$e"), new Path(s"$workingPath/${e}_old"))
fs.rename(new Path(s"$workingPath/resolvedGraph/$e"), new Path(s"$graphBasePath/$e"))
}
}
def resolveEntities(spark: SparkSession, workingPath: String, unresolvedPath: String) = {
implicit val resEncoder: Encoder[Result] = Encoders.kryo(classOf[Result])
import spark.implicits._
val rPid: Dataset[(String, String)] = spark.read.load(s"$workingPath/relationResolvedPid").as[(String, String)]
val up: Dataset[(String, Result)] = spark.read.text(unresolvedPath).as[String].map(s => mapper.readValue(s, classOf[Result])).map(r => (r.getId, r))(Encoders.tuple(Encoders.STRING, resEncoder))
rPid.joinWith(up, rPid("_2").equalTo(up("_1")), "inner").map {
r =>
val result = r._2._2
val dnetId = r._1._1
result.setId(dnetId)
result
}.write.mode(SaveMode.Overwrite).save(s"$workingPath/resolvedEntities")
}
def deserializeObject(input:String, entity:EntityType ) :Result = {
entity match {
case EntityType.publication => mapper.readValue(input, classOf[Publication])
case EntityType.dataset => mapper.readValue(input, classOf[OafDataset])
case EntityType.software=> mapper.readValue(input, classOf[Software])
case EntityType.otherresearchproduct=> mapper.readValue(input, classOf[OtherResearchProduct])
}
}
def generateResolvedEntities(spark:SparkSession, workingPath: String, graphBasePath:String) = {
implicit val resEncoder: Encoder[Result] = Encoders.kryo(classOf[Result])
import spark.implicits._
val re:Dataset[Result] = spark.read.load(s"$workingPath/resolvedEntities").as[Result]
entities.foreach {
e =>
spark.read.text(s"$graphBasePath/$e").as[String]
.map(s => deserializeObject(s, e))
.union(re)
.groupByKey(_.getId)
.reduceGroups {
(x, y) =>
x.mergeFrom(y)
x
}.map(_._2)
.filter(r => r.getClass.getSimpleName.toLowerCase != "result")
.map(r => mapper.writeValueAsString(r))(Encoders.STRING)
.write.mode(SaveMode.Overwrite).option("compression", "gzip").text(s"$workingPath/resolvedGraph/$e")
}
}
}

View File

@ -0,0 +1,161 @@
package eu.dnetlib.dhp.oa.graph.resolution
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.common.HdfsSupport
import eu.dnetlib.dhp.schema.oaf.{Relation, Result}
import eu.dnetlib.dhp.utils.DHPUtils
import org.apache.commons.io.IOUtils
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.spark.SparkConf
import org.apache.spark.rdd.RDD
import org.apache.spark.sql._
import org.json4s
import org.json4s.DefaultFormats
import org.json4s.JsonAST.{JField, JObject, JString}
import org.json4s.jackson.JsonMethods.parse
import org.slf4j.{Logger, LoggerFactory}
object SparkResolveRelation {
def main(args: Array[String]): Unit = {
val log: Logger = LoggerFactory.getLogger(getClass)
val conf: SparkConf = new SparkConf()
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/resolution/resolve_relations_params.json")))
parser.parseArgument(args)
val spark: SparkSession =
SparkSession
.builder()
.config(conf)
.appName(getClass.getSimpleName)
.master(parser.get("master")).getOrCreate()
val graphBasePath = parser.get("graphBasePath")
log.info(s"graphBasePath -> $graphBasePath")
val workingPath = parser.get("workingPath")
log.info(s"workingPath -> $workingPath")
implicit val relEncoder: Encoder[Relation] = Encoders.kryo(classOf[Relation])
import spark.implicits._
//CLEANING TEMPORARY FOLDER
HdfsSupport.remove(workingPath, spark.sparkContext.hadoopConfiguration)
val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration)
fs.mkdirs(new Path(workingPath))
extractPidResolvedTableFromJsonRDD(spark, graphBasePath, workingPath)
val mapper: ObjectMapper = new ObjectMapper()
val rPid: Dataset[(String, String)] = spark.read.load(s"$workingPath/relationResolvedPid").as[(String, String)]
val relationDs: Dataset[(String, Relation)] = spark.read.text(s"$graphBasePath/relation").as[String]
.map(s => mapper.readValue(s, classOf[Relation])).as[Relation]
.map(r => (r.getSource.toLowerCase, r))(Encoders.tuple(Encoders.STRING, relEncoder))
relationDs.joinWith(rPid, relationDs("_1").equalTo(rPid("_2")), "left").map {
m =>
val sourceResolved = m._2
val currentRelation = m._1._2
if (sourceResolved != null && sourceResolved._1 != null && sourceResolved._1.nonEmpty)
currentRelation.setSource(sourceResolved._1)
currentRelation
}.write
.mode(SaveMode.Overwrite)
.save(s"$workingPath/relationResolvedSource")
val relationSourceResolved: Dataset[(String, Relation)] = spark.read.load(s"$workingPath/relationResolvedSource").as[Relation]
.map(r => (r.getTarget.toLowerCase, r))(Encoders.tuple(Encoders.STRING, relEncoder))
relationSourceResolved.joinWith(rPid, relationSourceResolved("_1").equalTo(rPid("_2")), "left").map {
m =>
val targetResolved = m._2
val currentRelation = m._1._2
if (targetResolved != null && targetResolved._1.nonEmpty)
currentRelation.setTarget(targetResolved._1)
currentRelation
}
.write
.mode(SaveMode.Overwrite)
.save(s"$workingPath/relation_resolved")
// TO BE conservative we keep the original relation in the working dir
// and save the relation resolved on the graphBasePath
//In future this two line of code should be removed
fs.rename(new Path(s"$graphBasePath/relation"), new Path(s"$workingPath/relation"))
spark.read.load(s"$workingPath/relation_resolved").as[Relation]
.filter(r => !r.getSource.startsWith("unresolved") && !r.getTarget.startsWith("unresolved"))
.map(r => mapper.writeValueAsString(r))
.write
.option("compression", "gzip")
.mode(SaveMode.Overwrite)
.text(s"$graphBasePath/relation")
}
def extractInstanceCF(input: String): List[(String, String)] = {
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
lazy val json: json4s.JValue = parse(input)
val result: List[(String, String)] = for {
JObject(iObj) <- json \ "instance"
JField("collectedfrom", JObject(cf)) <- iObj
JField("instancetype", JObject(instancetype)) <- iObj
JField("value", JString(collectedFrom)) <- cf
JField("classname", JString(classname)) <- instancetype
} yield (classname, collectedFrom)
result
}
def extractPidsFromRecord(input: String): (String, List[(String, String)]) = {
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
lazy val json: json4s.JValue = parse(input)
val id: String = (json \ "id").extract[String]
val result: List[(String, String)] = for {
JObject(pids) <- json \\ "instance" \ "pid"
JField("value", JString(pidValue)) <- pids
JField("qualifier", JObject(qualifier)) <- pids
JField("classid", JString(pidType)) <- qualifier
} yield (pidValue, pidType)
(id, result)
}
private def isRelation(input: String): Boolean = {
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
lazy val json: json4s.JValue = parse(input)
val source = (json \ "source").extractOrElse[String](null)
source != null
}
def extractPidResolvedTableFromJsonRDD(spark: SparkSession, graphPath: String, workingPath: String) = {
import spark.implicits._
val d: RDD[(String, String)] = spark.sparkContext.textFile(s"$graphPath/*")
.filter(i => !isRelation(i))
.map(i => extractPidsFromRecord(i))
.filter(s => s != null && s._1 != null && s._2 != null && s._2.nonEmpty)
.flatMap { p =>
p._2.map(pid =>
(p._1, DHPUtils.generateUnresolvedIdentifier(pid._1, pid._2))
)
}.filter(r => r._1 != null || r._2 != null)
spark.createDataset(d)
.groupByKey(_._2)
.reduceGroups((x, y) => if (x._1.startsWith("50|doi") || x._1.startsWith("50|pmid")) x else y)
.map(s => s._2)
.write
.mode(SaveMode.Overwrite)
.save(s"$workingPath/relationResolvedPid")
}
}

View File

@ -59,7 +59,12 @@ object SparkConvertRDDtoDataset {
log.info("Converting Relation")
val rddRelation =spark.sparkContext.textFile(s"$sourcePath/relation").map(s => mapper.readValue(s, classOf[Relation]))
val relationSemanticFilter = List("cites", "iscitedby","merges", "ismergedin")
val rddRelation =spark.sparkContext.textFile(s"$sourcePath/relation")
.map(s => mapper.readValue(s, classOf[Relation]))
.filter(r=> r.getSource.startsWith("50") && r.getTarget.startsWith("50"))
.filter(r => !relationSemanticFilter.exists(k => k.equalsIgnoreCase(r.getRelClass)))
spark.createDataset(rddRelation).as[Relation].write.mode(SaveMode.Overwrite).save(s"$relPath")

View File

@ -51,10 +51,14 @@ object SparkCreateScholix {
relationDS.joinWith(summaryDS, relationDS("_1").equalTo(summaryDS("_1")), "left")
.map { input: ((String, Relation), (String, ScholixSummary)) =>
val rel: Relation = input._1._2
val source: ScholixSummary = input._2._2
(rel.getTarget, ScholixUtils.scholixFromSource(rel, source))
if (input._1!= null && input._2!= null) {
val rel: Relation = input._1._2
val source: ScholixSummary = input._2._2
(rel.getTarget, ScholixUtils.scholixFromSource(rel, source))
}
else null
}(Encoders.tuple(Encoders.STRING, scholixEncoder))
.filter(r => r!= null)
.write.mode(SaveMode.Overwrite).save(s"$targetPath/scholix_from_source")
val scholixSource: Dataset[(String, Scholix)] = spark.read.load(s"$targetPath/scholix_from_source").as[(String, Scholix)](Encoders.tuple(Encoders.STRING, scholixEncoder))

View File

@ -1,154 +0,0 @@
package eu.dnetlib.dhp.sx.graph
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.application.ArgumentApplicationParser
import eu.dnetlib.dhp.schema.oaf.{Relation, Result}
import org.apache.commons.io.IOUtils
import org.apache.hadoop.io.compress.GzipCodec
import org.apache.spark.SparkConf
import org.apache.spark.rdd.RDD
import org.apache.spark.sql._
import org.json4s
import org.json4s.DefaultFormats
import org.json4s.JsonAST.{JField, JObject, JString}
import org.json4s.jackson.JsonMethods.parse
import org.slf4j.{Logger, LoggerFactory}
import scala.collection.JavaConverters._
object SparkResolveRelation {
def main(args: Array[String]): Unit = {
val log: Logger = LoggerFactory.getLogger(getClass)
val conf: SparkConf = new SparkConf()
val parser = new ArgumentApplicationParser(IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/resolve_relations_params.json")))
parser.parseArgument(args)
val spark: SparkSession =
SparkSession
.builder()
.config(conf)
.appName(getClass.getSimpleName)
.master(parser.get("master")).getOrCreate()
val relationPath = parser.get("relationPath")
log.info(s"sourcePath -> $relationPath")
val entityPath = parser.get("entityPath")
log.info(s"entityPath -> $entityPath")
val workingPath = parser.get("workingPath")
log.info(s"workingPath -> $workingPath")
implicit val relEncoder: Encoder[Relation] = Encoders.kryo(classOf[Relation])
import spark.implicits._
extractPidResolvedTableFromJsonRDD(spark, entityPath, workingPath)
val mappper = new ObjectMapper()
val rPid:Dataset[(String,String)] = spark.read.load(s"$workingPath/relationResolvedPid").as[(String,String)]
val relationDs:Dataset[(String,Relation)] = spark.read.load(relationPath).as[Relation].map(r => (r.getSource.toLowerCase, r))(Encoders.tuple(Encoders.STRING, relEncoder))
relationDs.joinWith(rPid, relationDs("_1").equalTo(rPid("_2")), "left").map{
m =>
val sourceResolved = m._2
val currentRelation = m._1._2
if (sourceResolved!=null && sourceResolved._1!=null && sourceResolved._1.nonEmpty)
currentRelation.setSource(sourceResolved._1)
currentRelation
}.write
.mode(SaveMode.Overwrite)
.save(s"$workingPath/relationResolvedSource")
val relationSourceResolved:Dataset[(String,Relation)] = spark.read.load(s"$workingPath/relationResolvedSource").as[Relation].map(r => (r.getTarget.toLowerCase, r))(Encoders.tuple(Encoders.STRING, relEncoder))
relationSourceResolved.joinWith(rPid, relationSourceResolved("_1").equalTo(rPid("_2")), "left").map{
m =>
val targetResolved = m._2
val currentRelation = m._1._2
if (targetResolved!=null && targetResolved._1.nonEmpty)
currentRelation.setTarget(targetResolved._1)
currentRelation
}.filter(r => r.getSource.startsWith("50")&& r.getTarget.startsWith("50"))
.write
.mode(SaveMode.Overwrite)
.save(s"$workingPath/relation_resolved")
spark.read.load(s"$workingPath/relation_resolved").as[Relation]
.map(r => mappper.writeValueAsString(r))
.rdd.saveAsTextFile(s"$workingPath/relation", classOf[GzipCodec])
}
def extractPidsFromRecord(input:String):(String,List[(String,String)]) = {
implicit lazy val formats: DefaultFormats.type = org.json4s.DefaultFormats
lazy val json: json4s.JValue = parse(input)
val id:String = (json \ "id").extract[String]
val result: List[(String,String)] = for {
JObject(pids) <- json \ "pid"
JField("value", JString(pidValue)) <- pids
JField("qualifier", JObject(qualifier)) <- pids
JField("classname", JString(pidType)) <- qualifier
} yield (pidValue, pidType)
val alternateIds: List[(String,String)] = for {
JObject(pids) <- json \\ "alternateIdentifier"
JField("value", JString(pidValue)) <- pids
JField("qualifier", JObject(qualifier)) <- pids
JField("classname", JString(pidType)) <- qualifier
} yield (pidValue, pidType)
(id,result:::alternateIds)
}
private def extractPidResolvedTableFromJsonRDD(spark: SparkSession, entityPath: String, workingPath: String) = {
import spark.implicits._
val d: RDD[(String,String)] = spark.sparkContext.textFile(s"$entityPath/*")
.map(i => extractPidsFromRecord(i))
.filter(s => s != null && s._1!= null && s._2!=null && s._2.nonEmpty)
.flatMap{ p =>
p._2.map(pid =>
(p._1, convertPidToDNETIdentifier(pid._1, pid._2))
)
}.filter(r =>r._1 != null || r._2 != null)
spark.createDataset(d)
.groupByKey(_._2)
.reduceGroups((x, y) => if (x._1.startsWith("50|doi") || x._1.startsWith("50|pmid")) x else y)
.map(s => s._2)
.write
.mode(SaveMode.Overwrite)
.save(s"$workingPath/relationResolvedPid")
}
/*
This method should be used once we finally convert everythings in Kryo dataset
instead of using rdd of json
*/
private def extractPidResolvedTableFromKryo(spark: SparkSession, entityPath: String, workingPath: String) = {
import spark.implicits._
implicit val oafEncoder: Encoder[Result] = Encoders.kryo(classOf[Result])
val entities: Dataset[Result] = spark.read.load(s"$entityPath/*").as[Result]
entities.flatMap(e => e.getPid.asScala
.map(p =>
convertPidToDNETIdentifier(p.getValue, p.getQualifier.getClassid))
.filter(s => s != null)
.map(s => (s, e.getId))
).groupByKey(_._1)
.reduceGroups((x, y) => if (x._2.startsWith("50|doi") || x._2.startsWith("50|pmid")) x else y)
.map(s => s._2)
.write
.mode(SaveMode.Overwrite)
.save(s"$workingPath/relationResolvedPid")
}
def convertPidToDNETIdentifier(pid:String, pidType: String):String = {
if (pid==null || pid.isEmpty || pidType== null || pidType.isEmpty)
null
else
s"unresolved::${pid.toLowerCase}::${pidType.toLowerCase}"
}
}

View File

@ -289,7 +289,7 @@ object ScholixUtils {
if (r.getInstance() == null || r.getInstance().isEmpty)
return List()
r.getInstance().asScala.filter(i => i.getUrl!= null && !i.getUrl.isEmpty)
.filter(i => i.getPid!= null && i.getUrl != null)
.flatMap(i => findURLForPID(i.getPid.asScala.toList, i.getUrl.asScala.toList))
.map(i => new ScholixIdentifier(i._1.getValue, i._1.getQualifier.getClassid, i._2)).distinct.toList
}

View File

@ -23,16 +23,16 @@
"paramDescription": "metadata layout",
"paramRequired": true
},
{
"paramName": "m",
"paramLongName": "master",
"paramDescription": "should be yarn or local",
"paramRequired": true
},
{
"paramName": "i",
"paramLongName": "mdInterpretation",
"paramDescription": "metadata interpretation",
"paramRequired": true
},
{
"paramName": "isu",
"paramLongName": "isLookupUrl",
"paramDescription": "the url of the ISLookupService",
"paramRequired": true
}
]

View File

@ -258,7 +258,7 @@
<switch>
<case to="ImportDB">${wf:conf('reuseDB') eq false}</case>
<case to="reuse_odf">${wf:conf('reuseDB') eq true}</case>
<default to="ImportDB_claims"/>
<default to="ImportDB"/>
</switch>
</decision>
@ -553,7 +553,7 @@
<master>yarn</master>
<mode>cluster</mode>
<name>ImportOAF_hdfs_graph</name>
<class>eu.dnetlib.dhp.oa.graph.raw.CopyHdfsOafApplication</class>
<class>eu.dnetlib.dhp.oa.graph.raw.CopyHdfsOafSparkApplication</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory ${sparkExecutorMemory}
@ -568,8 +568,8 @@
<arg>--mdstoreManagerUrl</arg><arg>${mdstoreManagerUrl}</arg>
<arg>--mdFormat</arg><arg>OAF</arg>
<arg>--mdLayout</arg><arg>store</arg>
<arg>--master</arg><arg>yarn</arg>
<arg>--mdInterpretation</arg><arg>graph</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
</spark>
<ok to="wait_graphs"/>
<error to="Kill"/>

View File

@ -0,0 +1,72 @@
<workflow-app name="Resolve relation and entities" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>graphBasePath</name>
<description>the path of the graph</description>
</property>
<property>
<name>unresolvedPath</name>
<description>the path of the unresolved Entities</description>
</property>
</parameters>
<start to="ResolveRelations"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="ResolveRelations">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Resolve Relations in raw graph</name>
<class>eu.dnetlib.dhp.oa.graph.resolution.SparkResolveRelation</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.shuffle.partitions=15000
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--master</arg><arg>yarn</arg>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--workingPath</arg><arg>${workingDir}</arg>
</spark>
<ok to="ResolveEntities"/>
<error to="Kill"/>
</action>
<action name="ResolveEntities">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Resolve Entities in raw graph</name>
<class>eu.dnetlib.dhp.oa.graph.resolution.SparkResolveEntities</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.shuffle.partitions=10000
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--master</arg><arg>yarn</arg>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--unresolvedPath</arg><arg>${unresolvedPath}</arg>
<arg>--workingPath</arg><arg>${workingDir}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,6 @@
[
{"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true},
{"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the source Path", "paramRequired": true},
{"paramName":"u", "paramLongName":"unresolvedPath", "paramDescription": "the source Path", "paramRequired": true},
{"paramName":"g", "paramLongName":"graphBasePath", "paramDescription": "the path of the raw graph", "paramRequired": true}
]

Some files were not shown because too many files have changed in this diff Show More