merge from beta

mergeutils
Claudio Atzori 1 month ago
commit ecff0b4825

@ -145,105 +145,6 @@ public class AuthorMerger {
return null;
}
/**
* This method tries to figure out when two author are the same in the contest
* of ORCID enrichment
*
* @param left Author in the OAF entity
* @param right Author ORCID
* @return based on a heuristic on the names of the authors if they are the same.
*/
public static boolean checkORCIDSimilarity(final Author left, final Author right) {
final Person pl = parse(left);
final Person pr = parse(right);
// If one of them didn't have a surname we verify if they have the fullName not empty
// and verify if the normalized version is equal
if (!(pl.getSurname() != null && pl.getSurname().stream().anyMatch(StringUtils::isNotBlank) &&
pr.getSurname() != null && pr.getSurname().stream().anyMatch(StringUtils::isNotBlank))) {
if (pl.getFullname() != null && !pl.getFullname().isEmpty() && pr.getFullname() != null
&& !pr.getFullname().isEmpty()) {
return pl
.getFullname()
.stream()
.anyMatch(
fl -> pr.getFullname().stream().anyMatch(fr -> normalize(fl).equalsIgnoreCase(normalize(fr))));
} else {
return false;
}
}
// The Authors have one surname in common
if (pl.getSurname().stream().anyMatch(sl -> pr.getSurname().stream().anyMatch(sr -> sr.equalsIgnoreCase(sl)))) {
// If one of them has only a surname and is the same we can say that they are the same author
if ((pl.getName() == null || pl.getName().stream().allMatch(StringUtils::isBlank)) ||
(pr.getName() == null || pr.getName().stream().allMatch(StringUtils::isBlank)))
return true;
// The authors have the same initials of Name in common
if (pl
.getName()
.stream()
.anyMatch(
nl -> pr
.getName()
.stream()
.anyMatch(nr -> nr.equalsIgnoreCase(nl))))
return true;
}
// Sometimes we noticed that publication have author wrote in inverse order Surname, Name
// We verify if we have an exact match between name and surname
if (pl.getSurname().stream().anyMatch(sl -> pr.getName().stream().anyMatch(nr -> nr.equalsIgnoreCase(sl))) &&
pl.getName().stream().anyMatch(nl -> pr.getSurname().stream().anyMatch(sr -> sr.equalsIgnoreCase(nl))))
return true;
else
return false;
}
//
/**
* Method to enrich ORCID information in one list of authors based on another list
*
* @param baseAuthor the Author List in the OAF Entity
* @param orcidAuthor The list of ORCID Author intersected
* @return The Author List of the OAF Entity enriched with the orcid Author
*/
public static List<Author> enrichOrcid(List<Author> baseAuthor, List<Author> orcidAuthor) {
if (baseAuthor == null || baseAuthor.isEmpty())
return orcidAuthor;
if (orcidAuthor == null || orcidAuthor.isEmpty())
return baseAuthor;
if (baseAuthor.size() == 1 && orcidAuthor.size() > 10)
return baseAuthor;
final List<Author> oAuthor = new ArrayList<>();
oAuthor.addAll(orcidAuthor);
baseAuthor.forEach(ba -> {
Optional<Author> aMatch = oAuthor.stream().filter(oa -> checkORCIDSimilarity(ba, oa)).findFirst();
if (aMatch.isPresent()) {
final Author sameAuthor = aMatch.get();
addPid(ba, sameAuthor.getPid());
oAuthor.remove(sameAuthor);
}
});
return baseAuthor;
}
private static void addPid(final Author a, final List<StructuredProperty> pids) {
if (a.getPid() == null) {
a.setPid(new ArrayList<>());
}
a.getPid().addAll(pids);
}
public static String pidToComparableString(StructuredProperty pid) {
final String classid = pid.getQualifier().getClassid() != null ? pid.getQualifier().getClassid().toLowerCase()
: "";

@ -506,6 +506,8 @@ public class GraphCleaningFunctions extends CleaningFunctions {
.filter(Objects::nonNull)
.filter(sp -> StringUtils.isNotBlank(sp.getValue()))
.map(GraphCleaningFunctions::cleanValue)
.sorted((s1, s2) -> s2.getValue().length() - s1.getValue().length())
.limit(ModelHardLimits.MAX_ABSTRACTS)
.collect(Collectors.toList()));
}
if (Objects.isNull(r.getResourcetype()) || StringUtils.isBlank(r.getResourcetype().getClassid())) {

@ -64,6 +64,9 @@ public class PrepareAffiliationRelations implements Serializable {
final String pubmedInputPath = parser.get("pubmedInputPath");
log.info("pubmedInputPath: {}", pubmedInputPath);
final String openapcInputPath = parser.get("openapcInputPath");
log.info("openapcInputPath: {}", openapcInputPath);
final String outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath);
@ -85,8 +88,14 @@ public class PrepareAffiliationRelations implements Serializable {
JavaPairRDD<Text, Text> pubmedRelations = prepareAffiliationRelations(
spark, pubmedInputPath, collectedFromPubmed);
List<KeyValue> collectedFromOpenAPC = OafMapperUtils
.listKeyValues(ModelConstants.OPEN_APC_ID, "OpenAPC");
JavaPairRDD<Text, Text> openAPCRelations = prepareAffiliationRelations(
spark, openapcInputPath, collectedFromOpenAPC);
crossrefRelations
.union(pubmedRelations)
.union(openAPCRelations)
.saveAsHadoopFile(
outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);

@ -95,7 +95,7 @@ public class SparkAtomicActionScoreJob implements Serializable {
return projectScores.map((MapFunction<BipProjectModel, Project>) bipProjectScores -> {
Project project = new Project();
project.setId(bipProjectScores.getProjectId());
//project.setId(bipProjectScores.getProjectId());
project.setMeasures(bipProjectScores.toMeasures());
return project;
}, Encoders.bean(Project.class))

@ -75,6 +75,7 @@ public class GetFOSSparkJob implements Serializable {
fosData.map((MapFunction<Row, FOSDataModel>) r -> {
FOSDataModel fosDataModel = new FOSDataModel();
fosDataModel.setDoi(r.getString(0).toLowerCase());
fosDataModel.setOaid(r.getString(1).toLowerCase());
fosDataModel.setLevel1(r.getString(2));
fosDataModel.setLevel2(r.getString(3));
fosDataModel.setLevel3(r.getString(4));

@ -16,12 +16,14 @@ import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.jetbrains.annotations.NotNull;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.model.FOSDataModel;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
import eu.dnetlib.dhp.schema.oaf.Subject;
@ -52,61 +54,34 @@ public class PrepareFOSSparkJob implements Serializable {
final String outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath);
final Boolean distributeDOI = Optional
.ofNullable(parser.get("distributeDoi"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
distributeFOSdois(
spark,
sourcePath,
outputPath);
if (distributeDOI)
distributeFOSdois(
spark,
sourcePath,
outputPath);
else
distributeFOSoaid(spark, sourcePath, outputPath);
});
}
private static void distributeFOSdois(SparkSession spark, String sourcePath, String outputPath) {
private static void distributeFOSoaid(SparkSession spark, String sourcePath, String outputPath) {
Dataset<FOSDataModel> fosDataset = readPath(spark, sourcePath, FOSDataModel.class);
fosDataset
.groupByKey((MapFunction<FOSDataModel, String>) v -> v.getDoi().toLowerCase(), Encoders.STRING())
.groupByKey((MapFunction<FOSDataModel, String>) v -> v.getOaid().toLowerCase(), Encoders.STRING())
.mapGroups((MapGroupsFunction<String, FOSDataModel, Result>) (k, it) -> {
Result r = new Result();
FOSDataModel first = it.next();
r.setId(DHPUtils.generateUnresolvedIdentifier(k, DOI));
HashSet<String> level1 = new HashSet<>();
HashSet<String> level2 = new HashSet<>();
HashSet<String> level3 = new HashSet<>();
HashSet<String> level4 = new HashSet<>();
addLevels(level1, level2, level3, level4, first);
it.forEachRemaining(v -> addLevels(level1, level2, level3, level4, v));
List<Subject> sbjs = new ArrayList<>();
level1
.forEach(l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID)));
level2
.forEach(l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID)));
level3
.forEach(
l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID, true)));
level4
.forEach(
l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID, true)));
r.setSubject(sbjs);
r
.setDataInfo(
OafMapperUtils
.dataInfo(
false, null, true,
false,
OafMapperUtils
.qualifier(
ModelConstants.PROVENANCE_ENRICH,
null,
ModelConstants.DNET_PROVENANCE_ACTIONS,
ModelConstants.DNET_PROVENANCE_ACTIONS),
null));
return r;
return getResult(ModelSupport.getIdPrefix(Result.class) + "|" + k, it);
}, Encoders.bean(Result.class))
.write()
.mode(SaveMode.Overwrite)
@ -114,6 +89,61 @@ public class PrepareFOSSparkJob implements Serializable {
.json(outputPath + "/fos");
}
@NotNull
private static Result getResult(String k, Iterator<FOSDataModel> it) {
Result r = new Result();
FOSDataModel first = it.next();
r.setId(k);
HashSet<String> level1 = new HashSet<>();
HashSet<String> level2 = new HashSet<>();
HashSet<String> level3 = new HashSet<>();
HashSet<String> level4 = new HashSet<>();
addLevels(level1, level2, level3, level4, first);
it.forEachRemaining(v -> addLevels(level1, level2, level3, level4, v));
List<Subject> sbjs = new ArrayList<>();
level1
.forEach(l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID)));
level2
.forEach(l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID)));
level3
.forEach(
l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID, true)));
level4
.forEach(
l -> add(sbjs, getSubject(l, FOS_CLASS_ID, FOS_CLASS_NAME, UPDATE_SUBJECT_FOS_CLASS_ID, true)));
r.setSubject(sbjs);
r
.setDataInfo(
OafMapperUtils
.dataInfo(
false, null, true,
false,
OafMapperUtils
.qualifier(
ModelConstants.PROVENANCE_ENRICH,
null,
ModelConstants.DNET_PROVENANCE_ACTIONS,
ModelConstants.DNET_PROVENANCE_ACTIONS),
null));
return r;
}
private static void distributeFOSdois(SparkSession spark, String sourcePath, String outputPath) {
Dataset<FOSDataModel> fosDataset = readPath(spark, sourcePath, FOSDataModel.class);
fosDataset
.groupByKey((MapFunction<FOSDataModel, String>) v -> v.getDoi().toLowerCase(), Encoders.STRING())
.mapGroups(
(MapGroupsFunction<String, FOSDataModel, Result>) (k,
it) -> getResult(DHPUtils.generateUnresolvedIdentifier(k, DOI), it),
Encoders.bean(Result.class))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath + "/fos");
}
private static void add(List<Subject> sbsjs, Subject sbj) {
if (sbj != null)
sbsjs.add(sbj);

@ -0,0 +1,92 @@
package eu.dnetlib.dhp.actionmanager.fosnodoi;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.IOException;
import java.io.Serializable;
import java.util.*;
import org.apache.commons.cli.ParseException;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.actionmanager.opencitations.model.COCI;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.action.AtomicAction;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.utils.*;
import scala.Tuple2;
public class CreateActionSetSparkJob implements Serializable {
private static final Logger log = LoggerFactory.getLogger(CreateActionSetSparkJob.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
public static void main(final String[] args) throws IOException, ParseException {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
Objects
.requireNonNull(
CreateActionSetSparkJob.class
.getResourceAsStream(
"/eu/dnetlib/dhp/actionmanager/fosnodoi/as_parameters.json"))));
parser.parseArgument(args);
Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String inputPath = parser.get("sourcePath");
log.info("inputPath {}", inputPath);
final String outputPath = parser.get("outputPath");
log.info("outputPath {}", outputPath);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> createActionSet(spark, inputPath, outputPath));
}
private static void createActionSet(SparkSession spark, String inputPath, String outputPath) {
spark
.read()
.textFile(inputPath)
.map(
(MapFunction<String, Result>) value -> OBJECT_MAPPER.readValue(value, Result.class),
Encoders.bean(Result.class))
.toJavaRDD()
.map(p -> new AtomicAction(p.getClass(), p))
.mapToPair(
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
.saveAsHadoopFile(
outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
}
}

@ -22,12 +22,14 @@ import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.actionmanager.opencitations.model.COCI;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.action.AtomicAction;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.dhp.schema.oaf.utils.*;
import eu.dnetlib.dhp.utils.DHPUtils;
@ -37,16 +39,12 @@ public class CreateActionSetSparkJob implements Serializable {
public static final String OPENCITATIONS_CLASSID = "sysimport:crosswalk:opencitations";
public static final String OPENCITATIONS_CLASSNAME = "Imported from OpenCitations";
// DOI-to-DOI citations
public static final String COCI = "COCI";
// PMID-to-PMID citations
public static final String POCI = "POCI";
private static final String DOI_PREFIX = "50|doi_________::";
private static final String PMID_PREFIX = "50|pmid________::";
private static final String ARXIV_PREFIX = "50|arXiv_______::";
private static final String PMCID_PREFIX = "50|pmcid_______::";
private static final String TRUST = "0.91";
private static final Logger log = LoggerFactory.getLogger(CreateActionSetSparkJob.class);
@ -79,38 +77,30 @@ public class CreateActionSetSparkJob implements Serializable {
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));
spark -> extractContent(spark, inputPath, outputPath));
}
private static void extractContent(SparkSession spark, String inputPath, String outputPath,
boolean shouldDuplicateRels) {
private static void extractContent(SparkSession spark, String inputPath, String outputPath) {
getTextTextJavaPairRDD(spark, inputPath, shouldDuplicateRels, COCI)
.union(getTextTextJavaPairRDD(spark, inputPath, shouldDuplicateRels, POCI))
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
getTextTextJavaPairRDD(spark, inputPath)
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class);// , GzipCodec.class);
}
private static JavaPairRDD<Text, Text> getTextTextJavaPairRDD(SparkSession spark, String inputPath,
boolean shouldDuplicateRels, String prefix) {
private static JavaPairRDD<Text, Text> getTextTextJavaPairRDD(SparkSession spark, String inputPath) {
return spark
.read()
.textFile(inputPath + "/" + prefix + "/" + prefix + "_JSON/*")
.textFile(inputPath)
.map(
(MapFunction<String, COCI>) value -> OBJECT_MAPPER.readValue(value, COCI.class),
Encoders.bean(COCI.class))
.flatMap(
(FlatMapFunction<COCI, Relation>) value -> createRelation(
value, shouldDuplicateRels, prefix)
value)
.iterator(),
Encoders.bean(Relation.class))
.filter((FilterFunction<Relation>) Objects::nonNull)
@ -121,34 +111,68 @@ public class CreateActionSetSparkJob implements Serializable {
new Text(OBJECT_MAPPER.writeValueAsString(aa))));
}
private static List<Relation> createRelation(COCI value, boolean duplicate, String p) {
private static List<Relation> createRelation(COCI value) throws JsonProcessingException {
List<Relation> relationList = new ArrayList<>();
String prefix;
String citing;
String cited;
switch (p) {
case COCI:
prefix = DOI_PREFIX;
citing = prefix
switch (value.getCiting_pid()) {
case "doi":
citing = DOI_PREFIX
+ IdentifierFactory
.md5(PidCleaner.normalizePidValue(PidType.doi.toString(), value.getCiting()));
cited = prefix
+ IdentifierFactory
.md5(PidCleaner.normalizePidValue(PidType.doi.toString(), value.getCited()));
break;
case POCI:
prefix = PMID_PREFIX;
citing = prefix
case "pmid":
citing = PMID_PREFIX
+ IdentifierFactory
.md5(PidCleaner.normalizePidValue(PidType.pmid.toString(), value.getCiting()));
cited = prefix
break;
case "arxiv":
citing = ARXIV_PREFIX
+ IdentifierFactory
.md5(PidCleaner.normalizePidValue(PidType.arXiv.toString(), value.getCiting()));
break;
case "pmcid":
citing = PMCID_PREFIX
+ IdentifierFactory
.md5(PidCleaner.normalizePidValue(PidType.pmc.toString(), value.getCiting()));
break;
case "isbn":
case "issn":
return relationList;
default:
throw new IllegalStateException("Invalid prefix: " + new ObjectMapper().writeValueAsString(value));
}
switch (value.getCited_pid()) {
case "doi":
cited = DOI_PREFIX
+ IdentifierFactory
.md5(PidCleaner.normalizePidValue(PidType.doi.toString(), value.getCited()));
break;
case "pmid":
cited = PMID_PREFIX
+ IdentifierFactory
.md5(PidCleaner.normalizePidValue(PidType.pmid.toString(), value.getCited()));
break;
case "arxiv":
cited = ARXIV_PREFIX
+ IdentifierFactory
.md5(PidCleaner.normalizePidValue(PidType.arXiv.toString(), value.getCited()));
break;
case "pmcid":
cited = PMCID_PREFIX
+ IdentifierFactory
.md5(PidCleaner.normalizePidValue(PidType.pmc.toString(), value.getCited()));
break;
case "isbn":
case "issn":
return relationList;
default:
throw new IllegalStateException("Invalid prefix: " + p);
throw new IllegalStateException("Invalid prefix: " + new ObjectMapper().writeValueAsString(value));
}
if (!citing.equals(cited)) {
@ -157,15 +181,6 @@ public class CreateActionSetSparkJob implements Serializable {
getRelation(
citing,
cited, ModelConstants.CITES));
if (duplicate && value.getCiting().endsWith(".refs")) {
citing = prefix + IdentifierFactory
.md5(
CleaningFunctions
.normalizePidValue(
"doi", value.getCiting().substring(0, value.getCiting().indexOf(".refs"))));
relationList.add(getRelation(citing, cited, ModelConstants.CITES));
}
}
return relationList;

@ -12,10 +12,7 @@ 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.apache.hadoop.fs.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -37,17 +34,17 @@ public class GetOpenCitationsRefs implements Serializable {
parser.parseArgument(args);
final String[] inputFile = parser.get("inputFile").split(";");
log.info("inputFile {}", Arrays.asList(inputFile));
// final String[] inputFile = parser.get("inputFile").split(";");
// log.info("inputFile {}", Arrays.asList(inputFile));
final String workingPath = parser.get("workingPath");
log.info("workingPath {}", workingPath);
final String inputPath = parser.get("inputPath");
log.info("inputPath {}", inputPath);
final String hdfsNameNode = parser.get("hdfsNameNode");
log.info("hdfsNameNode {}", hdfsNameNode);
final String prefix = parser.get("prefix");
log.info("prefix {}", prefix);
final String outputPath = parser.get("outputPath");
log.info("outputPath {}", outputPath);
Configuration conf = new Configuration();
conf.set("fs.defaultFS", hdfsNameNode);
@ -56,41 +53,42 @@ public class GetOpenCitationsRefs implements Serializable {
GetOpenCitationsRefs ocr = new GetOpenCitationsRefs();
for (String file : inputFile) {
ocr.doExtract(workingPath + "/Original/" + file, workingPath, fileSystem, prefix);
}
ocr.doExtract(inputPath, outputPath, fileSystem);
}
private void doExtract(String inputFile, String workingPath, FileSystem fileSystem, String prefix)
private void doExtract(String inputPath, String outputPath, 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;
fileName = fileName.substring(0, fileName.lastIndexOf("."));
// count++;
try (
FSDataOutputStream out = fileSystem
.create(new Path(workingPath + "/" + prefix + "/" + fileName + ".gz"));
GZIPOutputStream gzipOs = new GZIPOutputStream(new BufferedOutputStream(out))) {
IOUtils.copy(zis, gzipOs);
RemoteIterator<LocatedFileStatus> fileStatusListIterator = fileSystem
.listFiles(
new Path(inputPath), true);
while (fileStatusListIterator.hasNext()) {
LocatedFileStatus fileStatus = fileStatusListIterator.next();
// do stuff with the file like ...
FSDataInputStream oc_zip = fileSystem.open(fileStatus.getPath());
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;
fileName = fileName.substring(0, fileName.lastIndexOf("."));
// count++;
try (
FSDataOutputStream out = fileSystem
.create(new Path(outputPath + "/" + fileName + ".gz"));
GZIPOutputStream gzipOs = new GZIPOutputStream(new BufferedOutputStream(out))) {
IOUtils.copy(zis, gzipOs);
}
}
}
}
}
}

@ -0,0 +1,171 @@
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.Arrays;
import java.util.Objects;
import java.util.Optional;
import java.util.stream.Collectors;
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.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.ForeachFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.actionmanager.opencitations.model.COCI;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import scala.Tuple2;
/**
* @author miriam.baglioni
* @Date 29/02/24
*/
public class MapOCIdsInPids implements Serializable {
private static final Logger log = LoggerFactory.getLogger(CreateActionSetSparkJob.class);
private static final String DELIMITER = ",";
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(
MapOCIdsInPids.class
.getResourceAsStream(
"/eu/dnetlib/dhp/actionmanager/opencitations/remap_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);
final String outputPath = parser.get("outputPath");
log.info("outputPath {}", outputPath);
final String nameNode = parser.get("nameNode");
log.info("nameNode {}", nameNode);
unzipCorrespondenceFile(inputPath, nameNode);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> mapIdentifiers(spark, inputPath, outputPath));
}
private static void unzipCorrespondenceFile(String inputPath, String hdfsNameNode) throws IOException {
Configuration conf = new Configuration();
conf.set("fs.defaultFS", hdfsNameNode);
final Path path = new Path(inputPath + "/correspondence/omid.zip");
FileSystem fileSystem = FileSystem.get(conf);
FSDataInputStream project_zip = fileSystem.open(path);
try (ZipInputStream zis = new ZipInputStream(project_zip)) {
ZipEntry entry = null;
while ((entry = zis.getNextEntry()) != null) {
if (!entry.isDirectory()) {
String fileName = entry.getName();
byte buffer[] = new byte[1024];
int count;
try (
FSDataOutputStream out = fileSystem
.create(new Path(inputPath + "/correspondence/omid.csv"))) {
while ((count = zis.read(buffer, 0, buffer.length)) != -1)
out.write(buffer, 0, count);
}
}
}
}
}
private static void mapIdentifiers(SparkSession spark, String inputPath, String outputPath) {
Dataset<COCI> coci = spark
.read()
.textFile(inputPath + "/JSON")
.map(
(MapFunction<String, COCI>) value -> OBJECT_MAPPER.readValue(value, COCI.class),
Encoders.bean(COCI.class));
Dataset<Tuple2<String, String>> correspondenceData = spark
.read()
.format("csv")
.option("sep", DELIMITER)
.option("inferSchema", "true")
.option("header", "true")
.option("quotes", "\"")
.load(inputPath + "/correspondence/omid.csv")
.repartition(5000)
.flatMap((FlatMapFunction<Row, Tuple2<String, String>>) r -> {
String ocIdentifier = r.getAs("omid");
String[] correspondentIdentifiers = ((String) r.getAs("id")).split(" ");
return Arrays
.stream(correspondentIdentifiers)
.map(ci -> new Tuple2<String, String>(ocIdentifier, ci))
.collect(Collectors.toList())
.iterator();
}, Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
Dataset<COCI> mappedCitingDataset = coci
.joinWith(correspondenceData, coci.col("citing").equalTo(correspondenceData.col("_1")))
.map((MapFunction<Tuple2<COCI, Tuple2<String, String>>, COCI>) t2 -> {
String correspondent = t2._2()._2();
t2._1().setCiting_pid(correspondent.substring(0, correspondent.indexOf(":")));
t2._1().setCiting(correspondent.substring(correspondent.indexOf(":") + 1));
return t2._1();
}, Encoders.bean(COCI.class));
mappedCitingDataset
.joinWith(correspondenceData, mappedCitingDataset.col("cited").equalTo(correspondenceData.col("_1")))
.map((MapFunction<Tuple2<COCI, Tuple2<String, String>>, COCI>) t2 -> {
String correspondent = t2._2()._2();
t2._1().setCited_pid(correspondent.substring(0, correspondent.indexOf(":")));
t2._1().setCited(correspondent.substring(correspondent.indexOf(":") + 1));
return t2._1();
}, Encoders.bean(COCI.class))
.write()
.mode(SaveMode.Append)
.option("compression", "gzip")
.json(outputPath);
}
}

@ -12,11 +12,9 @@ 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.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.*;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.*;
import org.slf4j.Logger;
@ -42,19 +40,21 @@ public class ReadCOCI implements Serializable {
final String outputPath = parser.get("outputPath");
log.info("outputPath: {}", outputPath);
final String[] inputFile = parser.get("inputFile").split(";");
log.info("inputFile {}", Arrays.asList(inputFile));
final String hdfsNameNode = parser.get("hdfsNameNode");
log.info("hdfsNameNode {}", hdfsNameNode);
Boolean isSparkSessionManaged = isSparkSessionManaged(parser);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String workingPath = parser.get("workingPath");
final String workingPath = parser.get("inputPath");
log.info("workingPath {}", workingPath);
final String format = parser.get("format");
log.info("format {}", format);
SparkConf sconf = new SparkConf();
Configuration conf = new Configuration();
conf.set("fs.defaultFS", hdfsNameNode);
FileSystem fileSystem = FileSystem.get(conf);
final String delimiter = Optional
.ofNullable(parser.get("delimiter"))
.orElse(DEFAULT_DELIMITER);
@ -66,20 +66,21 @@ public class ReadCOCI implements Serializable {
doRead(
spark,
workingPath,
inputFile,
fileSystem,
outputPath,
delimiter,
format);
delimiter);
});
}
private static void doRead(SparkSession spark, String workingPath, String[] inputFiles,
private static void doRead(SparkSession spark, String workingPath, FileSystem fileSystem,
String outputPath,
String delimiter, String format) {
for (String inputFile : inputFiles) {
String pString = workingPath + "/" + inputFile + ".gz";
String delimiter) throws IOException {
RemoteIterator<LocatedFileStatus> fileStatusListIterator = fileSystem
.listFiles(
new Path(workingPath), true);
while (fileStatusListIterator.hasNext()) {
LocatedFileStatus fileStatus = fileStatusListIterator.next();
log.info("extracting file {}", fileStatus.getPath().toString());
Dataset<Row> cociData = spark
.read()
.format("csv")
@ -87,26 +88,26 @@ public class ReadCOCI implements Serializable {
.option("inferSchema", "true")
.option("header", "true")
.option("quotes", "\"")
.load(pString)
.load(fileStatus.getPath().toString())
.repartition(100);
cociData.map((MapFunction<Row, COCI>) row -> {
COCI coci = new COCI();
if (format.equals("COCI")) {
coci.setCiting(row.getString(1));
coci.setCited(row.getString(2));
} else {
coci.setCiting(String.valueOf(row.getInt(1)));
coci.setCited(String.valueOf(row.getInt(2)));
}
coci.setCiting(row.getString(1));
coci.setCited(row.getString(2));
coci.setOci(row.getString(0));
return coci;
}, Encoders.bean(COCI.class))
.filter((FilterFunction<COCI>) c -> c != null)
.write()
.mode(SaveMode.Overwrite)
.mode(SaveMode.Append)
.option("compression", "gzip")
.json(outputPath + inputFile);
.json(outputPath);
fileSystem.rename(fileStatus.getPath(), new Path("/tmp/miriam/OC/DONE"));
}
}

@ -9,8 +9,10 @@ public class COCI implements Serializable {
private String oci;
private String citing;
private String citing_pid;
private String cited;
private String cited_pid;
public String getOci() {
return oci;
@ -25,6 +27,8 @@ public class COCI implements Serializable {
}
public void setCiting(String citing) {
if (citing != null && citing.startsWith("omid:"))
citing = citing.substring(5);
this.citing = citing;
}
@ -33,7 +37,24 @@ public class COCI implements Serializable {
}
public void setCited(String cited) {
if (cited != null && cited.startsWith("omid:"))
cited = cited.substring(5);
this.cited = cited;
}
public String getCiting_pid() {
return citing_pid;
}
public void setCiting_pid(String citing_pid) {
this.citing_pid = citing_pid;
}
public String getCited_pid() {
return cited_pid;
}
public void setCited_pid(String cited_pid) {
this.cited_pid = cited_pid;
}
}

@ -0,0 +1,195 @@
package eu.dnetlib.dhp.actionmanager.transformativeagreement;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.IOException;
import java.io.Serializable;
import java.util.*;
import org.apache.commons.cli.ParseException;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.actionmanager.transformativeagreement.model.TransformativeAgreementModel;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.action.AtomicAction;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.Country;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.utils.*;
import scala.Tuple2;
public class CreateActionSetSparkJob implements Serializable {
private static final Logger log = LoggerFactory.getLogger(CreateActionSetSparkJob.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static final String IREL_PROJECT = "40|100018998___::1e5e62235d094afd01cd56e65112fc63";
private static final String TRANSFORMATIVE_AGREEMENT = "openapc::transformativeagreement";
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/transformativeagreement/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);
final String outputPath = parser.get("outputPath");
log.info("outputPath {}", outputPath);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> createActionSet(spark, inputPath, outputPath));
}
private static void createActionSet(SparkSession spark, String inputPath, String outputPath) {
JavaRDD<AtomicAction> relations = spark
.read()
.textFile(inputPath)
.map(
(MapFunction<String, TransformativeAgreementModel>) value -> OBJECT_MAPPER
.readValue(value, TransformativeAgreementModel.class),
Encoders.bean(TransformativeAgreementModel.class))
.flatMap(
(FlatMapFunction<TransformativeAgreementModel, Relation>) value -> createRelation(
value)
.iterator(),
Encoders.bean(Relation.class))
.filter((FilterFunction<Relation>) Objects::nonNull)
.toJavaRDD()
.map(p -> new AtomicAction(p.getClass(), p));
//TODO relations in stand-by waiting to know if we need to create them or not In case we need just make a union before saving the sequence file
spark
.read()
.textFile(inputPath)
.map(
(MapFunction<String, TransformativeAgreementModel>) value -> OBJECT_MAPPER
.readValue(value, TransformativeAgreementModel.class),
Encoders.bean(TransformativeAgreementModel.class))
.map(
(MapFunction<TransformativeAgreementModel, Result>) value -> createResult(
value),
Encoders.bean(Result.class))
.filter((FilterFunction<Result>) r -> r != 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, GzipCodec.class);
}
private static Result createResult(TransformativeAgreementModel value) {
Result r = new Result();
r
.setId(
"50|doi_________::"
+ IdentifierFactory
.md5(PidCleaner.normalizePidValue(PidType.doi.toString(), value.getDoi())));
r.setTransformativeAgreement(value.getAgreement());
Country country = new Country();
country.setClassid(value.getCountry());
country.setClassname(value.getCountry());
country
.setDataInfo(
OafMapperUtils
.dataInfo(
false, ModelConstants.SYSIMPORT_ACTIONSET, false, false,
OafMapperUtils
.qualifier(
"openapc::transformativeagreement",
"Harvested from Trnasformative Agreement file from OpenAPC",
ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS),
"0.9"));
country.setSchemeid(ModelConstants.DNET_COUNTRY_TYPE);
country.setSchemename(ModelConstants.DNET_COUNTRY_TYPE);
r.setCountry(Arrays.asList(country));
return r;
}
private static List<Relation> createRelation(TransformativeAgreementModel value) {
List<Relation> relationList = new ArrayList<>();
if (value.getAgreement().startsWith("IReL")) {
String paper;
paper = "50|doi_________::"
+ IdentifierFactory
.md5(PidCleaner.normalizePidValue(PidType.doi.toString(), value.getDoi()));
relationList
.add(
getRelation(
paper,
IREL_PROJECT, ModelConstants.IS_PRODUCED_BY));
relationList.add(getRelation(IREL_PROJECT, paper, ModelConstants.PRODUCES));
}
return relationList;
}
public static Relation getRelation(
String source,
String target,
String relClass) {
return OafMapperUtils
.getRelation(
source,
target,
ModelConstants.RESULT_PROJECT,
ModelConstants.OUTCOME,
relClass,
Arrays
.asList(
OafMapperUtils.keyValue(ModelConstants.OPEN_APC_ID, ModelConstants.OPEN_APC_NAME)),
OafMapperUtils
.dataInfo(
false, null, false, false,
OafMapperUtils
.qualifier(
TRANSFORMATIVE_AGREEMENT, "Transformative Agreement",
ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS),
"0.9"),
null);
}
}

@ -0,0 +1,51 @@
package eu.dnetlib.dhp.actionmanager.transformativeagreement.model;
import java.io.Serializable;
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
/**
* @author miriam.baglioni
* @Date 18/12/23
*/
@JsonIgnoreProperties(ignoreUnknown = true)
public class TransformativeAgreementModel implements Serializable {
private String institution;
private String doi;
private String agreement;
private String country;
public String getCountry() {
return country;
}
public void setCountry(String country) {
this.country = country;
}
public String getInstitution() {
return institution;
}
public void setInstitution(String institution) {
this.institution = institution;
}
public String getDoi() {
return doi;
}
public void setDoi(String doi) {
this.doi = doi;
}
public String getAgreement() {
return agreement;
}
public void setAgreement(String agreement) {
this.agreement = agreement;
}
}

@ -48,7 +48,7 @@ public class XSLTTransformationFunction implements MapFunction<MetadataRecord, M
@Override
public MetadataRecord call(MetadataRecord value) {
aggregationCounter.getTotalItems().add(1);
try {
Processor processor = new Processor(false);
processor.registerExtensionFunction(cleanFunction);
@ -60,11 +60,18 @@ public class XSLTTransformationFunction implements MapFunction<MetadataRecord, M
comp.setParameter(datasourceIDParam, new XdmAtomicValue(value.getProvenance().getDatasourceId()));
QName datasourceNameParam = new QName(DATASOURCE_NAME_PARAM);
comp.setParameter(datasourceNameParam, new XdmAtomicValue(value.getProvenance().getDatasourceName()));
XsltExecutable xslt = comp
.compile(new StreamSource(IOUtils.toInputStream(transformationRule, StandardCharsets.UTF_8)));
XdmNode source = processor
.newDocumentBuilder()
.build(new StreamSource(IOUtils.toInputStream(value.getBody(), StandardCharsets.UTF_8)));
XsltExecutable xslt;
XdmNode source;
try {
xslt = comp
.compile(new StreamSource(IOUtils.toInputStream(transformationRule, StandardCharsets.UTF_8)));
source = processor
.newDocumentBuilder()
.build(new StreamSource(IOUtils.toInputStream(value.getBody(), StandardCharsets.UTF_8)));
} catch (Throwable e) {
throw new RuntimeException("Error on parsing xslt", e);
}
try {
XsltTransformer trans = xslt.load();
trans.setInitialContextNode(source);
final StringWriter output = new StringWriter();

@ -17,6 +17,12 @@
"paramDescription": "the path to get the input data from Pubmed",
"paramRequired": true
},
{
"paramName": "oip",
"paramLongName": "openapcInputPath",
"paramDescription": "the path to get the input data from OpenAPC",
"paramRequired": true
},
{
"paramName": "o",
"paramLongName": "outputPath",

@ -31,6 +31,7 @@ spark2SqlQueryExecutionListeners=com.cloudera.spark.lineage.NavigatorQueryListen
# The following is needed as a property of a workflow
oozie.wf.application.path=${oozieTopWfApplicationPath}
crossrefInputPath=/data/bip-affiliations/data.json
crossrefInputPath=/data/bip-affiliations/crossref-data.json
pubmedInputPath=/data/bip-affiliations/pubmed-data.json
openapcInputPath=/data/bip-affiliations/openapc-data.json
outputPath=/tmp/crossref-affiliations-output-v5

@ -9,6 +9,10 @@
<name>pubmedInputPath</name>
<description>the path where to find the inferred affiliation relations from Pubmed</description>
</property>
<property>
<name>openapcInputPath</name>
<description>the path where to find the inferred affiliation relations from OpenAPC</description>
</property>
<property>
<name>outputPath</name>
<description>the path where to store the actionset</description>
@ -102,6 +106,7 @@
</spark-opts>
<arg>--crossrefInputPath</arg><arg>${crossrefInputPath}</arg>
<arg>--pubmedInputPath</arg><arg>${pubmedInputPath}</arg>
<arg>--openapcInputPath</arg><arg>${openapcInputPath}</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
</spark>
<ok to="End"/>

@ -16,5 +16,10 @@
"paramLongName": "outputPath",
"paramDescription": "the path of the new ActionSet",
"paramRequired": true
}
}, {
"paramName": "fd",
"paramLongName": "distributeDoi",
"paramDescription": "the path of the new ActionSet",
"paramRequired": false
}
]

@ -0,0 +1,20 @@
[
{
"paramName": "sp",
"paramLongName": "sourcePath",
"paramDescription": "the zipped opencitations file",
"paramRequired": true
},
{
"paramName": "op",
"paramLongName": "outputPath",
"paramDescription": "the working path",
"paramRequired": true
},
{
"paramName": "issm",
"paramLongName": "isSparkSessionManaged",
"paramDescription": "the hdfs name node",
"paramRequired": false
}
]

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

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

@ -1,13 +1,13 @@
[
{
"paramName": "if",
"paramLongName": "inputFile",
"paramName": "ip",
"paramLongName": "inputPath",
"paramDescription": "the zipped opencitations file",
"paramRequired": true
},
{
"paramName": "wp",
"paramLongName": "workingPath",
"paramName": "op",
"paramLongName": "outputPath",
"paramDescription": "the working path",
"paramRequired": true
},
@ -16,11 +16,5 @@
"paramLongName": "hdfsNameNode",
"paramDescription": "the hdfs name node",
"paramRequired": true
},
{
"paramName": "p",
"paramLongName": "prefix",
"paramDescription": "COCI or POCI",
"paramRequired": true
}
]

@ -1,7 +1,7 @@
[
{
"paramName": "wp",
"paramLongName": "workingPath",
"paramName": "ip",
"paramLongName": "inputPath",
"paramDescription": "the zipped opencitations file",
"paramRequired": true
},
@ -24,15 +24,9 @@
"paramLongName": "outputPath",
"paramDescription": "the hdfs name node",
"paramRequired": true
},
{
"paramName": "if",
"paramLongName": "inputFile",
"paramDescription": "the hdfs name node",
"paramRequired": true
}, {
"paramName": "f",
"paramLongName": "format",
}, {
"paramName": "nn",
"paramLongName": "hdfsNameNode",
"paramDescription": "the hdfs name node",
"paramRequired": true
}

@ -27,7 +27,9 @@
<case to="download">${wf:conf('resumeFrom') eq 'DownloadDump'}</case>
<case to="extract">${wf:conf('resumeFrom') eq 'ExtractContent'}</case>
<case to="read">${wf:conf('resumeFrom') eq 'ReadContent'}</case>
<default to="create_actionset"/> <!-- first action to be done when downloadDump is to be performed -->
<case to="remap">${wf:conf('resumeFrom') eq 'MapContent'}</case>
<case to="create_actionset">${wf:conf('resumeFrom') eq 'CreateAS'}</case>
<default to="deleteoutputpath"/> <!-- first action to be done when downloadDump is to be performed -->
</switch>
</decision>
@ -35,6 +37,15 @@
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="deleteoutputpath">
<fs>
<delete path='${inputPath}'/>
<mkdir path='${inputPath}'/>
</fs>
<ok to="download"/>
<error to="Kill"/>
</action>
<action name="download">
<shell xmlns="uri:oozie:shell-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
@ -47,7 +58,28 @@
</configuration>
<exec>download.sh</exec>
<argument>${filelist}</argument>
<argument>${workingPath}/${prefix}/Original</argument>
<argument>${inputPath}/Original</argument>
<env-var>HADOOP_USER_NAME=${wf:user()}</env-var>
<file>download.sh</file>
<capture-output/>
</shell>
<ok to="download_correspondence"/>
<error to="Kill"/>
</action>
<!-- downloads the correspondence from the omid and the pid (doi, pmid etc)-->
<action name="download_correspondence">
<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_corr.sh</exec>
<argument>${filecorrespondence}</argument>
<argument>${inputPath}/correspondence</argument>
<env-var>HADOOP_USER_NAME=${wf:user()}</env-var>
<file>download.sh</file>
<capture-output/>
@ -60,9 +92,19 @@
<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}/${prefix}</arg>
<arg>--prefix</arg><arg>${prefix}</arg>
<arg>--inputPath</arg><arg>${inputPath}/Original</arg>
<arg>--outputPath</arg><arg>${inputPath}/Extracted</arg>
</java>
<ok to="read"/>
<error to="Kill"/>
</action>
<action name="extract_correspondence">
<java>
<main-class>eu.dnetlib.dhp.actionmanager.opencitations.GetOpenCitationsRefs</main-class>
<arg>--hdfsNameNode</arg><arg>${nameNode}</arg>
<arg>--inputPath</arg><arg>${inputPath}/correspondence</arg>
<arg>--outputPath</arg><arg>${inputPath}/correspondence_extracted</arg>
</java>
<ok to="read"/>
<error to="Kill"/>
@ -85,11 +127,35 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--workingPath</arg><arg>${workingPath}/${prefix}/${prefix}</arg>
<arg>--outputPath</arg><arg>${workingPath}/${prefix}/${prefix}_JSON/</arg>
<arg>--inputPath</arg><arg>${inputPath}/Extracted</arg>
<arg>--outputPath</arg><arg>${inputPath}/JSON</arg>
<arg>--delimiter</arg><arg>${delimiter}</arg>
<arg>--inputFile</arg><arg>${inputFileCoci}</arg>
<arg>--format</arg><arg>${prefix}</arg>
<arg>--hdfsNameNode</arg><arg>${nameNode}</arg>
</spark>
<ok to="remap"/>
<error to="Kill"/>
</action>
<action name="remap">
<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.MapOCIdsInPids</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>${inputPath}</arg>
<arg>--outputPath</arg><arg>${outputPathExtraction}</arg>
<arg>--nameNode</arg><arg>${nameNode}</arg>
</spark>
<ok to="create_actionset"/>
<error to="Kill"/>
@ -112,7 +178,7 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--inputPath</arg><arg>${workingPath}</arg>
<arg>--inputPath</arg><arg>${outputPathExtraction}</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
</spark>
<ok to="End"/>

@ -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": "isSparkSessionManged",
"paramDescription": "the hdfs name node",
"paramRequired": false
},{
"paramName": "nn",
"paramLongName": "nameNode",
"paramDescription": "the hdfs name node",
"paramRequired": true
}
]

@ -0,0 +1,20 @@
[
{
"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
}
]

@ -0,0 +1,30 @@
[
{
"paramName": "issm",
"paramLongName": "isSparkSessionManaged",
"paramDescription": "the hdfs name node",
"paramRequired": false
},
{
"paramName": "d",
"paramLongName": "delimiter",
"paramDescription": "the hdfs name node",
"paramRequired": false
},
{
"paramName": "op",
"paramLongName": "outputPath",
"paramDescription": "the hdfs name node",
"paramRequired": true
},
{
"paramName": "if",
"paramLongName": "inputFile",
"paramDescription": "the hdfs name node",
"paramRequired": true
}
]

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

@ -0,0 +1,82 @@
<workflow-app name="Transfomative Agreement 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>
<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>${inputFile}</argument>
<argument>${workingDir}/transformativeagreement/transformativeAgreement.json</argument>
<env-var>HADOOP_USER_NAME=${wf:user()}</env-var>
<file>download.sh</file>
<capture-output/>
</shell>
<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 the Transformative Agreement</name>
<class>eu.dnetlib.dhp.actionmanager.transformativeagreement.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>${workingDir}/transformativeagreement/</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

@ -303,7 +303,9 @@ base_dc:link (I used dc:identifier)
<oaf:accessrights>
<xsl:choose>
<xsl:when test="//base_dc:oa[.='0']">CLOSED</xsl:when>
<xsl:when test="//base_dc:oa[.='1']">OPEN</xsl:when>
<xsl:when test="//base_dc:oa[.='2']">UNKNOWN</xsl:when>
<xsl:when test="//base_dc:rightsnorm">
<xsl:value-of select="vocabulary:clean(//base_dc:rightsnorm, 'dnet:access_modes')" />
</xsl:when>

@ -153,14 +153,18 @@
</xsl:for-each>
</datacite:subjects>
<datacite:publisher>
<xsl:value-of select="normalize-space(//dc:publisher)" />
</datacite:publisher>
<datacite:publicationYear>
<xsl:value-of select="normalize-space(//base_dc:year)" />
</datacite:publicationYear>
<xsl:for-each select="//dc:publisher">
<datacite:publisher>
<xsl:value-of select="normalize-space(.)" />
</datacite:publisher>
</xsl:for-each>
<xsl:for-each select="//base_dc:year">
<datacite:publicationYear>
<xsl:value-of select="normalize-space(.)" />
</datacite:publicationYear>
</xsl:for-each>
<datacite:formats>
<xsl:for-each select="//dc:format">
<datacite:format>
@ -174,6 +178,9 @@
</datacite:language>
<oaf:accessrights>
<xsl:if test="//base_dc:oa[.='0']">
<datacite:rights rightsURI="http://purl.org/coar/access_right/c_16ec">restricted access</datacite:rights>
</xsl:if>
<xsl:if test="//base_dc:oa[.='1']">
<datacite:rights rightsURI="http://purl.org/coar/access_right/c_abf2">open access</datacite:rights>
</xsl:if>
@ -337,7 +344,9 @@
<oaf:accessrights>
<xsl:choose>
<xsl:when test="//base_dc:oa[.='0']">CLOSED</xsl:when>
<xsl:when test="//base_dc:oa[.='1']">OPEN</xsl:when>
<xsl:when test="//base_dc:oa[.='2']">UNKNOWN</xsl:when>
<xsl:when test="//base_dc:rightsnorm">
<xsl:value-of select="vocabulary:clean(//base_dc:rightsnorm, 'dnet:access_modes')" />
</xsl:when>

@ -78,10 +78,6 @@ public class PrepareAffiliationRelationsTest {
.getResource("/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror.json")
.getPath();
String pubmedAffiliationRelationsPath = getClass()
.getResource("/eu/dnetlib/dhp/actionmanager/bipaffiliations/doi_to_ror.json")
.getPath();
String outputPath = workingDir.toString() + "/actionSet";
PrepareAffiliationRelations
@ -89,7 +85,8 @@ public class PrepareAffiliationRelationsTest {
new String[] {
"-isSparkSessionManaged", Boolean.FALSE.toString(),
"-crossrefInputPath", crossrefAffiliationRelationPath,
"-pubmedInputPath", pubmedAffiliationRelationsPath,
"-pubmedInputPath", crossrefAffiliationRelationPath,
"-openapcInputPath", crossrefAffiliationRelationPath,
"-outputPath", outputPath
});
@ -106,7 +103,7 @@ public class PrepareAffiliationRelationsTest {
// );
// }
// count the number of relations
assertEquals(40, tmp.count());
assertEquals(60, tmp.count());
Dataset<Relation> dataset = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class));
dataset.createOrReplaceTempView("result");
@ -117,7 +114,7 @@ public class PrepareAffiliationRelationsTest {
// verify that we have equal number of bi-directional relations
Assertions
.assertEquals(
20, execVerification
30, execVerification
.filter(
"relClass='" + ModelConstants.HAS_AUTHOR_INSTITUTION + "'")
.collectAsList()
@ -125,7 +122,7 @@ public class PrepareAffiliationRelationsTest {
Assertions
.assertEquals(
20, execVerification
30, execVerification
.filter(
"relClass='" + ModelConstants.IS_AUTHOR_INSTITUTION_OF + "'")
.collectAsList()

@ -0,0 +1,104 @@
package eu.dnetlib.dhp.actionmanager.fosnodoi;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
import org.junit.jupiter.api.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.GetFOSSparkJob;
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.PrepareTest;
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.ProduceTest;
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.model.FOSDataModel;
/**
* @author miriam.baglioni
* @Date 13/02/23
*/
public class GetFosTest {
private static final Logger log = LoggerFactory.getLogger(ProduceTest.class);
private static Path workingDir;
private static SparkSession spark;
private static LocalFileSystem fs;
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
@BeforeAll
public static void beforeAll() throws IOException {
workingDir = Files.createTempDirectory(PrepareTest.class.getSimpleName());
fs = FileSystem.getLocal(new Configuration());
log.info("using work dir {}", workingDir);
SparkConf conf = new SparkConf();
conf.setAppName(ProduceTest.class.getSimpleName());
conf.setMaster("local[*]");
conf.set("spark.driver.host", "localhost");
conf.set("hive.metastore.local", "true");
conf.set("spark.ui.enabled", "false");
conf.set("spark.sql.warehouse.dir", workingDir.toString());
conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString());
spark = SparkSession
.builder()
.appName(PrepareTest.class.getSimpleName())
.config(conf)
.getOrCreate();
}
@AfterAll
public static void afterAll() throws IOException {
FileUtils.deleteDirectory(workingDir.toFile());
spark.stop();
}
@Test
@Disabled
void test3() throws Exception {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/actionmanager/fosnodoi/fosnodoi.csv")
.getPath();
final String outputPath = workingDir.toString() + "/fos.json";
GetFOSSparkJob
.main(
new String[] {
"--isSparkSessionManaged", Boolean.FALSE.toString(),
"--sourcePath", sourcePath,
"-outputPath", outputPath,
"-delimiter", ","
});
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<FOSDataModel> tmp = sc
.textFile(outputPath)
.map(item -> OBJECT_MAPPER.readValue(item, FOSDataModel.class));
tmp.foreach(t -> Assertions.assertTrue(t.getOaid() != null));
tmp.foreach(t -> Assertions.assertTrue(t.getLevel1() != null));
tmp.foreach(t -> Assertions.assertTrue(t.getLevel2() != null));
tmp.foreach(t -> Assertions.assertTrue(t.getLevel3() != null));
tmp.foreach(t -> System.out.println(new ObjectMapper().writeValueAsString(t)));
}
}

@ -0,0 +1,99 @@
package eu.dnetlib.dhp.actionmanager.fosnodoi;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.PrepareFOSSparkJob;
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.PrepareSDGSparkJob;
import eu.dnetlib.dhp.actionmanager.createunresolvedentities.ProduceTest;
import eu.dnetlib.dhp.schema.oaf.Result;
public class PrepareTest {
private static final Logger log = LoggerFactory.getLogger(ProduceTest.class);
private static Path workingDir;
private static SparkSession spark;
private static LocalFileSystem fs;
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
@BeforeAll
public static void beforeAll() throws IOException {
workingDir = Files.createTempDirectory(PrepareTest.class.getSimpleName());
fs = FileSystem.getLocal(new Configuration());
log.info("using work dir {}", workingDir);
SparkConf conf = new SparkConf();
conf.setAppName(ProduceTest.class.getSimpleName());
conf.setMaster("local[*]");
conf.set("spark.driver.host", "localhost");
conf.set("hive.metastore.local", "true");
conf.set("spark.ui.enabled", "false");
conf.set("spark.sql.warehouse.dir", workingDir.toString());
conf.set("hive.metastore.warehouse.dir", workingDir.resolve("warehouse").toString());
spark = SparkSession
.builder()
.appName(PrepareTest.class.getSimpleName())
.config(conf)
.getOrCreate();
}
@AfterAll
public static void afterAll() throws IOException {
FileUtils.deleteDirectory(workingDir.toFile());
spark.stop();
}
@Test
void fosPrepareTest() throws Exception {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/actionmanager/fosnodoi/fosnodoi.json")
.getPath();
PrepareFOSSparkJob
.main(
new String[] {
"--isSparkSessionManaged", Boolean.FALSE.toString(),
"--sourcePath", sourcePath,
"-outputPath", workingDir.toString() + "/work",
"-distributeDoi", Boolean.FALSE.toString()
});
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<Result> tmp = sc
.textFile(workingDir.toString() + "/work/fos")
.map(item -> OBJECT_MAPPER.readValue(item, Result.class));
tmp.foreach(t -> System.out.println(new ObjectMapper().writeValueAsString(t)));
}
}

@ -76,7 +76,7 @@ public class CreateOpenCitationsASTest {
String inputPath = getClass()
.getResource(
"/eu/dnetlib/dhp/actionmanager/opencitations/COCI")
"/eu/dnetlib/dhp/actionmanager/opencitations/COCI/inputremap/jsonforas")
.getPath();
CreateActionSetSparkJob
@ -84,8 +84,6 @@ public class CreateOpenCitationsASTest {
new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-shouldDuplicateRels",
Boolean.TRUE.toString(),
"-inputPath",
inputPath,
"-outputPath",
@ -99,9 +97,10 @@ public class CreateOpenCitationsASTest {
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Relation) aa.getPayload()));
assertEquals(31, tmp.count());
Assertions.assertEquals(27, tmp.count());
tmp.foreach(r -> Assertions.assertEquals(1, r.getCollectedfrom().size()));
// tmp.foreach(r -> System.out.println(OBJECT_MAPPER.writeValueAsString(r)));
tmp.foreach(r -> System.out.println(OBJECT_MAPPER.writeValueAsString(r)));
}

@ -0,0 +1,90 @@
package eu.dnetlib.dhp.actionmanager.opencitations;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Encoders;
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.opencitations.model.COCI;
/**
* @author miriam.baglioni
* @Date 07/03/24
*/
public class RemapTest {
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static SparkSession spark;
private static Path workingDir;
private static final Logger log = LoggerFactory
.getLogger(RemapTest.class);
@BeforeAll
public static void beforeAll() throws IOException {
workingDir = Files
.createTempDirectory(RemapTest.class.getSimpleName());
log.info("using work dir {}", workingDir);
SparkConf conf = new SparkConf();
conf.setAppName(RemapTest.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(RemapTest.class.getSimpleName())
.config(conf)
.getOrCreate();
}
@AfterAll
public static void afterAll() throws IOException {
FileUtils.deleteDirectory(workingDir.toFile());
spark.stop();
}
@Test
void testRemap() throws Exception {
String inputPath = getClass()
.getResource(
"/eu/dnetlib/dhp/actionmanager/opencitations/COCI/inputremap")
.getPath();
MapOCIdsInPids
.main(
new String[] {
"-isSparkSessionManged",
Boolean.FALSE.toString(),
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/out/",
"-nameNode", "input1;input2;input3;input4;input5"
});
}
}

@ -0,0 +1,324 @@
package eu.dnetlib.dhp.actionmanager.transformativeagreement;
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.SparkSession;
import org.junit.jupiter.api.AfterAll;
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.opencitations.CreateActionSetSparkJob;
import eu.dnetlib.dhp.actionmanager.opencitations.CreateOpenCitationsASTest;
import eu.dnetlib.dhp.schema.action.AtomicAction;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.schema.oaf.utils.CleaningFunctions;
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory;
/**
* @author miriam.baglioni
* @Date 13/02/24
*/
public class CreateTAActionSetTest {
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(CreateTAActionSetTest.class.getSimpleName());
log.info("using work dir {}", workingDir);
SparkConf conf = new SparkConf();
conf.setAppName(CreateTAActionSetTest.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(CreateTAActionSetTest.class.getSimpleName())
.config(conf)
.getOrCreate();
}
@AfterAll
public static void afterAll() throws IOException {
FileUtils.deleteDirectory(workingDir.toFile());
spark.stop();
}
@Test
void createActionSet() throws Exception {
String inputPath = getClass()
.getResource(
"/eu/dnetlib/dhp/actionmanager/transformativeagreement/facts.json")
.getPath();
eu.dnetlib.dhp.actionmanager.transformativeagreement.CreateActionSetSparkJob
.main(
new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet1"
});
}
@Test
void testNumberofRelations2() throws Exception {
String inputPath = getClass()
.getResource(
"/eu/dnetlib/dhp/actionmanager/opencitations/COCI")
.getPath();
eu.dnetlib.dhp.actionmanager.opencitations.CreateActionSetSparkJob
.main(
new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet2"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet2", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Relation) aa.getPayload()));
assertEquals(23, 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/COCI")
.getPath();
eu.dnetlib.dhp.actionmanager.opencitations.CreateActionSetSparkJob
.main(
new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet3"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet3", 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/COCI")
.getPath();
eu.dnetlib.dhp.actionmanager.opencitations.CreateActionSetSparkJob
.main(
new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet4"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet4", 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(
eu.dnetlib.dhp.actionmanager.opencitations.CreateActionSetSparkJob.OPENCITATIONS_CLASSID,
r.getDataInfo().getProvenanceaction().getClassid());
assertEquals(
eu.dnetlib.dhp.actionmanager.opencitations.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/COCI")
.getPath();
eu.dnetlib.dhp.actionmanager.opencitations.CreateActionSetSparkJob
.main(
new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet5"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet5", 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(23, tmp.filter(r -> r.getRelClass().equals("Cites")).count());
assertEquals(0, tmp.filter(r -> r.getRelClass().equals("IsCitedBy")).count());
}
@Test
void testRelationsSourceTargetPrefix() throws Exception {
String inputPath = getClass()
.getResource(
"/eu/dnetlib/dhp/actionmanager/opencitations/COCI")
.getPath();
eu.dnetlib.dhp.actionmanager.opencitations.CreateActionSetSparkJob
.main(
new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet6"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet6", 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/COCI")
.getPath();
CreateActionSetSparkJob
.main(
new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-inputPath",
inputPath,
"-outputPath",
workingDir.toString() + "/actionSet7"
});
final JavaSparkContext sc = new JavaSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.sequenceFile(workingDir.toString() + "/actionSet7", 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(5, 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()));
}
}

@ -1,6 +1,8 @@
package eu.dnetlib.dhp.collection.plugin.base;
import static org.junit.jupiter.api.Assertions.assertThrows;
import java.io.IOException;
import org.apache.commons.io.IOUtils;
@ -63,6 +65,21 @@ public class BaseTransfomationTest extends AbstractVocabularyTest {
System.out.println(result.getBody());
}
@Test
void testBase2ODF_wrong_date() throws Exception {
final MetadataRecord mr = new MetadataRecord();
mr.setProvenance(new Provenance("DSID", "DSNAME", "PREFIX"));
mr.setBody(IOUtils.toString(getClass().getResourceAsStream("record_wrong_1.xml")));
final XSLTTransformationFunction tr = loadTransformationRule("xml/base2oaf.transformationRule.xml");
assertThrows(NullPointerException.class, () -> {
final MetadataRecord result = tr.call(mr);
System.out.println(result.getBody());
});
}
private XSLTTransformationFunction loadTransformationRule(final String path) throws Exception {
final String xslt = new SAXReader()
.read(this.getClass().getResourceAsStream(path))

@ -3,6 +3,7 @@ package eu.dnetlib.dhp.transformation;
import static eu.dnetlib.dhp.common.Constants.MDSTORE_DATA_PATH;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
import java.io.IOException;
import java.nio.file.Path;
@ -279,6 +280,19 @@ class TransformationJobTest extends AbstractVocabularyTest {
// TODO Create significant Assert
}
@Test
public void testInvalidXSLT() throws Exception{
final MetadataRecord mr = new MetadataRecord();
mr.setProvenance(new Provenance("openaire____::cnr_explora", "CNR ExploRA", "cnr_________"));
mr.setBody(IOUtils.toString(getClass().getResourceAsStream("/eu/dnetlib/dhp/transform/input_cnr_explora.xml")));
// We Load the XSLT transformation Rule from the classpath
final XSLTTransformationFunction tr = loadTransformationRule("/eu/dnetlib/dhp/transform/invalid.xslt");
assertThrows(RuntimeException.class,()->tr.call(mr));
}
private XSLTTransformationFunction loadTransformationRule(final String path) throws Exception {
final String trValue = IOUtils.toString(this.getClass().getResourceAsStream(path));
final LongAccumulator la = new LongAccumulator();

@ -0,0 +1,19 @@
DOI,OAID,level1,level2,level3,level4,score_for_L3,score_for_L4
N/A,78975075580c::e680668c98366c9cd6349afc62486a7f,03 medical and health sciences,0301 basic medicine,030104 developmental biology,N/A,0.5,0.0
N/A,78975075580c::e680668c98366c9cd6349afc62486a7f,03 medical and health sciences,0303 health sciences,030304 developmental biology,N/A,0.5,0.0
N/A,od______2806::a1da9d2678b12969a9ab5f50b5e71d0a,05 social sciences,0501 psychology and cognitive sciences,050109 social psychology,05010904 Group processes/Collective identity,0.5589094161987305,0.5166763067245483
N/A,od______2806::a1da9d2678b12969a9ab5f50b5e71d0a,05 social sciences,0501 psychology and cognitive sciences,050105 experimental psychology,05010501 Emotion/Affective science,0.44109055399894714,0.4833236634731293
N/A,doajarticles::76535d77fd2a5fe9810aefafffb8ef6c,05 social sciences,0502 economics and business,050203 business & management,05020302 Supply chain management/Business terms,0.5459638833999634,0.5460261106491089
N/A,doajarticles::76535d77fd2a5fe9810aefafffb8ef6c,05 social sciences,0502 economics and business,050211 marketing,05021102 Services marketing/Retailing,0.4540362060070038,0.4539738595485687
N/A,od_______156::a3a0119c6d9d3a66943f8da042e97a5e,01 natural sciences,0105 earth and related environmental sciences,010504 meteorology & atmospheric sciences,01050407 Geomagnetism/Ionosphere,0.5131047964096069,0.4990350902080536
N/A,od_______156::a3a0119c6d9d3a66943f8da042e97a5e,01 natural sciences,0105 earth and related environmental sciences,010502 geochemistry & geophysics,01050203 Seismology/Seismology measurement,0.4868951737880707,0.500964879989624
N/A,od______2806::4b9a664dd6b8b04204cb613e7bc9c873,03 medical and health sciences,0302 clinical medicine,030220 oncology & carcinogenesis,03022002 Medical imaging/Medical physics,0.5068133473396301,0.10231181626910052
N/A,od______2806::4b9a664dd6b8b04204cb613e7bc9c873,03 medical and health sciences,0302 clinical medicine,030204 cardiovascular system & hematology,N/A,0.49318668246269226,0.0
N/A,od______3341::ef754de29464abf9bc9b99664630ce74,03 medical and health sciences,0302 clinical medicine,030220 oncology & carcinogenesis,03022012 Oncology/Infectious causes of cancer,0.5,0.5
N/A,od______3341::ef754de29464abf9bc9b99664630ce74,03 medical and health sciences,0302 clinical medicine,030220 oncology & carcinogenesis,03022012 Oncology/Infectious causes of cancer,0.5,0.5
N/A,od______3978::6704dcced0fe3dd6fbf985dc2507f61c,03 medical and health sciences,0302 clinical medicine,030217 neurology & neurosurgery,03021702 Aging-associated diseases/Cognitive disorders,0.5134317874908447,0.09614889098529535
N/A,od______3978::6704dcced0fe3dd6fbf985dc2507f61c,03 medical and health sciences,0301 basic medicine,030104 developmental biology,N/A,0.48656824231147766,0.0
N/A,dedup_wf_001::b77264819800b90c0328c4d17eea5c1a,02 engineering and technology,0209 industrial biotechnology,020901 industrial engineering & automation,02090105 Control theory/Advanced driver assistance systems,0.5178514122962952,0.5198937654495239
N/A,dedup_wf_001::b77264819800b90c0328c4d17eea5c1a,02 engineering and technology,"0202 electrical engineering, electronic engineering, information engineering",020201 artificial intelligence & image processing,02020108 Fuzzy logic/Artificial neural networks/Computational neuroscience,0.48214852809906006,0.4801062345504761
N/A,od______2806::a938609e9f36ada6629a1bcc50c88230,03 medical and health sciences,0302 clinical medicine,030217 neurology & neurosurgery,03021708 Neurotrauma/Stroke,0.5014800429344177,0.5109656453132629
N/A,od______2806::a938609e9f36ada6629a1bcc50c88230,02 engineering and technology,0206 medical engineering,020601 biomedical engineering,02060102 Medical terminology/Patient,0.4985199570655823,0.4890343248844147
1 DOI OAID level1 level2 level3 level4 score_for_L3 score_for_L4
2 N/A 78975075580c::e680668c98366c9cd6349afc62486a7f 03 medical and health sciences 0301 basic medicine 030104 developmental biology N/A 0.5 0.0
3 N/A 78975075580c::e680668c98366c9cd6349afc62486a7f 03 medical and health sciences 0303 health sciences 030304 developmental biology N/A 0.5 0.0
4 N/A od______2806::a1da9d2678b12969a9ab5f50b5e71d0a 05 social sciences 0501 psychology and cognitive sciences 050109 social psychology 05010904 Group processes/Collective identity 0.5589094161987305 0.5166763067245483
5 N/A od______2806::a1da9d2678b12969a9ab5f50b5e71d0a 05 social sciences 0501 psychology and cognitive sciences 050105 experimental psychology 05010501 Emotion/Affective science 0.44109055399894714 0.4833236634731293
6 N/A doajarticles::76535d77fd2a5fe9810aefafffb8ef6c 05 social sciences 0502 economics and business 050203 business & management 05020302 Supply chain management/Business terms 0.5459638833999634 0.5460261106491089
7 N/A doajarticles::76535d77fd2a5fe9810aefafffb8ef6c 05 social sciences 0502 economics and business 050211 marketing 05021102 Services marketing/Retailing 0.4540362060070038 0.4539738595485687
8 N/A od_______156::a3a0119c6d9d3a66943f8da042e97a5e 01 natural sciences 0105 earth and related environmental sciences 010504 meteorology & atmospheric sciences 01050407 Geomagnetism/Ionosphere 0.5131047964096069 0.4990350902080536
9 N/A od_______156::a3a0119c6d9d3a66943f8da042e97a5e 01 natural sciences 0105 earth and related environmental sciences 010502 geochemistry & geophysics 01050203 Seismology/Seismology measurement 0.4868951737880707 0.500964879989624
10 N/A od______2806::4b9a664dd6b8b04204cb613e7bc9c873 03 medical and health sciences 0302 clinical medicine 030220 oncology & carcinogenesis 03022002 Medical imaging/Medical physics 0.5068133473396301 0.10231181626910052
11 N/A od______2806::4b9a664dd6b8b04204cb613e7bc9c873 03 medical and health sciences 0302 clinical medicine 030204 cardiovascular system & hematology N/A 0.49318668246269226 0.0
12 N/A od______3341::ef754de29464abf9bc9b99664630ce74 03 medical and health sciences 0302 clinical medicine 030220 oncology & carcinogenesis 03022012 Oncology/Infectious causes of cancer 0.5 0.5
13 N/A od______3341::ef754de29464abf9bc9b99664630ce74 03 medical and health sciences 0302 clinical medicine 030220 oncology & carcinogenesis 03022012 Oncology/Infectious causes of cancer 0.5 0.5
14 N/A od______3978::6704dcced0fe3dd6fbf985dc2507f61c 03 medical and health sciences 0302 clinical medicine 030217 neurology & neurosurgery 03021702 Aging-associated diseases/Cognitive disorders 0.5134317874908447 0.09614889098529535
15 N/A od______3978::6704dcced0fe3dd6fbf985dc2507f61c 03 medical and health sciences 0301 basic medicine 030104 developmental biology N/A 0.48656824231147766 0.0
16 N/A dedup_wf_001::b77264819800b90c0328c4d17eea5c1a 02 engineering and technology 0209 industrial biotechnology 020901 industrial engineering & automation 02090105 Control theory/Advanced driver assistance systems 0.5178514122962952 0.5198937654495239
17 N/A dedup_wf_001::b77264819800b90c0328c4d17eea5c1a 02 engineering and technology 0202 electrical engineering, electronic engineering, information engineering 020201 artificial intelligence & image processing 02020108 Fuzzy logic/Artificial neural networks/Computational neuroscience 0.48214852809906006 0.4801062345504761
18 N/A od______2806::a938609e9f36ada6629a1bcc50c88230 03 medical and health sciences 0302 clinical medicine 030217 neurology & neurosurgery 03021708 Neurotrauma/Stroke 0.5014800429344177 0.5109656453132629
19 N/A od______2806::a938609e9f36ada6629a1bcc50c88230 02 engineering and technology 0206 medical engineering 020601 biomedical engineering 02060102 Medical terminology/Patient 0.4985199570655823 0.4890343248844147

@ -0,0 +1,18 @@
{"doi":"n/a","oaid":"od______3341::ef754de29464abf9bc9b99664630ce74","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030220 oncology & carcinogenesis","level4":"03022012 Oncology/Infectious causes of cancer","scoreL3":"0.5","scoreL4":"0.5"}
{"doi":"n/a","oaid":"78975075580c::e680668c98366c9cd6349afc62486a7f","level1":"03 medical and health sciences","level2":"0301 basic medicine","level3":"030104 developmental biology","level4":"N/A","scoreL3":"0.5","scoreL4":"0.0"}
{"doi":"n/a","oaid":"od______3341::ef754de29464abf9bc9b99664630ce74","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030220 oncology & carcinogenesis","level4":"03022012 Oncology/Infectious causes of cancer","scoreL3":"0.5","scoreL4":"0.5"}
{"doi":"n/a","oaid":"78975075580c::e680668c98366c9cd6349afc62486a7f","level1":"03 medical and health sciences","level2":"0303 health sciences","level3":"030304 developmental biology","level4":"N/A","scoreL3":"0.5","scoreL4":"0.0"}
{"doi":"n/a","oaid":"od______3978::6704dcced0fe3dd6fbf985dc2507f61c","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030217 neurology & neurosurgery","level4":"03021702 Aging-associated diseases/Cognitive disorders","scoreL3":"0.5134317874908447","scoreL4":"0.09614889098529535"}
{"doi":"n/a","oaid":"od______2806::a1da9d2678b12969a9ab5f50b5e71d0a","level1":"05 social sciences","level2":"0501 psychology and cognitive sciences","level3":"050109 social psychology","level4":"05010904 Group processes/Collective identity","scoreL3":"0.5589094161987305","scoreL4":"0.5166763067245483"}
{"doi":"n/a","oaid":"od______3978::6704dcced0fe3dd6fbf985dc2507f61c","level1":"03 medical and health sciences","level2":"0301 basic medicine","level3":"030104 developmental biology","level4":"N/A","scoreL3":"0.48656824231147766","scoreL4":"0.0"}
{"doi":"n/a","oaid":"od______2806::a1da9d2678b12969a9ab5f50b5e71d0a","level1":"05 social sciences","level2":"0501 psychology and cognitive sciences","level3":"050105 experimental psychology","level4":"05010501 Emotion/Affective science","scoreL3":"0.44109055399894714","scoreL4":"0.4833236634731293"}
{"doi":"n/a","oaid":"dedup_wf_001::b77264819800b90c0328c4d17eea5c1a","level1":"02 engineering and technology","level2":"0209 industrial biotechnology","level3":"020901 industrial engineering & automation","level4":"02090105 Control theory/Advanced driver assistance systems","scoreL3":"0.5178514122962952","scoreL4":"0.5198937654495239"}
{"doi":"n/a","oaid":"doajarticles::76535d77fd2a5fe9810aefafffb8ef6c","level1":"05 social sciences","level2":"0502 economics and business","level3":"050203 business & management","level4":"05020302 Supply chain management/Business terms","scoreL3":"0.5459638833999634","scoreL4":"0.5460261106491089"}
{"doi":"n/a","oaid":"doajarticles::76535d77fd2a5fe9810aefafffb8ef6c","level1":"05 social sciences","level2":"0502 economics and business","level3":"050211 marketing","level4":"05021102 Services marketing/Retailing","scoreL3":"0.4540362060070038","scoreL4":"0.4539738595485687"}
{"doi":"n/a","oaid":"dedup_wf_001::b77264819800b90c0328c4d17eea5c1a","level1":"02 engineering and technology","level2":"0202 electrical engineering, electronic engineering, information engineering","level3":"020201 artificial intelligence & image processing","level4":"02020108 Fuzzy logic/Artificial neural networks/Computational neuroscience","scoreL3":"0.48214852809906006","scoreL4":"0.4801062345504761"}
{"doi":"n/a","oaid":"od_______156::a3a0119c6d9d3a66943f8da042e97a5e","level1":"01 natural sciences","level2":"0105 earth and related environmental sciences","level3":"010504 meteorology & atmospheric sciences","level4":"01050407 Geomagnetism/Ionosphere","scoreL3":"0.5131047964096069","scoreL4":"0.4990350902080536"}
{"doi":"n/a","oaid":"od______2806::a938609e9f36ada6629a1bcc50c88230","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030217 neurology & neurosurgery","level4":"03021708 Neurotrauma/Stroke","scoreL3":"0.5014800429344177","scoreL4":"0.5109656453132629"}
{"doi":"n/a","oaid":"od_______156::a3a0119c6d9d3a66943f8da042e97a5e","level1":"01 natural sciences","level2":"0105 earth and related environmental sciences","level3":"010502 geochemistry & geophysics","level4":"01050203 Seismology/Seismology measurement","scoreL3":"0.4868951737880707","scoreL4":"0.500964879989624"}
{"doi":"n/a","oaid":"od______2806::a938609e9f36ada6629a1bcc50c88230","level1":"02 engineering and technology","level2":"0206 medical engineering","level3":"020601 biomedical engineering","level4":"02060102 Medical terminology/Patient","scoreL3":"0.4985199570655823","scoreL4":"0.4890343248844147"}
{"doi":"n/a","oaid":"od______2806::4b9a664dd6b8b04204cb613e7bc9c873","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030220 oncology & carcinogenesis","level4":"03022002 Medical imaging/Medical physics","scoreL3":"0.5068133473396301","scoreL4":"0.10231181626910052"}
{"doi":"n/a","oaid":"od______2806::4b9a664dd6b8b04204cb613e7bc9c873","level1":"03 medical and health sciences","level2":"0302 clinical medicine","level3":"030204 cardiovascular system & hematology","level4":"N/A","scoreL3":"0.49318668246269226","scoreL4":"0.0"}

@ -0,0 +1,31 @@
{"cited":"br/061201599020", "citing":"br/06203041400","oci":"oci:06701327944-06504326071"}
{"cited":"br/061201599020","citing":"br/06502272390","oci":"oci:06502272390-061301355525"}
{"cited":"br/061201599020", "citing":"br/06120941789","oci":"oci:0670804699-067055659"}
{"cited":"br/06210273177","citing":"br/06203041400","oci":"oci:061502003994-062201281456"}
{"cited":"br/06210273177", "citing":"br/06502272390","oci":"oci:06502272390-0660806688"}
{"cited":"br/06210273177", "citing":"br/06120941789","oci":"oci:06502307119-0620223645"}
{"cited":"br/0660613430","citing":"br/06203041400","oci":"oci:061502004011-061902692285"}
{"cited":"br/0660613430", "citing":"br/06502272390","oci":"oci:0660549063-0610398792"}
{"cited":"br/0660613430", "citing":"br/06120941789","oci":"oci:06420189324-06301543046"}
{"cited":"br/062602732073","citing":"br/06203041400","oci":"oci:06380130275-061502004367"}
{"cited":"br/062602732073","citing":"br/06502272390","oci":"oci:062403449086-062501448395"}
{"cited":"br/062602732073","citing":"br/06120941789","oci":"oci:06420189328-061202007182"}
{"cited":"br/061103703697","citing":"br/06203041400","oci":"oci:062603906965-061701362658"}
{"cited":"br/061103703697", "citing":"br/06502272390","oci":"oci:0670294309-06104327031"}
{"cited":"br/061103703697","citing":"br/06120941789","oci":"oci:061702060228-061301712529"}
{"cited":"br/06230199640", "citing":"br/0670517081","oci":"oci:06901104174-06503692526"}
{"cited":"br/061703513967","citing":"br/061702310822","oci":"oci:061702310822-061703513967"}
{"cited":"br/062104002953","citing":"br/061702311472","oci":"oci:061702311472-062104002953"}
{"cited":"br/061101204417","citing":"br/062102701590","oci":"oci:062102701590-061101204417"}
{"cited":"br/062403787088","citing":"br/061401499173","oci":"oci:061401499173-062403787088"}
{"cited":"br/061203576338","citing":"br/06110279619","oci":"oci:06110279619-061203576338"}
{"cited":"br/061601962207","citing":"br/061502004018","oci":"oci:061502004018-061601962207"}
{"cited":"br/06101014588", "citing":"br/061502004027","oci":"oci:061502004027-06101014588"}
{"cited":"br/06704040804", "citing":"br/06220799044","oci":"oci:06220799044-06704040804"}
{"cited":"br/061401105151","citing":"br/061502004037","oci":"oci:061502004037-061401105151"}
{"cited":"br/0640821079", "citing":"br/061702311537","oci":"oci:061702311537-0640821079"}
{"cited":"br/06604165310", "citing":"br/062501970289","oci":"oci:062501970289-06604165310"}
{"cited":"br/061501351689","citing":"br/061203895786","oci":"oci:061203895786-061501351689"}
{"cited":"br/06202223692", "citing":"br/06110298832","oci":"oci:06110298832-06202223692"}
{"cited":"br/06104310727", "citing":"br/0660439086","oci":"oci:0660439086-06104310727"}
{"cited":"br/06150216214", "citing":"br/06340150329","oci":"oci:06340150329-06150216214"}

@ -0,0 +1,48 @@
omid,id
br/061201599020,doi:10.1142/s0219887817501687
br/06203041400,doi:10.1111/j.1523-5378.2005.00327.x pmid:16104945
br/06210273177,doi:10.1090/qam/20394
br/06502272390,pmid:32235596 doi:10.3390/nano10040644
br/0660613430,doi:10.1007/bf00470411
br/06120941789,doi:10.1098/rspa.2006.1747
br/062602732073,doi:10.1007/978-3-642-38844-6_25
br/06230199640,pmid:25088780 doi:10.1016/j.ymeth.2014.07.008
br/061103703697,pmid:2682767
br/0670517081,doi:10.1016/j.foodpol.2021.102189
br/06502310477,doi:10.1142/s0218127416500450
br/06520113284,doi:10.1109/cfasta57821.2023.10243367
br/062303652439,pmid:5962654 doi:10.1016/0020-708x(66)90001-9
br/06250691436,doi:10.1042/bst20150052 pmid:26009172
br/061201665577,doi:10.1097/00115550-200205000-00018
br/06503490336,pmid:34689254 doi:10.1007/s10072-021-05687-0
br/06220615942,pmid:25626134 doi:10.1016/j.jcis.2015.01.008
br/061103389243,doi:10.4324/9780203702819-10
br/062303011271,doi:10.1109/icassp.2011.5946250
br/061302926083,doi:10.4018/978-1-6684-3937-1.ch002
br/061402485360,doi:10.1109/iciict.2015.7396079
br/06410101083,doi:10.1016/j.autcon.2023.104828
br/062202243386,doi:10.1016/0001-8791(81)90022-1
br/06170421486,doi:10.1130/0016-7606(2003)115<0166:dsagmf>2.0.co;2
br/061201983865,doi:10.4324/9781315109008 isbn:9781315109008
br/061701697230,doi:10.1016/j.trd.2012.07.006
br/061201137111,doi:10.1109/access.2020.2971656
br/06120436283,pmid:2254430 doi:10.1128/jcm.28.11.2551-2554.1990
br/061903968916,doi:10.1111/j.1742-1241.1988.tb08627.x
br/06201583482,doi:10.1016/0016-5085(78)93139-6
br/06130338317,doi:10.2134/agronj1952.00021962004400080013x
br/062601538320,doi:10.1371/journal.pone.0270593 pmid:35789338
br/062401098626,pmid:22385804 doi:10.1016/j.talanta.2011.12.034
br/06190436492,doi:10.1039/c7dt01499f pmid:28644489
br/06202819247,doi:10.1007/978-3-319-45823-6_57
br/0648013560,doi:10.1080/14772000.2012.705356
br/0690214059,doi:10.2752/175630608x329217
br/06601640415,doi:10.1080/18128600508685647
br/061503394761,doi:10.1002/0471443395.img018
br/061702861849,pmid:31203682 doi:10.1080/10428194.2019.1627538
br/06450133713,doi:10.1093/acprof:oso/9780199670888.003.0008
br/0628074892,doi:10.1097/hnp.0000000000000597
br/061601032219,doi:10.1002/bdm.2102
br/06602079930,doi:10.1101/2020.08.25.267500
br/0604192147,doi:10.11501/3307395
br/061101933800,doi:10.1142/s0217732398002242
br/06504184118,pmid:10091417
1 omid id
2 br/061201599020 doi:10.1142/s0219887817501687
3 br/06203041400 doi:10.1111/j.1523-5378.2005.00327.x pmid:16104945
4 br/06210273177 doi:10.1090/qam/20394
5 br/06502272390 pmid:32235596 doi:10.3390/nano10040644
6 br/0660613430 doi:10.1007/bf00470411
7 br/06120941789 doi:10.1098/rspa.2006.1747
8 br/062602732073 doi:10.1007/978-3-642-38844-6_25
9 br/06230199640 pmid:25088780 doi:10.1016/j.ymeth.2014.07.008
10 br/061103703697 pmid:2682767
11 br/0670517081 doi:10.1016/j.foodpol.2021.102189
12 br/06502310477 doi:10.1142/s0218127416500450
13 br/06520113284 doi:10.1109/cfasta57821.2023.10243367
14 br/062303652439 pmid:5962654 doi:10.1016/0020-708x(66)90001-9
15 br/06250691436 doi:10.1042/bst20150052 pmid:26009172
16 br/061201665577 doi:10.1097/00115550-200205000-00018
17 br/06503490336 pmid:34689254 doi:10.1007/s10072-021-05687-0
18 br/06220615942 pmid:25626134 doi:10.1016/j.jcis.2015.01.008
19 br/061103389243 doi:10.4324/9780203702819-10
20 br/062303011271 doi:10.1109/icassp.2011.5946250
21 br/061302926083 doi:10.4018/978-1-6684-3937-1.ch002
22 br/061402485360 doi:10.1109/iciict.2015.7396079
23 br/06410101083 doi:10.1016/j.autcon.2023.104828
24 br/062202243386 doi:10.1016/0001-8791(81)90022-1
25 br/06170421486 doi:10.1130/0016-7606(2003)115<0166:dsagmf>2.0.co;2
26 br/061201983865 doi:10.4324/9781315109008 isbn:9781315109008
27 br/061701697230 doi:10.1016/j.trd.2012.07.006
28 br/061201137111 doi:10.1109/access.2020.2971656
29 br/06120436283 pmid:2254430 doi:10.1128/jcm.28.11.2551-2554.1990
30 br/061903968916 doi:10.1111/j.1742-1241.1988.tb08627.x
31 br/06201583482 doi:10.1016/0016-5085(78)93139-6
32 br/06130338317 doi:10.2134/agronj1952.00021962004400080013x
33 br/062601538320 doi:10.1371/journal.pone.0270593 pmid:35789338
34 br/062401098626 pmid:22385804 doi:10.1016/j.talanta.2011.12.034
35 br/06190436492 doi:10.1039/c7dt01499f pmid:28644489
36 br/06202819247 doi:10.1007/978-3-319-45823-6_57
37 br/0648013560 doi:10.1080/14772000.2012.705356
38 br/0690214059 doi:10.2752/175630608x329217
39 br/06601640415 doi:10.1080/18128600508685647
40 br/061503394761 doi:10.1002/0471443395.img018
41 br/061702861849 pmid:31203682 doi:10.1080/10428194.2019.1627538
42 br/06450133713 doi:10.1093/acprof:oso/9780199670888.003.0008
43 br/0628074892 doi:10.1097/hnp.0000000000000597
44 br/061601032219 doi:10.1002/bdm.2102
45 br/06602079930 doi:10.1101/2020.08.25.267500
46 br/0604192147 doi:10.11501/3307395
47 br/061101933800 doi:10.1142/s0217732398002242
48 br/06504184118 pmid:10091417

@ -0,0 +1,27 @@
{"oci":"oci:06701327944-06504326071","citing":"16104945","citing_pid":"pmid","cited":"10.1142/s0219887817501687","cited_pid":"doi"}
{"oci":"oci:06701327944-06504326071","citing":"10.1111/j.1523-5378.2005.00327.x","citing_pid":"doi","cited":"10.1142/s0219887817501687","cited_pid":"doi"}
{"oci":"oci:06502272390-061301355525","citing":"10.3390/nano10040644","citing_pid":"doi","cited":"10.1142/s0219887817501687","cited_pid":"doi"}
{"oci":"oci:06502272390-061301355525","citing":"32235596","citing_pid":"pmid","cited":"10.1142/s0219887817501687","cited_pid":"doi"}
{"oci":"oci:0670804699-067055659","citing":"10.1098/rspa.2006.1747","citing_pid":"doi","cited":"10.1142/s0219887817501687","cited_pid":"doi"}
{"oci":"oci:061502003994-062201281456","citing":"16104945","citing_pid":"pmid","cited":"10.1090/qam/20394","cited_pid":"doi"}
{"oci":"oci:061502003994-062201281456","citing":"10.1111/j.1523-5378.2005.00327.x","citing_pid":"doi","cited":"10.1090/qam/20394","cited_pid":"doi"}
{"oci":"oci:06502272390-0660806688","citing":"10.3390/nano10040644","citing_pid":"doi","cited":"10.1090/qam/20394","cited_pid":"doi"}
{"oci":"oci:06502272390-0660806688","citing":"32235596","citing_pid":"pmid","cited":"10.1090/qam/20394","cited_pid":"doi"}
{"oci":"oci:06502307119-0620223645","citing":"10.1098/rspa.2006.1747","citing_pid":"doi","cited":"10.1090/qam/20394","cited_pid":"doi"}
{"oci":"oci:061502004011-061902692285","citing":"16104945","citing_pid":"pmid","cited":"10.1007/bf00470411","cited_pid":"doi"}
{"oci":"oci:061502004011-061902692285","citing":"10.1111/j.1523-5378.2005.00327.x","citing_pid":"doi","cited":"10.1007/bf00470411","cited_pid":"doi"}
{"oci":"oci:0660549063-0610398792","citing":"10.3390/nano10040644","citing_pid":"doi","cited":"10.1007/bf00470411","cited_pid":"doi"}
{"oci":"oci:0660549063-0610398792","citing":"32235596","citing_pid":"pmid","cited":"10.1007/bf00470411","cited_pid":"doi"}
{"oci":"oci:06420189324-06301543046","citing":"10.1098/rspa.2006.1747","citing_pid":"doi","cited":"10.1007/bf00470411","cited_pid":"doi"}
{"oci":"oci:06380130275-061502004367","citing":"16104945","citing_pid":"pmid","cited":"10.1007/978-3-642-38844-6_25","cited_pid":"doi"}
{"oci":"oci:06380130275-061502004367","citing":"10.1111/j.1523-5378.2005.00327.x","citing_pid":"doi","cited":"10.1007/978-3-642-38844-6_25","cited_pid":"doi"}
{"oci":"oci:062403449086-062501448395","citing":"10.3390/nano10040644","citing_pid":"doi","cited":"10.1007/978-3-642-38844-6_25","cited_pid":"doi"}
{"oci":"oci:062403449086-062501448395","citing":"32235596","citing_pid":"pmid","cited":"10.1007/978-3-642-38844-6_25","cited_pid":"doi"}
{"oci":"oci:06420189328-061202007182","citing":"10.1098/rspa.2006.1747","citing_pid":"doi","cited":"10.1007/978-3-642-38844-6_25","cited_pid":"doi"}
{"oci":"oci:062603906965-061701362658","citing":"16104945","citing_pid":"pmid","cited":"2682767","cited_pid":"pmid"}
{"oci":"oci:062603906965-061701362658","citing":"10.1111/j.1523-5378.2005.00327.x","citing_pid":"doi","cited":"2682767","cited_pid":"pmid"}
{"oci":"oci:0670294309-06104327031","citing":"10.3390/nano10040644","citing_pid":"doi","cited":"2682767","cited_pid":"pmid"}
{"oci":"oci:0670294309-06104327031","citing":"32235596","citing_pid":"pmid","cited":"2682767","cited_pid":"pmid"}
{"oci":"oci:061702060228-061301712529","citing":"10.1098/rspa.2006.1747","citing_pid":"doi","cited":"2682767","cited_pid":"pmid"}
{"oci":"oci:06901104174-06503692526","citing":"10.1016/j.foodpol.2021.102189","citing_pid":"doi","cited":"10.1016/j.ymeth.2014.07.008","cited_pid":"doi"}
{"oci":"oci:06901104174-06503692526","citing":"10.1016/j.foodpol.2021.102189","citing_pid":"doi","cited":"25088780","cited_pid":"pmid"}

@ -0,0 +1,36 @@
<?xml version="1.0" encoding="UTF-8"?>
<record>
<header xmlns="http://www.openarchives.org/OAI/2.0/">
<identifier>ftunivminnesdc:oai:conservancy.umn.edu:11299/109914</identifier>
<datestamp>2023-07-18T20:05:40Z</datestamp>
</header>
<metadata xmlns="http://www.openarchives.org/OAI/2.0/" xmlns:base_dc="http://oai.base-search.net/base_dc/" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns:dc="http://purl.org/dc/elements/1.1/">
<base_dc:dc xsi:schemaLocation="http://oai.base-search.net/base_dc/ http://oai.base-search.net/base_dc/base_dc.xsd">
<base_dc:global_id>ftunivminnesdc:oai:conservancy.umn.edu:11299/109914</base_dc:global_id>
<base_dc:continent>cna</base_dc:continent>
<base_dc:country>us</base_dc:country>
<base_dc:collection opendoar_id="1008">ftunivminnesdc</base_dc:collection>
<base_dc:collname>University of Minnesota Digital Conservancy</base_dc:collname>
<dc:title>An Experimental Investigation of the influence of an air bubble layer on radiated noise and surface pressure fluctuations in a turbulent boundary layer</dc:title>
<dc:creator>Killen, John M.</dc:creator>
<dc:subject>Boundary layer noise</dc:subject>
<dc:subject>Kinetic energy dissipation</dc:subject>
<dc:subject>Air bubble layer</dc:subject>
<dc:subject>Maximum stable bubble size</dc:subject>
<dc:subject>bubble noise spectra</dc:subject>
<dc:description>between the noise spectra of a layer of air bubbles in a turbulent flow and' a maximum stable bubble size which can exist in the same flow. An air bubbie layer with individual bubble sizes greater than a maximum stable size was introduced into the boundary layer of water flowing along a smooth flat plate. It was found that the maximum stable bubble size was related to the turbulent kinetic energy dissipation as had been previously shown by other investigators ; Office of Naval Research Arlington, Virginia 22217</dc:description>
<dc:date>1981-09-31T21:42:52Z</dc:date>
<base_dc:year>1981</base_dc:year>
<dc:type>Report</dc:type>
<base_dc:typenorm>14</base_dc:typenorm>
<dc:identifier>http://purl.umn.edu/109914</dc:identifier>
<base_dc:link>http://purl.umn.edu/109914</base_dc:link>
<dc:language>en_US</dc:language>
<dc:relation>Project Reports</dc:relation>
<dc:relation>202</dc:relation>
<dc:relation>http://purl.umn.edu/109914</dc:relation>
<base_dc:oa>2</base_dc:oa>
<base_dc:lang>eng</base_dc:lang>
</base_dc:dc>
</metadata>
</record>

@ -0,0 +1,185 @@
<xsl:stylesheet
xmlns:xsl="http://www.w3.org/1999/XSL/Transform"
xmlns:oaire="http://namespace.openaire.eu/schema/oaire/"
xmlns:vocabulary="http://eu/dnetlib/transform/clean"
xmlns:dateCleaner="http://eu/dnetlib/transform/dateISO"
xmlns:oaf="http://namespace.openaire.eu/oaf"
xmlns:oai="http://www.openarchives.org/OAI/2.0/"
xmlns:datacite="http://datacite.org/schema/kernel-4"
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
xmlns:xs="http://www.w3.org/2001/XMLSchema"
xmlns:dr="http://www.driver-repository.eu/namespace/dr"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xmlns:dc="http://purl.org/dc/elements/1.1/"
exclude-result-prefixes="xsl vocabulary dateCleaner" version="2.0">
<xsl:param name="varOfficialName" />
<xsl:param name="varDataSourceId" />
<xsl:param name="varFP7" select="'corda_______::'" />
<xsl:param name="varH2020" select="'corda__h2020::'" />
<xsl:param name="repoCode"
select="substring-before(//*[local-name() = 'header']/*[local-name()='recordIdentifier'], ':')" />
<xsl:param name="index" select="0" />
<xsl:param name="transDate" select="current-dateTime()" />
asdf;klamsdof'sdn &
<xsl:template match="/asdfasdf asdf&">
c:format'" />
</xsl:call-template>
<xsl:call-template name="allElements">
<xsl:with-param name="sourceElement" select="//dc:type" />
<xsl:with-param name="targetElement" select="'dc:type'" />
</xsl:call-template>
<xsl:call-template name="allElements">
<xsl:with-param name="sourceElement" select="//dc:source" />
<xsl:with-param name="targetElement" select="'dc:source'" />
</xsl:call-template>
<dc:language>
<xsl:value-of select="vocabulary:clean( //dc:language, 'dnet:languages')" />
</dc:language>
<xsl:call-template name="allElements">
<xsl:with-param name="sourceElement" select="//dc:rights" />
<xsl:with-param name="targetElement" select="'dc:rights'" />
</xsl:call-template>
<xsl:call-template name="allElements">
<xsl:with-param name="sourceElement" select="//dc:relation[not(starts-with(.,'info:cnr-pdr'))]" />
<xsl:with-param name="targetElement" select="'dc:relation'" />
</xsl:call-template>
<xsl:call-template name="allElements">
<xsl:with-param name="sourceElement" select="//dc:identifier[starts-with(., 'http')]" />
<xsl:with-param name="targetElement" select="'dc:identifier'" />
</xsl:call-template>
<xsl:for-each select="//dc:relation">
<xsl:if test="matches(normalize-space(.), '(info:eu-repo/grantagreement/ec/fp7/)(\d\d\d\d\d\d)(.*)', 'i')">
<oaf:projectid>
<xsl:value-of select="concat($varFP7, replace(normalize-space(.), '(info:eu-repo/grantagreement/ec/fp7/)(\d\d\d\d\d\d)(.*)', '$2', 'i'))" />
</oaf:projectid>
</xsl:if>
<xsl:if test="matches(normalize-space(.), '(info:eu-repo/grantagreement/ec/h2020/)(\d\d\d\d\d\d)(.*)', 'i')">
<oaf:projectid>
<xsl:value-of select="concat($varH2020, replace(normalize-space(.), '(info:eu-repo/grantagreement/ec/h2020/)(\d\d\d\d\d\d)(.*)', '$2', 'i'))" />
</oaf:projectid>
</xsl:if>
</xsl:for-each>
<oaf:accessrights>
<xsl:value-of select="vocabulary:clean( //dc:rights, 'dnet:access_modes')" />
</oaf:accessrights>
<xsl:variable name="varCobjCategory" select="vocabulary:clean( //dc:type, 'dnet:publication_resource')" />
<xsl:variable name="varSuperType" select="vocabulary:clean( $varCobjCategory, 'dnet:result_typologies')" />
<dr:CobjCategory type="{$varSuperType}"><xsl:value-of select="$varCobjCategory" /></dr:CobjCategory>
<xsl:variable name="varRefereedConvt" select="for $i in (//dc:type, //dc:description, //oai:setSpec) return vocabulary:clean( normalize-space($i), 'dnet:review_levels')" />
<xsl:variable name="varRefereedIdntf" select="(//*[string(node-name(.)) = 'dc:identifier' and matches(lower-case(.), '(^|.*[\.\-_/\s\(\)%\d#])pre[\.\-_/\s\(\)%\d#]?prints?([\.\-_/\s\(\)%\d#].*)?$')][count(//dc:identifier) = 1]/'0002', //*[string(node-name(.)) = 'dc:identifier' and matches(lower-case(.), '(^|.*[\.\-_/\s\(\)%\d#])refereed([\.\-_/\s\(\)\d%\d#].*)?$')]/'0001', //*[string(node-name(.)) = 'dc:identifier' and matches(lower-case(.), '.*-peer-reviewed-(fulltext-)?article-.*')]/'0001')" />
<xsl:variable name="varRefereedSourc" select="//*[string(node-name(.)) = ('dc:source', 'dc:publisher') and matches(lower-case(.), '^(.*\s)?pre[\s\-_]*prints?([\s\.,].*)?$')]/'0002'" />
<xsl:variable name="varRefereedDescr" select="(//dc:description[matches(lower-case(.), '.*(this\s*book|this\s*volume|it)\s*constitutes\s*the\s*(thoroughly\s*)?refereed') or matches(lower-case(.), '.*peer[\.\-_/\s\(\)]?review\s*under\s*responsibility\s*of.*') or matches(lower-case(.), '(this|a)\s*(article|preprint)\s*(has\s*been\s*)?(peer[\-\s]*)?reviewed\s*and\s*recommended\s*by\s*peer[\-\s]*community')]/'0001', //dc:description[matches(., '^version\s*(préliminaire.*|preliminary.*|0$)')]/'0002')" />
<xsl:variable name="varRefereedTitle" select="(//dc:title[matches(lower-case(.), '.*\[.*peer[\s\-\._]*review\s*:.*\]\s*$')]/'0001', //dc:title[matches(lower-case(.), '.*\(\s*pre[\s\-\._]*prints?\s*\)\s*$')]/'0002')" />
<xsl:variable name="varRefereedSubjt" select="(//dc:subject[matches(lower-case(.), '^\s*refereed\s*$')][//oaf:datasourceprefix = 'narcis______']/'0001', //dc:subject[matches(lower-case(.), '^\s*no[nt].{0,3}refereed\s*$')][//oaf:datasourceprefix = 'narcis______']/'0002')" />
<xsl:variable name="varRefereed" select="($varRefereedConvt, $varRefereedIdntf, $varRefereedSourc, $varRefereedDescr, $varRefereedTitle, $varRefereedSubjt)" />
<xsl:choose>
<xsl:when test="count($varRefereed[. = '0001']) &gt; 0">
<oaf:refereed>
<xsl:value-of select="'0001'" />
</oaf:refereed>
</xsl:when>
<xsl:when test="count($varRefereed[. = '0002']) &gt; 0">
<oaf:refereed>
<xsl:value-of select="'0002'" />
</oaf:refereed>
</xsl:when>
</xsl:choose>
<oaf:dateAccepted>
<xsl:value-of select="dateCleaner:dateISO( //dc:date[1] )" />
</oaf:dateAccepted>
<xsl:if test="//dc:relation[starts-with(., 'http')] and //dc:rights[.='info:eu-repo/semantics/openAccess']">
<oaf:fulltext>
<xsl:value-of select="//dc:relation[starts-with(., 'http')]" />
</oaf:fulltext>
</xsl:if>
<oaf:hostedBy name="{$varOfficialName}" id="{$varDataSourceId}" />
<oaf:collectedFrom name="{$varOfficialName}" id="{$varDataSourceId}" />
<xsl:variable name="varKnownFileEndings" select="('.bmp', '.doc', '.docx', '.epub', '.flv', '.jpeg', '.jpg', '.m4v', '.mp4', '.mpg', '.odp', '.pdf', '.png', '.ppt', '.tiv', '.txt', '.xls', '.xlsx', '.zip')" />
<xsl:variable name="varIdDoi" select="distinct-values((//dc:identifier[starts-with(., '10.')][matches(., '(10[.][0-9]{4,}[^\s/&gt;]*/[^\s&gt;]+)')], //dc:identifier[starts-with(., 'http') and (contains(., '://dx.doi.org/10.') or contains(., '://doi.org/10.'))]/substring-after(., 'doi.org/'), //dc:identifier[starts-with(lower-case(.), 'doi:10.')]/substring-after(lower-case(.), 'doi:')))" />
<xsl:for-each select="$varIdDoi">
<oaf:identifier identifierType="doi">
<xsl:value-of select="." />
</oaf:identifier>
</xsl:for-each>
<xsl:variable name="varIdHdl" select="distinct-values(//dc:identifier[starts-with(., 'http') and contains(., '://hdl.handle.net/')]/substring-after(., 'hdl.handle.net/'))" />
<xsl:for-each select="$varIdHdl" >
<oaf:identifier identifierType="handle">
<xsl:value-of select="." />
</oaf:identifier>
</xsl:for-each>
<xsl:variable name="varIdUrn" select="distinct-values(//dc:identifier[starts-with(., 'urn:nbn:nl:') or starts-with(., 'URN:NBN:NL:')])" />
<xsl:for-each select="$varIdUrn">
<oaf:identifier identifierType="urn">
<xsl:value-of select="." />
</oaf:identifier>
</xsl:for-each>
<xsl:variable name="varOrigBaseUrl" select="//*[local-name() = 'about']/*[local-name() = 'provenance']//*[local-name() = 'originDescription' and not(./*[local-name() = 'originDescription'])]/*[local-name() = 'baseURL']" />
<xsl:variable name="varIdLdpg" select="distinct-values(//dc:identifier[(contains(substring-after(., '://'), '/') and contains($varOrigBaseUrl, substring-before(substring-after(., '://'), '/'))) or (contains(substring-after(., '://'), ':') and contains($varOrigBaseUrl, substring-before(substring-after(., '://'), ':')))][not(replace(lower-case(.), '.*(\.[a-z]*)$', '$1') = $varKnownFileEndings)])" />
<xsl:for-each select="$varIdLdpg">
<oaf:identifier identifierType="landingPage">
<xsl:value-of select="." />
</oaf:identifier>
</xsl:for-each>
<xsl:variable name="varIdUrl" select="distinct-values(//dc:identifier[starts-with(., 'http')][not(contains(., '://dx.doi.org/') or contains(., '://doi.org/') or contains(., '://hdl.handle.net/'))][count(index-of($varIdLdpg, .)) = 0])" />
<xsl:for-each select="$varIdUrl">
<oaf:identifier identifierType="url">
<xsl:value-of select="." />
</oaf:identifier>
</xsl:for-each>
<xsl:for-each select="//oai:setSpec">
<xsl:variable name="rorDsId" select="vocabulary:clean(., 'cnr:institutes')" />
<xsl:if test="contains($rorDsId, '/ror.org/')">
<oaf:relation relType="resultOrganization" subRelType="affiliation" relClass="hasAuthorInstitution">
<xsl:value-of select="concat('ror_________::', $rorDsId)" />
</oaf:relation>
</xsl:if>
</xsl:for-each>
</metadata>
<xsl:copy-of select="//*[local-name() = 'about']" />
</record>
</xsl:template>
<xsl:template name="allElements">
<xsl:param name="sourceElement" />
<xsl:param name="targetElement" />
<xsl:for-each select="$sourceElement">
<xsl:element name="{$targetElement}">
<xsl:value-of select="normalize-space(.)" />
</xsl:element>
</xsl:for-each>
</xsl:template>
<xsl:template match="//*[local-name() = 'header']">
<xsl:copy>
<xsl:apply-templates select="node()|@*" />
<xsl:element name="dr:dateOfTransformation">
<xsl:value-of select="$transDate" />
</xsl:element>
</xsl:copy>
</xsl:template>
<xsl:template match="node()|@*">
<xsl:copy>
<xsl:apply-templates select="node()|@*" />
</xsl:copy>
</xsl:template>
</xsl:stylesheet>

@ -0,0 +1,39 @@
package eu.dnetlib.dhp.bulktag.actions;
import java.io.Serializable;
import java.util.List;
/**
* @author miriam.baglioni
* @Date 22/01/24
*/
public class Action implements Serializable {
private String clazz;
private String method;
private List<Parameters> params;
public String getClazz() {
return clazz;
}
public void setClazz(String clazz) {
this.clazz = clazz;
}
public String getMethod() {
return method;
}
public void setMethod(String method) {
this.method = method;
}
public List<Parameters> getParams() {
return params;
}
public void setParams(List<Parameters> params) {
this.params = params;
}
}

@ -0,0 +1,45 @@
package eu.dnetlib.dhp.bulktag.actions;
import java.io.Serializable;
/**
* @author miriam.baglioni
* @Date 19/01/24
*/
public class ExecSubstringAction implements Serializable {
private String value;
private String from;
private String to;
public String getValue() {
return value;
}
public void setValue(String value) {
this.value = value;
}
public String getFrom() {
return from;
}
public void setFrom(String from) {
this.from = from;
}
public String getTo() {
return to;
}
public void setTo(String to) {
this.to = to;
}
public String execSubstring() {
return this.value.substring(Integer.valueOf(this.from), Integer.valueOf(this.to));
}
}

@ -0,0 +1,30 @@
package eu.dnetlib.dhp.bulktag.actions;
import java.io.Serializable;
/**
* @author miriam.baglioni
* @Date 22/01/24
*/
public class MapModel implements Serializable {
private String path;
private Action action;
public String getPath() {
return path;
}
public void setPath(String path) {
this.path = path;
}
public Action getAction() {
return action;
}
public void setAction(Action action) {
this.action = action;
}
}

@ -0,0 +1,29 @@
package eu.dnetlib.dhp.bulktag.actions;
import java.io.Serializable;
/**
* @author miriam.baglioni
* @Date 22/01/24
*/
public class Parameters implements Serializable {
private String paramName;
private String paramValue;
public String getParamName() {
return paramName;
}
public void setParamName(String paramName) {
this.paramName = paramName;
}
public String getParamValue() {
return paramValue;
}
public void setParamValue(String paramValue) {
this.paramValue = paramValue;
}
}

@ -4,7 +4,9 @@ package eu.dnetlib.dhp.bulktag.community;
import java.io.Serializable;
import java.util.HashMap;
public class ProtoMap extends HashMap<String, String> implements Serializable {
import eu.dnetlib.dhp.bulktag.actions.MapModel;
public class ProtoMap extends HashMap<String, MapModel> implements Serializable {
public ProtoMap() {
super();

@ -5,6 +5,8 @@ import static eu.dnetlib.dhp.bulktag.community.TaggingConstants.*;
import static eu.dnetlib.dhp.schema.common.ModelConstants.*;
import java.io.Serializable;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.util.*;
import java.util.stream.Collectors;
@ -15,7 +17,10 @@ import org.slf4j.LoggerFactory;
import com.google.gson.Gson;
import com.jayway.jsonpath.DocumentContext;
import com.jayway.jsonpath.JsonPath;
import com.jayway.jsonpath.PathNotFoundException;
import eu.dnetlib.dhp.bulktag.actions.MapModel;
import eu.dnetlib.dhp.bulktag.actions.Parameters;
import eu.dnetlib.dhp.bulktag.eosc.EoscIFTag;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
@ -35,27 +40,59 @@ public class ResultTagger implements Serializable {
return (tmp != clist.size());
}
private Map<String, List<String>> getParamMap(final Result result, Map<String, String> params) {
private Map<String, List<String>> getParamMap(final Result result, Map<String, MapModel> params)
throws NoSuchMethodException, InvocationTargetException {
Map<String, List<String>> param = new HashMap<>();
String json = new Gson().toJson(result, Result.class);
DocumentContext jsonContext = JsonPath.parse(json);
if (params == null) {
params = new HashMap<>();
}
for (String key : params.keySet()) {
MapModel mapModel = params.get(key);
try {
param.put(key, jsonContext.read(params.get(key)));
} catch (com.jayway.jsonpath.PathNotFoundException e) {
String path = mapModel.getPath();
Object obj = jsonContext.read(path);
List<String> pathValue;
if (obj instanceof java.lang.String)
pathValue = Arrays.asList((String) obj);
else
pathValue = (List<String>) obj;
if (Optional.ofNullable(mapModel.getAction()).isPresent()) {
Class<?> c = Class.forName(mapModel.getAction().getClazz());
Object class_instance = c.newInstance();
Method setField = c.getMethod("setValue", String.class);
setField.invoke(class_instance, pathValue.get(0));
for (Parameters p : mapModel.getAction().getParams()) {
setField = c.getMethod("set" + p.getParamName(), String.class);
setField.invoke(class_instance, p.getParamValue());
}
param
.put(
key, Arrays
.asList((String) c.getMethod(mapModel.getAction().getMethod()).invoke(class_instance)));
}
else {
param.put(key, pathValue);
}
} catch (PathNotFoundException | ClassNotFoundException | InstantiationException
| IllegalAccessException e) {
param.put(key, new ArrayList<>());
}
}
return param;
}
public <R extends Result> R enrichContextCriteria(
final R result, final CommunityConfiguration conf, final Map<String, String> criteria) {
final Map<String, List<String>> param = getParamMap(result, criteria);
final R result, final CommunityConfiguration conf, final Map<String, MapModel> criteria)
throws InvocationTargetException, NoSuchMethodException {
// Verify if the entity is deletedbyinference. In case verify if to clean the context list
// from all the zenodo communities
@ -64,6 +101,8 @@ public class ResultTagger implements Serializable {
return result;
}
final Map<String, List<String>> param = getParamMap(result, criteria);
// Execute the EOSCTag for the services
switch (result.getResulttype().getClassid()) {
case PUBLICATION_RESULTTYPE_CLASSID:

@ -0,0 +1,34 @@
package eu.dnetlib.dhp.bulktag.criteria;
import java.io.Serializable;
/**
* @author miriam.baglioni
* @Date 11/11/22
*/
@VerbClass("greater_than")
public class GreatThanVerb implements Selection, Serializable {
private String param;
public GreatThanVerb() {
}
public GreatThanVerb(final String param) {
this.param = param;
}
@Override
public boolean apply(String value) {
return value.compareTo(param) > 0;
}
public String getParam() {
return param;
}
public void setParam(String param) {
this.param = param;
}
}

@ -0,0 +1,34 @@
package eu.dnetlib.dhp.bulktag.criteria;
import java.io.Serializable;
/**
* @author miriam.baglioni
* @Date 11/11/22
*/
@VerbClass("lesser_than")
public class LessThanVerb implements Selection, Serializable {
private String param;
public LessThanVerb() {
}
public LessThanVerb(final String param) {
this.param = param;
}
@Override
public boolean apply(String value) {
return value.compareTo(param) < 0;
}
public String getParam() {
return param;
}
public void setParam(String param) {
this.param = param;
}
}

@ -13,6 +13,7 @@ import org.apache.commons.io.IOUtils;
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.FilterFunction;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
@ -31,18 +32,26 @@ public class BulkTagJobTest {
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
public static final String pathMap = "{ \"author\" : \"$['author'][*]['fullname']\","
+ " \"title\" : \"$['title'][*]['value']\","
+ " \"orcid\" : \"$['author'][*]['pid'][*][?(@['key']=='ORCID')]['value']\","
+ " \"contributor\" : \"$['contributor'][*]['value']\","
+ " \"description\" : \"$['description'][*]['value']\", "
+ " \"subject\" :\"$['subject'][*]['value']\" , " +
"\"fos\" : \"$['subject'][?(@['qualifier']['classid']=='FOS')].value\"," +
"\"sdg\" : \"$['subject'][?(@['qualifier']['classid']=='SDG')].value\"," +
"\"hostedby\" : \"$['instance'][*]['hostedby']['key']\" , " +
"\"collectedfrom\" : \"$['instance'][*]['collectedfrom']['key']\"," +
"\"publisher\":\"$['publisher'].value\"," +
"\"publicationyear\":\"$['dateofacceptance'].value\"} ";
public static final String pathMap = "{\"author\":{\"path\":\"$['author'][*]['fullname']\"}," +
" \"title\":{\"path\":\"$['title'][*]['value']\"}, " +
" \"orcid\":{\"path\":\"$['author'][*]['pid'][*][?(@['qualifier']['classid']=='orcid')]['value']\"} , " +
" \"orcid_pending\":{\"path\":\"$['author'][*]['pid'][*][?(@['qualifier']['classid']=='orcid_pending')]['value']\"} ,"
+
"\"contributor\" : {\"path\":\"$['contributor'][*]['value']\"}," +
" \"description\" : {\"path\":\"$['description'][*]['value']\"}," +
" \"subject\" :{\"path\":\"$['subject'][*]['value']\"}, " +
" \"fos\" : {\"path\":\"$['subject'][?(@['qualifier']['classid']=='FOS')].value\"} , " +
"\"sdg\" : {\"path\":\"$['subject'][?(@['qualifier']['classid']=='SDG')].value\"}," +
"\"journal\":{\"path\":\"$['journal'].name\"}," +
"\"hostedby\":{\"path\":\"$['instance'][*]['hostedby']['key']\"}," +
"\"collectedfrom\":{\"path\":\"$['instance'][*]['collectedfrom']['key']\"}," +
"\"publisher\":{\"path\":\"$['publisher'].value\"}," +
"\"publicationyear\":{\"path\":\"$['dateofacceptance'].value\", " +
" \"action\":{\"clazz\":\"eu.dnetlib.dhp.bulktag.actions.ExecSubstringAction\"," +
"\"method\":\"execSubstring\"," +
"\"params\":[" +
"{\"paramName\":\"From\", \"paramValue\":0}, " +
"{\"paramName\":\"To\",\"paramValue\":4}]}}}";
private static SparkSession spark;
@ -1600,4 +1609,94 @@ public class BulkTagJobTest {
Assertions.assertEquals(0, spark.sql(query).count());
}
@Test
void pubdateTest() throws Exception {
final String pathMap = BulkTagJobTest.pathMap;
SparkBulkTagJob
.main(
new String[] {
"-isSparkSessionManaged", Boolean.FALSE.toString(),
"-sourcePath",
getClass().getResource("/eu/dnetlib/dhp/bulktag/sample/dataset/publicationyear/").getPath(),
"-taggingConf",
IOUtils
.toString(
BulkTagJobTest.class
.getResourceAsStream(
"/eu/dnetlib/dhp/bulktag/communityconfiguration/tagging_conf_publicationdate.xml")),
"-outputPath", workingDir.toString() + "/",
"-pathMap", pathMap
});
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<Dataset> tmp = sc
.textFile(workingDir.toString() + "/dataset")
.map(item -> OBJECT_MAPPER.readValue(item, Dataset.class));
Assertions.assertEquals(10, tmp.count());
org.apache.spark.sql.Dataset<Dataset> verificationDataset = spark
.createDataset(tmp.rdd(), Encoders.bean(Dataset.class));
verificationDataset.createOrReplaceTempView("dataset");
String query = "select id, MyT.id community, MyD.provenanceaction.classid "
+ "from dataset "
+ "lateral view explode(context) c as MyT "
+ "lateral view explode(MyT.datainfo) d as MyD "
+ "where MyD.inferenceprovenance = 'bulktagging'";
org.apache.spark.sql.Dataset<Row> queryResult = spark.sql(query);
queryResult.show(false);
Assertions.assertEquals(5, queryResult.count());
Assertions
.assertEquals(
1,
queryResult
.filter(
(FilterFunction<Row>) r -> r
.getAs("id")
.equals("50|od______3989::02dd5d2c222191b0b9bd4f33c8e96529"))
.count());
Assertions
.assertEquals(
1,
queryResult
.filter(
(FilterFunction<Row>) r -> r
.getAs("id")
.equals("50|od______3989::2f4f3c820c450bd08dac08d07cc82dcf"))
.count());
Assertions
.assertEquals(
1,
queryResult
.filter(
(FilterFunction<Row>) r -> r
.getAs("id")
.equals("50|od______3989::7fcbe3a03280663cddebfd3cb9203177"))
.count());
Assertions
.assertEquals(
1,
queryResult
.filter(
(FilterFunction<Row>) r -> r
.getAs("id")
.equals("50|od______3989::d791339867bec6d3eb2104deeb4e4961"))
.count());
Assertions
.assertEquals(
1,
queryResult
.filter(
(FilterFunction<Row>) r -> r
.getAs("id")
.equals("50|od______3989::d90d3a1f64ad264b5ebed8a35b280343"))
.count());
}
}

@ -1,16 +1,50 @@
package eu.dnetlib.dhp.oa.graph.raw;
import static eu.dnetlib.dhp.schema.common.ModelConstants.*;
import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.*;
import static eu.dnetlib.dhp.schema.common.ModelConstants.AFFILIATION;
import static eu.dnetlib.dhp.schema.common.ModelConstants.DNET_PID_TYPES;
import static eu.dnetlib.dhp.schema.common.ModelConstants.HAS_AUTHOR_INSTITUTION;
import static eu.dnetlib.dhp.schema.common.ModelConstants.IS_AUTHOR_INSTITUTION_OF;
import static eu.dnetlib.dhp.schema.common.ModelConstants.IS_PRODUCED_BY;
import static eu.dnetlib.dhp.schema.common.ModelConstants.OPENAIRE_COAR_RESOURCE_TYPES_3_1;
import static eu.dnetlib.dhp.schema.common.ModelConstants.OUTCOME;
import static eu.dnetlib.dhp.schema.common.ModelConstants.PRODUCES;
import static eu.dnetlib.dhp.schema.common.ModelConstants.REPOSITORY_PROVENANCE_ACTIONS;
import static eu.dnetlib.dhp.schema.common.ModelConstants.RESULT_ORGANIZATION;
import static eu.dnetlib.dhp.schema.common.ModelConstants.RESULT_PROJECT;
import static eu.dnetlib.dhp.schema.common.ModelConstants.UNKNOWN;
import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.createOpenaireId;
import java.util.*;
import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.dataInfo;
import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.field;
import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.journal;
import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.keyValue;
import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.listFields;
import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.oaiIProvenance;
import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.qualifier;
import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.structuredProperty;
import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.subject;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.validator.routines.UrlValidator;
import org.dom4j.*;
import org.dom4j.Document;
import org.dom4j.DocumentException;
import org.dom4j.DocumentFactory;
import org.dom4j.DocumentHelper;
import org.dom4j.Element;
import org.dom4j.Node;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -21,7 +55,29 @@ import eu.dnetlib.dhp.common.Constants;
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
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.AccessRight;
import eu.dnetlib.dhp.schema.oaf.Author;
import eu.dnetlib.dhp.schema.oaf.Context;
import eu.dnetlib.dhp.schema.oaf.Country;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.Dataset;
import eu.dnetlib.dhp.schema.oaf.EoscIfGuidelines;
import eu.dnetlib.dhp.schema.oaf.Field;
import eu.dnetlib.dhp.schema.oaf.GeoLocation;
import eu.dnetlib.dhp.schema.oaf.Instance;
import eu.dnetlib.dhp.schema.oaf.InstanceTypeMapping;
import eu.dnetlib.dhp.schema.oaf.Journal;
import eu.dnetlib.dhp.schema.oaf.KeyValue;
import eu.dnetlib.dhp.schema.oaf.OAIProvenance;
import eu.dnetlib.dhp.schema.oaf.Oaf;
import eu.dnetlib.dhp.schema.oaf.OafEntity;
import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct;
import eu.dnetlib.dhp.schema.oaf.Publication;
import eu.dnetlib.dhp.schema.oaf.Qualifier;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.Software;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
import eu.dnetlib.dhp.schema.oaf.Subject;
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory;
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
@ -42,12 +98,8 @@ public abstract class AbstractMdRecordToOafMapper {
protected static final String DATACITE_SCHEMA_KERNEL_3 = "http://datacite.org/schema/kernel-3";
protected static final String DATACITE_SCHEMA_KERNEL_3_SLASH = "http://datacite.org/schema/kernel-3/";
protected static final Qualifier ORCID_PID_TYPE = qualifier(
ModelConstants.ORCID_PENDING,
ModelConstants.ORCID_CLASSNAME,
DNET_PID_TYPES, DNET_PID_TYPES);
protected static final Qualifier MAG_PID_TYPE = qualifier(
"MAGIdentifier", "Microsoft Academic Graph Identifier", DNET_PID_TYPES, DNET_PID_TYPES);
protected static final Qualifier ORCID_PID_TYPE = qualifier(ModelConstants.ORCID_PENDING, ModelConstants.ORCID_CLASSNAME, DNET_PID_TYPES, DNET_PID_TYPES);
protected static final Qualifier MAG_PID_TYPE = qualifier("MAGIdentifier", "Microsoft Academic Graph Identifier", DNET_PID_TYPES, DNET_PID_TYPES);
protected static final String DEFAULT_TRUST_FOR_VALIDATED_RELS = "0.999";
@ -70,14 +122,14 @@ public abstract class AbstractMdRecordToOafMapper {
static {
IdentifierFactory.PID_AUTHORITY
.keySet()
.stream()
.forEach(entry -> pidTypeWithAuthority.put(entry.toString().toLowerCase(), entry.toString()));
.keySet()
.stream()
.forEach(entry -> pidTypeWithAuthority.put(entry.toString().toLowerCase(), entry.toString()));
}
protected AbstractMdRecordToOafMapper(final VocabularyGroup vocs, final boolean invisible,
final boolean shouldHashId, final boolean forceOriginalId) {
final boolean shouldHashId, final boolean forceOriginalId) {
this.vocs = vocs;
this.invisible = invisible;
this.shouldHashId = shouldHashId;
@ -85,7 +137,7 @@ public abstract class AbstractMdRecordToOafMapper {
}
protected AbstractMdRecordToOafMapper(final VocabularyGroup vocs, final boolean invisible,
final boolean shouldHashId) {
final boolean shouldHashId) {
this.vocs = vocs;
this.invisible = invisible;
this.shouldHashId = shouldHashId;
@ -97,28 +149,22 @@ public abstract class AbstractMdRecordToOafMapper {
DocumentFactory.getInstance().setXPathNamespaceURIs(nsContext);
try {
final Document doc = DocumentHelper
.parseText(
xml
.replaceAll(DATACITE_SCHEMA_KERNEL_4, DATACITE_SCHEMA_KERNEL_3)
.replaceAll(DATACITE_SCHEMA_KERNEL_4_SLASH, DATACITE_SCHEMA_KERNEL_3)
.replaceAll(DATACITE_SCHEMA_KERNEL_3_SLASH, DATACITE_SCHEMA_KERNEL_3));
.parseText(xml
.replaceAll(DATACITE_SCHEMA_KERNEL_4, DATACITE_SCHEMA_KERNEL_3)
.replaceAll(DATACITE_SCHEMA_KERNEL_4_SLASH, DATACITE_SCHEMA_KERNEL_3)
.replaceAll(DATACITE_SCHEMA_KERNEL_3_SLASH, DATACITE_SCHEMA_KERNEL_3));
final KeyValue collectedFrom = getProvenanceDatasource(
doc, "//oaf:collectedFrom/@id", "//oaf:collectedFrom/@name");
final KeyValue collectedFrom = getProvenanceDatasource(doc, "//oaf:collectedFrom/@id", "//oaf:collectedFrom/@name");
if (collectedFrom == null) {
return Lists.newArrayList();
}
if (collectedFrom == null) { return Lists.newArrayList(); }
final KeyValue hostedBy = StringUtils.isBlank(doc.valueOf("//oaf:hostedBy/@id"))
? collectedFrom
: getProvenanceDatasource(doc, "//oaf:hostedBy/@id", "//oaf:hostedBy/@name");
? collectedFrom
: getProvenanceDatasource(doc, "//oaf:hostedBy/@id", "//oaf:hostedBy/@name");
if (hostedBy == null) {
return Lists.newArrayList();
}
if (hostedBy == null) { return Lists.newArrayList(); }
final DataInfo entityInfo = prepareDataInfo(doc, invisible);
final DataInfo entityInfo = prepareDataInfo(doc, this.invisible);
final long lastUpdateTimestamp = new Date().getTime();
final List<Instance> instances = prepareInstances(doc, entityInfo, collectedFrom, hostedBy);
@ -126,7 +172,7 @@ public abstract class AbstractMdRecordToOafMapper {
final String type = getResultType(doc, instances);
return createOafs(doc, type, instances, collectedFrom, entityInfo, lastUpdateTimestamp);
} catch (DocumentException e) {
} catch (final DocumentException e) {
log.error("Error with record:\n" + xml);
return Lists.newArrayList();
}
@ -135,17 +181,17 @@ public abstract class AbstractMdRecordToOafMapper {
protected String getResultType(final Document doc, final List<Instance> instances) {
final String type = doc.valueOf("//dr:CobjCategory/@type");
if (StringUtils.isBlank(type) && vocs.vocabularyExists(ModelConstants.DNET_RESULT_TYPOLOGIES)) {
if (StringUtils.isBlank(type) && this.vocs.vocabularyExists(ModelConstants.DNET_RESULT_TYPOLOGIES)) {
final String instanceType = instances
.stream()
.map(i -> i.getInstancetype().getClassid())
.findFirst()
.filter(s -> !UNKNOWN.equalsIgnoreCase(s))
.orElse("0000"); // Unknown
.stream()
.map(i -> i.getInstancetype().getClassid())
.findFirst()
.filter(s -> !UNKNOWN.equalsIgnoreCase(s))
.orElse("0000"); // Unknown
return Optional
.ofNullable(vocs.getSynonymAsQualifier(ModelConstants.DNET_RESULT_TYPOLOGIES, instanceType))
.map(Qualifier::getClassid)
.orElse("0000");
.ofNullable(this.vocs.getSynonymAsQualifier(ModelConstants.DNET_RESULT_TYPOLOGIES, instanceType))
.map(Qualifier::getClassid)
.orElse("0000");
}
return type;
@ -155,30 +201,27 @@ public abstract class AbstractMdRecordToOafMapper {
final String dsId = doc.valueOf(xpathId);
final String dsName = doc.valueOf(xpathName);
if (StringUtils.isBlank(dsId) || StringUtils.isBlank(dsName)) {
return null;
}
if (StringUtils.isBlank(dsId) || StringUtils.isBlank(dsName)) { return null; }
return keyValue(createOpenaireId(10, dsId, true), dsName);
}
protected List<Oaf> createOafs(
final Document doc,
final String type,
final List<Instance> instances,
final KeyValue collectedFrom,
final DataInfo info,
final long lastUpdateTimestamp) {
final Document doc,
final String type,
final List<Instance> instances,
final KeyValue collectedFrom,
final DataInfo info,
final long lastUpdateTimestamp) {
final OafEntity entity = createEntity(
doc, type, instances, collectedFrom, info, lastUpdateTimestamp);
final OafEntity entity = createEntity(doc, type, instances, collectedFrom, info, lastUpdateTimestamp);
final Set<String> originalId = Sets.newHashSet(entity.getOriginalId());
originalId.add(entity.getId());
entity.setOriginalId(Lists.newArrayList(originalId));
if (!forceOriginalId) {
final String id = IdentifierFactory.createIdentifier(entity, shouldHashId);
if (!this.forceOriginalId) {
final String id = IdentifierFactory.createIdentifier(entity, this.shouldHashId);
if (!id.equals(entity.getId())) {
entity.setId(id);
}
@ -189,7 +232,7 @@ public abstract class AbstractMdRecordToOafMapper {
final DataInfo relationInfo = prepareDataInfo(doc, false);
if (!oafs.isEmpty()) {
Set<Oaf> rels = Sets.newHashSet();
final Set<Oaf> rels = Sets.newHashSet();
rels.addAll(addProjectRels(doc, entity, relationInfo));
rels.addAll(addOtherResultRels(doc, entity, relationInfo));
@ -203,51 +246,52 @@ public abstract class AbstractMdRecordToOafMapper {
}
private OafEntity createEntity(final Document doc,
final String type,
final List<Instance> instances,
final KeyValue collectedFrom,
final DataInfo info,
final long lastUpdateTimestamp) {
final String type,
final List<Instance> instances,
final KeyValue collectedFrom,
final DataInfo info,
final long lastUpdateTimestamp) {
switch (type.toLowerCase()) {
case "publication":
final Publication p = new Publication();
populateResultFields(p, doc, instances, collectedFrom, info, lastUpdateTimestamp);
p.setJournal(prepareJournal(doc, info));
return p;
case "dataset":
final Dataset d = new Dataset();
populateResultFields(d, doc, instances, collectedFrom, info, lastUpdateTimestamp);
d.setStoragedate(prepareDatasetStorageDate(doc, info));
d.setDevice(prepareDatasetDevice(doc, info));
d.setSize(prepareDatasetSize(doc, info));
d.setVersion(prepareDatasetVersion(doc, info));
d.setLastmetadataupdate(prepareDatasetLastMetadataUpdate(doc, info));
d.setMetadataversionnumber(prepareDatasetMetadataVersionNumber(doc, info));
d.setGeolocation(prepareDatasetGeoLocations(doc, info));
return d;
case "software":
final Software s = new Software();
populateResultFields(s, doc, instances, collectedFrom, info, lastUpdateTimestamp);
s.setDocumentationUrl(prepareSoftwareDocumentationUrls(doc, info));
s.setLicense(prepareSoftwareLicenses(doc, info));
s.setCodeRepositoryUrl(prepareSoftwareCodeRepositoryUrl(doc, info));
s.setProgrammingLanguage(prepareSoftwareProgrammingLanguage(doc, info));
return s;
case "":
case "otherresearchproducts":
default:
final OtherResearchProduct o = new OtherResearchProduct();
populateResultFields(o, doc, instances, collectedFrom, info, lastUpdateTimestamp);
o.setContactperson(prepareOtherResearchProductContactPersons(doc, info));
o.setContactgroup(prepareOtherResearchProductContactGroups(doc, info));
o.setTool(prepareOtherResearchProductTools(doc, info));
return o;
case "publication":
final Publication p = new Publication();
populateResultFields(p, doc, instances, collectedFrom, info, lastUpdateTimestamp);
p.setJournal(prepareJournal(doc, info));
return p;
case "dataset":
final Dataset d = new Dataset();
populateResultFields(d, doc, instances, collectedFrom, info, lastUpdateTimestamp);
d.setStoragedate(prepareDatasetStorageDate(doc, info));
d.setDevice(prepareDatasetDevice(doc, info));
d.setSize(prepareDatasetSize(doc, info));
d.setVersion(prepareDatasetVersion(doc, info));
d.setLastmetadataupdate(prepareDatasetLastMetadataUpdate(doc, info));
d.setMetadataversionnumber(prepareDatasetMetadataVersionNumber(doc, info));
d.setGeolocation(prepareDatasetGeoLocations(doc, info));
return d;
case "software":
final Software s = new Software();
populateResultFields(s, doc, instances, collectedFrom, info, lastUpdateTimestamp);
s.setDocumentationUrl(prepareSoftwareDocumentationUrls(doc, info));
s.setLicense(prepareSoftwareLicenses(doc, info));
s.setCodeRepositoryUrl(prepareSoftwareCodeRepositoryUrl(doc, info));
s.setProgrammingLanguage(prepareSoftwareProgrammingLanguage(doc, info));
return s;
case "":
case "otherresearchproducts":
default:
final OtherResearchProduct o = new OtherResearchProduct();
populateResultFields(o, doc, instances, collectedFrom, info, lastUpdateTimestamp);
o.setContactperson(prepareOtherResearchProductContactPersons(doc, info));
o.setContactgroup(prepareOtherResearchProductContactGroups(doc, info));
o.setTool(prepareOtherResearchProductTools(doc, info));
return o;
}
}
private List<Oaf> addProjectRels(
final Document doc,
final OafEntity entity, DataInfo info) {
final Document doc,
final OafEntity entity,
final DataInfo info) {
final List<Oaf> res = new ArrayList<>();
@ -263,29 +307,25 @@ public abstract class AbstractMdRecordToOafMapper {
final String projectId = createOpenaireId(40, originalId, true);
res
.add(
OafMapperUtils
.getRelation(
docId, projectId, RESULT_PROJECT, OUTCOME, IS_PRODUCED_BY, entity.getCollectedfrom(),
info, entity.getLastupdatetimestamp(), validationdDate, null));
.add(OafMapperUtils
.getRelation(docId, projectId, RESULT_PROJECT, OUTCOME, IS_PRODUCED_BY, entity.getCollectedfrom(), info, entity
.getLastupdatetimestamp(), validationdDate, null));
res
.add(
OafMapperUtils
.getRelation(
projectId, docId, RESULT_PROJECT, OUTCOME, PRODUCES, entity.getCollectedfrom(), info,
entity.getLastupdatetimestamp(), validationdDate, null));
.add(OafMapperUtils
.getRelation(projectId, docId, RESULT_PROJECT, OUTCOME, PRODUCES, entity.getCollectedfrom(), info, entity
.getLastupdatetimestamp(), validationdDate, null));
}
}
return res;
}
private List<Oaf> addRelations(Document doc, OafEntity entity, DataInfo info) {
private List<Oaf> addRelations(final Document doc, final OafEntity entity, final DataInfo info) {
final List<Oaf> rels = Lists.newArrayList();
for (Object o : doc.selectNodes("//oaf:relation")) {
Element element = (Element) o;
for (final Object o : doc.selectNodes("//oaf:relation")) {
final Element element = (Element) o;
final String target = StringUtils.trim(element.getText());
final String relType = element.attributeValue("relType");
@ -293,11 +333,11 @@ public abstract class AbstractMdRecordToOafMapper {
final String relClass = element.attributeValue("relClass");
if (StringUtils.isNotBlank(target) && StringUtils.isNotBlank(relType) && StringUtils.isNotBlank(subRelType)
&& StringUtils.isNotBlank(relClass)) {
&& StringUtils.isNotBlank(relClass)) {
final String relClassInverse = ModelSupport
.findInverse(ModelSupport.rel(relType, subRelType, relClass))
.getInverseRelClass();
.findInverse(ModelSupport.rel(relType, subRelType, relClass))
.getInverseRelClass();
final String validationDate = ((Node) o).valueOf("@validationDate");
if (StringUtils.isNotBlank(target)) {
@ -305,19 +345,13 @@ public abstract class AbstractMdRecordToOafMapper {
if (StringUtils.isNotBlank(targetType)) {
final String targetId = createOpenaireId(targetType, target, true);
rels
.add(
OafMapperUtils
.getRelation(
entity.getId(), targetId, relType, subRelType, relClass,
entity.getCollectedfrom(), info,
entity.getLastupdatetimestamp(), validationDate, null));
.add(OafMapperUtils
.getRelation(entity.getId(), targetId, relType, subRelType, relClass, entity.getCollectedfrom(), info, entity
.getLastupdatetimestamp(), validationDate, null));
rels
.add(
OafMapperUtils
.getRelation(
targetId, entity.getId(), relType, subRelType, relClassInverse,
entity.getCollectedfrom(), info,
entity.getLastupdatetimestamp(), validationDate, null));
.add(OafMapperUtils
.getRelation(targetId, entity.getId(), relType, subRelType, relClassInverse, entity.getCollectedfrom(), info, entity
.getLastupdatetimestamp(), validationDate, null));
}
}
}
@ -325,24 +359,24 @@ public abstract class AbstractMdRecordToOafMapper {
return rels;
}
private List<Oaf> addAffiliations(Document doc, OafEntity entity, DataInfo info) {
private List<Oaf> addAffiliations(final Document doc, final OafEntity entity, final DataInfo info) {
final List<Oaf> rels = Lists.newArrayList();
for (Object o : doc.selectNodes("//datacite:affiliation[@affiliationIdentifierScheme='ROR']")) {
Element element = (Element) o;
for (final Object o : doc.selectNodes("//datacite:affiliation[@affiliationIdentifierScheme='ROR']")) {
final Element element = (Element) o;
String rorId = element.attributeValue("affiliationIdentifier");
final String rorId = element.attributeValue("affiliationIdentifier");
if (StringUtils.isNotBlank(rorId)) {
String fullRorId = Constants.ROR_NS_PREFIX + "::" + rorId;
final String fullRorId = Constants.ROR_NS_PREFIX + "::" + rorId;
String resultId = entity.getId();
String orgId = createOpenaireId("organization", fullRorId, true);
final String resultId = entity.getId();
final String orgId = createOpenaireId("organization", fullRorId, true);
List<KeyValue> properties = Lists.newArrayList();
final List<KeyValue> properties = Lists.newArrayList();
String apcAmount = doc.valueOf("//oaf:processingchargeamount");
String apcCurrency = doc.valueOf("//oaf:processingchargeamount/@currency");
final String apcAmount = doc.valueOf("//oaf:processingchargeamount");
final String apcCurrency = doc.valueOf("//oaf:processingchargeamount/@currency");
if (StringUtils.isNotBlank(apcAmount) && StringUtils.isNotBlank(apcCurrency)) {
properties.add(OafMapperUtils.keyValue("apc_amount", apcAmount));
@ -350,35 +384,30 @@ public abstract class AbstractMdRecordToOafMapper {
}
rels
.add(
OafMapperUtils
.getRelation(
resultId, orgId, RESULT_ORGANIZATION, AFFILIATION, HAS_AUTHOR_INSTITUTION,
entity.getCollectedfrom(), info, entity.getLastupdatetimestamp(), null,
properties));
.add(OafMapperUtils
.getRelation(resultId, orgId, RESULT_ORGANIZATION, AFFILIATION, HAS_AUTHOR_INSTITUTION, entity.getCollectedfrom(), info, entity
.getLastupdatetimestamp(), null, properties));
rels
.add(
OafMapperUtils
.getRelation(
orgId, resultId, RESULT_ORGANIZATION, AFFILIATION, IS_AUTHOR_INSTITUTION_OF,
entity.getCollectedfrom(), info, entity.getLastupdatetimestamp(), null,
properties));
.add(OafMapperUtils
.getRelation(orgId, resultId, RESULT_ORGANIZATION, AFFILIATION, IS_AUTHOR_INSTITUTION_OF, entity
.getCollectedfrom(), info, entity.getLastupdatetimestamp(), null, properties));
}
}
return rels;
}
protected abstract List<Oaf> addOtherResultRels(
final Document doc,
final OafEntity entity, DataInfo info);
final Document doc,
final OafEntity entity,
DataInfo info);
private void populateResultFields(
final Result r,
final Document doc,
final List<Instance> instances,
final KeyValue collectedFrom,
final DataInfo info,
final long lastUpdateTimestamp) {
final Result r,
final Document doc,
final List<Instance> instances,
final KeyValue collectedFrom,
final DataInfo info,
final long lastUpdateTimestamp) {
r.setDataInfo(info);
r.setLastupdatetimestamp(lastUpdateTimestamp);
r.setId(createOpenaireId(50, doc.valueOf("//dri:objIdentifier"), false));
@ -391,7 +420,7 @@ public abstract class AbstractMdRecordToOafMapper {
r.setOaiprovenance(prepareOAIprovenance(doc));
r.setAuthor(prepareAuthors(doc, info));
r.setLanguage(prepareLanguages(doc));
r.setCountry(new ArrayList<>()); // NOT PRESENT IN MDSTORES
r.setCountry(prepareCountries(doc, info));
r.setSubject(prepareSubjects(doc, info));
r.setTitle(prepareTitles(doc, info));
r.setRelevantdate(prepareRelevantDates(doc, info));
@ -407,16 +436,31 @@ public abstract class AbstractMdRecordToOafMapper {
r.setCoverage(prepareCoverages(doc, info));
r.setContext(prepareContexts(doc, info));
r.setExternalReference(new ArrayList<>()); // NOT PRESENT IN MDSTORES
r
.setProcessingchargeamount(field(doc.valueOf("//oaf:processingchargeamount"), info));
r
.setProcessingchargecurrency(field(doc.valueOf("//oaf:processingchargeamount/@currency"), info));
r.setProcessingchargeamount(field(doc.valueOf("//oaf:processingchargeamount"), info));
r.setProcessingchargecurrency(field(doc.valueOf("//oaf:processingchargeamount/@currency"), info));
r.setInstance(instances);
r.setBestaccessright(OafMapperUtils.createBestAccessRights(instances));
r.setEoscifguidelines(prepareEOSCIfGuidelines(doc, info));
}
private List<Country> prepareCountries(final Document doc, final DataInfo info) {
final List<Country> list = new ArrayList<>();
for (final Object n : doc.selectNodes("//oaf:country")) {
final String code = ((Node) n).getText().trim();
if (StringUtils.isNotBlank(code)) {
final Qualifier q = this.vocs.getTermAsQualifier(ModelConstants.DNET_COUNTRY_TYPE, code);
final Country country = new Country();
country.setClassid(q.getClassid());
country.setClassname(q.getClassname());
country.setSchemeid(q.getSchemeid());
country.setSchemename(q.getSchemename());
country.setDataInfo(info);
list.add(country);
}
}
return list;
}
protected abstract List<StructuredProperty> prepareResultPids(Document doc, DataInfo info);
private List<Context> prepareContexts(final Document doc, final DataInfo info) {
@ -433,7 +477,7 @@ public abstract class AbstractMdRecordToOafMapper {
return list;
}
private List<EoscIfGuidelines> prepareEOSCIfGuidelines(Document doc, DataInfo info) {
private List<EoscIfGuidelines> prepareEOSCIfGuidelines(final Document doc, final DataInfo info) {
final Set<EoscIfGuidelines> set = Sets.newHashSet();
for (final Object o : doc.selectNodes("//oaf:eoscifguidelines")) {
final String code = ((Node) o).valueOf("@code");
@ -455,10 +499,10 @@ public abstract class AbstractMdRecordToOafMapper {
protected abstract Qualifier prepareResourceType(Document doc, DataInfo info);
protected abstract List<Instance> prepareInstances(
Document doc,
DataInfo info,
KeyValue collectedfrom,
KeyValue hostedby);
Document doc,
DataInfo info,
KeyValue collectedfrom,
KeyValue hostedby);
protected abstract List<Field<String>> prepareSources(Document doc, DataInfo info);
@ -483,16 +527,16 @@ public abstract class AbstractMdRecordToOafMapper {
protected abstract List<Author> prepareAuthors(Document doc, DataInfo info);
protected abstract List<Field<String>> prepareOtherResearchProductTools(
Document doc,
DataInfo info);
Document doc,
DataInfo info);
protected abstract List<Field<String>> prepareOtherResearchProductContactGroups(
Document doc,
DataInfo info);
Document doc,
DataInfo info);
protected abstract List<Field<String>> prepareOtherResearchProductContactPersons(
Document doc,
DataInfo info);
Document doc,
DataInfo info);
protected abstract Qualifier prepareSoftwareProgrammingLanguage(Document doc, DataInfo info);
@ -501,8 +545,8 @@ public abstract class AbstractMdRecordToOafMapper {
protected abstract List<StructuredProperty> prepareSoftwareLicenses(Document doc, DataInfo info);
protected abstract List<Field<String>> prepareSoftwareDocumentationUrls(
Document doc,
DataInfo info);
Document doc,
DataInfo info);
protected abstract List<GeoLocation> prepareDatasetGeoLocations(Document doc, DataInfo info);
@ -520,15 +564,15 @@ public abstract class AbstractMdRecordToOafMapper {
protected abstract String findOriginalType(Document doc);
protected List<InstanceTypeMapping> prepareInstanceTypeMapping(Document doc) {
protected List<InstanceTypeMapping> prepareInstanceTypeMapping(final Document doc) {
return Optional
.ofNullable(findOriginalType(doc))
.map(originalType -> {
final List<InstanceTypeMapping> mappings = Lists.newArrayList();
mappings.add(OafMapperUtils.instanceTypeMapping(originalType, OPENAIRE_COAR_RESOURCE_TYPES_3_1));
return mappings;
})
.orElse(new ArrayList<>());
.ofNullable(findOriginalType(doc))
.map(originalType -> {
final List<InstanceTypeMapping> mappings = Lists.newArrayList();
mappings.add(OafMapperUtils.instanceTypeMapping(originalType, OPENAIRE_COAR_RESOURCE_TYPES_3_1));
return mappings;
})
.orElse(new ArrayList<>());
}
private Journal prepareJournal(final Document doc, final DataInfo info) {
@ -543,9 +587,7 @@ public abstract class AbstractMdRecordToOafMapper {
final String sp = n.valueOf("@sp");
final String vol = n.valueOf("@vol");
final String edition = n.valueOf("@edition");
if (StringUtils.isNotBlank(name)) {
return journal(name, issnPrinted, issnOnline, issnLinking, ep, iss, sp, vol, edition, null, null, info);
}
if (StringUtils.isNotBlank(name)) { return journal(name, issnPrinted, issnOnline, issnLinking, ep, iss, sp, vol, edition, null, null, info); }
}
return null;
}
@ -554,18 +596,13 @@ public abstract class AbstractMdRecordToOafMapper {
final Node n = doc.selectSingleNode("//*[local-name()='provenance']/*[local-name()='originDescription']");
if (n != null) {
final String id = n.valueOf("./*[local-name()='identifier']");
if (StringUtils.isNotBlank(id)) {
return Lists.newArrayList(id);
}
if (StringUtils.isNotBlank(id)) { return Lists.newArrayList(id); }
}
final List<String> idList = doc
.selectNodes(
"normalize-space(//*[local-name()='header']/*[local-name()='identifier' or local-name()='recordIdentifier']/text())");
.selectNodes("normalize-space(//*[local-name()='header']/*[local-name()='identifier' or local-name()='recordIdentifier']/text())");
final Set<String> originalIds = Sets.newHashSet(idList);
if (originalIds.isEmpty()) {
throw new IllegalStateException("missing originalID on " + doc.asXML());
}
if (originalIds.isEmpty()) { throw new IllegalStateException("missing originalID on " + doc.asXML()); }
return Lists.newArrayList(originalIds);
}
@ -587,32 +624,32 @@ public abstract class AbstractMdRecordToOafMapper {
}
protected Qualifier prepareQualifier(final String classId, final String schemeId) {
return vocs.getTermAsQualifier(schemeId, classId);
return this.vocs.getTermAsQualifier(schemeId, classId);
}
protected List<StructuredProperty> prepareListStructPropsWithValidQualifier(
final Node node,
final String xpath,
final String xpathClassId,
final String schemeId,
final DataInfo info) {
final Node node,
final String xpath,
final String xpathClassId,
final String schemeId,
final DataInfo info) {
final List<StructuredProperty> res = new ArrayList<>();
for (final Object o : node.selectNodes(xpath)) {
final Node n = (Node) o;
final String classId = n.valueOf(xpathClassId).trim();
if (vocs.termExists(schemeId, classId)) {
res.add(structuredProperty(n.getText(), vocs.getTermAsQualifier(schemeId, classId), info));
if (this.vocs.termExists(schemeId, classId)) {
res.add(structuredProperty(n.getText(), this.vocs.getTermAsQualifier(schemeId, classId), info));
}
}
return res;
}
protected List<StructuredProperty> prepareListStructProps(
final Node node,
final String xpath,
final Qualifier qualifier,
final DataInfo info) {
final Node node,
final String xpath,
final Qualifier qualifier,
final DataInfo info) {
final List<StructuredProperty> res = new ArrayList<>();
for (final Object o : node.selectNodes(xpath)) {
final Node n = (Node) o;
@ -622,33 +659,28 @@ public abstract class AbstractMdRecordToOafMapper {
}
protected List<StructuredProperty> prepareListStructProps(
final Node node,
final String xpath,
final DataInfo info) {
final Node node,
final String xpath,
final DataInfo info) {
final List<StructuredProperty> res = new ArrayList<>();
for (final Object o : node.selectNodes(xpath)) {
final Node n = (Node) o;
res
.add(
structuredProperty(
n.getText(), n.valueOf("@classid"), n.valueOf("@classname"), n.valueOf("@schemeid"),
n.valueOf("@schemename"), info));
.add(structuredProperty(n.getText(), n.valueOf("@classid"), n.valueOf("@classname"), n.valueOf("@schemeid"), n
.valueOf("@schemename"), info));
}
return res;
}
protected List<Subject> prepareSubjectList(
final Node node,
final String xpath,
final DataInfo info) {
final Node node,
final String xpath,
final DataInfo info) {
final List<Subject> res = new ArrayList<>();
for (final Object o : node.selectNodes(xpath)) {
final Node n = (Node) o;
res
.add(
subject(
n.getText(), n.valueOf("@classid"), n.valueOf("@classname"), n.valueOf("@schemeid"),
n.valueOf("@schemename"), info));
.add(subject(n.getText(), n.valueOf("@classid"), n.valueOf("@classname"), n.valueOf("@schemeid"), n.valueOf("@schemename"), info));
}
return res;
}
@ -656,14 +688,12 @@ public abstract class AbstractMdRecordToOafMapper {
protected OAIProvenance prepareOAIprovenance(final Document doc) {
final Node n = doc.selectSingleNode("//*[local-name()='provenance']/*[local-name()='originDescription']");
if (n == null) {
return null;
}
if (n == null) { return null; }
final String identifier = n.valueOf("./*[local-name()='identifier']");
final String baseURL = n.valueOf("./*[local-name()='baseURL']");
final String metadataNamespace = n.valueOf("./*[local-name()='metadataNamespace']");
final boolean altered = n.valueOf("@altered").equalsIgnoreCase("true");
final boolean altered = "true".equalsIgnoreCase(n.valueOf("@altered"));
final String datestamp = n.valueOf("./*[local-name()='datestamp']");
final String harvestDate = n.valueOf("@harvestDate");
@ -673,9 +703,7 @@ public abstract class AbstractMdRecordToOafMapper {
protected DataInfo prepareDataInfo(final Document doc, final boolean invisible) {
final Node n = doc.selectSingleNode("//oaf:datainfo");
if (n == null) {
return dataInfo(false, null, false, invisible, REPOSITORY_PROVENANCE_ACTIONS, "0.9");
}
if (n == null) { return dataInfo(false, null, false, invisible, REPOSITORY_PROVENANCE_ACTIONS, "0.9"); }
final String paClassId = n.valueOf("./oaf:provenanceaction/@classid");
final String paClassName = n.valueOf("./oaf:provenanceaction/@classname");
@ -687,14 +715,11 @@ public abstract class AbstractMdRecordToOafMapper {
final Boolean inferred = Boolean.parseBoolean(n.valueOf("./oaf:inferred"));
final String trust = n.valueOf("./oaf:trust");
return dataInfo(
deletedbyinference, inferenceprovenance, inferred, invisible,
qualifier(paClassId, paClassName, paSchemeId, paSchemeName), trust);
return dataInfo(deletedbyinference, inferenceprovenance, inferred, invisible, qualifier(paClassId, paClassName, paSchemeId, paSchemeName), trust);
}
protected List<Field<String>> prepareListURL(final Node node, final String xpath, final DataInfo info) {
return listFields(
info, prepareListString(node, xpath)
return listFields(info, prepareListString(node, xpath)
.stream()
.filter(URL_VALIDATOR::isValid)
.collect(Collectors.toList()));
@ -705,9 +730,9 @@ public abstract class AbstractMdRecordToOafMapper {
}
protected List<Field<String>> prepareListFields(
final Node node,
final String xpath,
final DataInfo info) {
final Node node,
final String xpath,
final DataInfo info) {
return listFields(info, prepareListString(node, xpath));
}
@ -722,15 +747,13 @@ public abstract class AbstractMdRecordToOafMapper {
return res;
}
protected Set<String> validateUrl(Collection<String> url) {
protected Set<String> validateUrl(final Collection<String> url) {
if (Objects.isNull(url)) {
return new HashSet<>();
}
if (Objects.isNull(url)) { return new HashSet<>(); }
return url
.stream()
.filter(URL_VALIDATOR::isValid)
.collect(Collectors.toCollection(HashSet::new));
.stream()
.filter(URL_VALIDATOR::isValid)
.collect(Collectors.toCollection(HashSet::new));
}
}

@ -12,6 +12,16 @@
<name>targetPath</name>
<description>the output path of the graph enriched</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>
</parameters>
<start to="EnrichGraph"/>
@ -31,8 +41,8 @@
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.executor.memoryOverhead=2g
--conf spark.sql.shuffle.partitions=3000
--conf spark.executor.memoryOverhead=${sparkExecutorMemory}
--conf spark.sql.shuffle.partitions=5000
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}

@ -1,40 +0,0 @@
package eu.dnetlib.dhp.enrich.orcid
import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf.{Author, Publication}
import eu.dnetlib.dhp.schema.sx.OafUtils
import org.apache.spark.sql.Row
import scala.collection.JavaConverters._
object AuthorEnricher extends Serializable {
def createAuthor(givenName: String, familyName: String, orcid: String): Author = {
val a = new Author
a.setName(givenName)
a.setSurname(familyName)
a.setFullname(s"$givenName $familyName")
val pid = OafUtils.createSP(orcid, ModelConstants.ORCID, ModelConstants.ORCID)
pid.setDataInfo(OafUtils.generateDataInfo())
pid.getDataInfo.setProvenanceaction(OafUtils.createQualifier("ORCID_ENRICHMENT", "ORCID_ENRICHMENT"))
a.setPid(List(pid).asJava)
a
}
def toOAFAuthor(r: Row): java.util.List[Author] = {
r.getList[Row](1)
.asScala
.map(s => createAuthor(s.getAs[String]("givenName"), s.getAs[String]("familyName"), s.getAs[String]("orcid")))
.toList
.asJava
}
// def enrichAuthor(p:Publication,r:Row): Unit = {
// val k:Map[String, OAuthor] =r.getList[Row](1).asScala.map(s => (s.getAs[String]("orcid"), OAuthor(s.getAs[String]("givenName") ,s.getAs[String]("familyName") ))).groupBy(_._1).mapValues(_.map(_._2).head)
// println(k)
//
//
//
// }
}

@ -0,0 +1,128 @@
package eu.dnetlib.dhp.enrich.orcid
import eu.dnetlib.dhp.schema.common.ModelConstants
import eu.dnetlib.dhp.schema.oaf.{Author, StructuredProperty}
import eu.dnetlib.dhp.schema.sx.OafUtils
import java.util
import scala.beans.BeanProperty
import scala.collection.JavaConverters._
import scala.util.control.Breaks.{break, breakable}
case class ORCIDAuthorEnricherResult(
@BeanProperty var id: String,
@BeanProperty var enriched_author: java.util.List[Author],
@BeanProperty var author_matched: java.util.List[MatchedAuthors],
@BeanProperty var author_unmatched: java.util.List[Author],
@BeanProperty var orcid_unmatched: java.util.List[OrcidAutor]
)
object ORCIDAuthorEnricher extends Serializable {
def enrichOrcid(
id: String,
graph_authors: java.util.List[Author],
orcid_authors: java.util.List[OrcidAutor]
): ORCIDAuthorEnricherResult = {
// Author enriching strategy:
// 1) create a copy of graph author list in unmatched_authors
// 2) find best match in unmatched_authors, remove it from unmatched_authors and enrich it so
// that the enrichment is reflected in graph_authors (they share author instances)
// 3) repeat (2) till the end of the list and then with different matching algorithms that have decreasing
// trust in their output
// At the end unmatched_authors will contain authors not matched with any of the matching algos
val unmatched_authors = new util.ArrayList[Author](graph_authors)
val matches = {
// Look after exact fullname match, reconstruct ORCID fullname as givenName + familyName
extractAndEnrichMatches(
unmatched_authors,
orcid_authors,
(author, orcid) =>
ORCIDAuthorMatchers.matchEqualsIgnoreCase(author.getFullname, orcid.givenName + " " + orcid.familyName),
"fullName"
) ++
// Look after exact reversed fullname match, reconstruct ORCID fullname as familyName + givenName
extractAndEnrichMatches(
unmatched_authors,
orcid_authors,
(author, orcid) =>
ORCIDAuthorMatchers.matchEqualsIgnoreCase(author.getFullname, orcid.familyName + " " + orcid.givenName),
"reversedFullName"
) ++
// split author names in tokens, order the tokens, then check for matches of full tokens or abbreviations
extractAndEnrichMatches(
unmatched_authors,
orcid_authors,
(author, orcid) =>
ORCIDAuthorMatchers
.matchOrderedTokenAndAbbreviations(author.getFullname, orcid.givenName + " " + orcid.familyName),
"orderedTokens"
) ++
// look after exact matches of ORCID creditName
extractAndEnrichMatches(
unmatched_authors,
orcid_authors,
(author, orcid) => ORCIDAuthorMatchers.matchEqualsIgnoreCase(author.getFullname, orcid.creditName),
"creditName"
) ++
// look after exact matches in ORCID otherNames
extractAndEnrichMatches(
unmatched_authors,
orcid_authors,
(author, orcid) =>
orcid.otherNames != null && ORCIDAuthorMatchers.matchOtherNames(author.getFullname, orcid.otherNames.asScala),
"otherNames"
)
}
ORCIDAuthorEnricherResult(id, graph_authors, matches.asJava, unmatched_authors, orcid_authors)
}
private def extractAndEnrichMatches(
graph_authors: java.util.List[Author],
orcid_authors: java.util.List[OrcidAutor],
matchingFunc: (Author, OrcidAutor) => Boolean,
matchName: String
) = {
val matched = scala.collection.mutable.ArrayBuffer.empty[MatchedAuthors]
if (graph_authors != null && !graph_authors.isEmpty) {
val ait = graph_authors.iterator
while (ait.hasNext) {
val author = ait.next()
val oit = orcid_authors.iterator
breakable {
while (oit.hasNext) {
val orcid = oit.next()
if (matchingFunc(author, orcid)) {
ait.remove()
oit.remove()
matched += MatchedAuthors(author, orcid, matchName)
if (author.getPid == null) {
author.setPid(new util.ArrayList[StructuredProperty]())
}
val orcidPID = OafUtils.createSP(orcid.orcid, ModelConstants.ORCID, ModelConstants.ORCID)
orcidPID.setDataInfo(OafUtils.generateDataInfo())
orcidPID.getDataInfo.setProvenanceaction(
OafUtils.createQualifier("ORCID_ENRICHMENT", "ORCID_ENRICHMENT")
)
author.getPid.add(orcidPID)
break()
}
}
}
}
}
matched
}
}

@ -0,0 +1,65 @@
package eu.dnetlib.dhp.enrich.orcid
import java.util.Locale
import java.util.regex.Pattern
object ORCIDAuthorMatchers {
val SPLIT_REGEX = Pattern.compile("[\\s,\\.]+")
val WORD_DIFF = 2
def matchEqualsIgnoreCase(a1: String, a2: String): Boolean = {
if (a1 == null || a2 == null)
false
else
a1 == a2 || a1.toLowerCase(Locale.ROOT).equals(a2.toLowerCase(Locale.ROOT))
}
def matchOtherNames(fullName: String, otherNames: Seq[String]): Boolean = {
if (otherNames != null) {
otherNames.exists(matchEqualsIgnoreCase(fullName, _))
} else {
false
}
}
def matchOrderedTokenAndAbbreviations(a1: String, a2: String): Boolean = {
val p1: Array[String] = SPLIT_REGEX.split(a1.trim.toLowerCase(Locale.ROOT)).filter(_.nonEmpty).sorted
val p2: Array[String] = SPLIT_REGEX.split(a2.trim.toLowerCase(Locale.ROOT)).filter(_.nonEmpty).sorted
if (p1.length < 2 || p2.length < 2) return false
if (Math.abs(p1.length - p2.length) > WORD_DIFF) return false // use alternative comparison algo
var p1Idx: Int = 0
var p2Idx: Int = 0
var shortMatches: Int = 0
var longMatches: Int = 0
while (p1Idx < p1.length && p2Idx < p2.length) {
val e1: String = p1(p1Idx)
val c1: Char = e1.charAt(0)
val e2: String = p2(p2Idx)
val c2: Char = e2.charAt(0)
if (c1 < c2) p1Idx += 1
else if (c1 > c2) p2Idx += 1
else {
var res: Boolean = false
if (e1.length != 1 && e2.length != 1) {
res = e1 == e2
longMatches += 1
} else {
res = true
shortMatches += 1
}
if (res) {
p1Idx += 1
p2Idx += 1
} else {
val diff: Int = e1.compareTo(e2)
if (diff < 0) p1Idx += 1
else if (diff > 0) p2Idx += 1
}
}
}
longMatches > 0 && (shortMatches + longMatches) == Math.min(p1.length, p2.length)
}
}

@ -1,14 +1,39 @@
package eu.dnetlib.dhp.enrich.orcid
import eu.dnetlib.dhp.application.AbstractScalaApplication
import eu.dnetlib.dhp.oa.merge.AuthorMerger
import eu.dnetlib.dhp.schema.common.ModelSupport
import eu.dnetlib.dhp.schema.oaf.{OtherResearchProduct, Publication, Result, Software}
import org.apache.spark.sql.functions._
import eu.dnetlib.dhp.schema.oaf._
import org.apache.spark.sql._
import org.apache.spark.sql.functions._
import org.slf4j.{Logger, LoggerFactory}
import scala.beans.BeanProperty
import scala.collection.JavaConverters._
case class OrcidAutor(
@BeanProperty var orcid: String,
@BeanProperty var familyName: String,
@BeanProperty var givenName: String,
@BeanProperty var creditName: String,
@BeanProperty var otherNames: java.util.List[String]
) {
def this() = this("null", "null", "null", "null", null)
}
case class MatchData(
@BeanProperty var id: String,
@BeanProperty var graph_authors: java.util.List[Author],
@BeanProperty var orcid_authors: java.util.List[OrcidAutor]
) {
def this() = this("null", null, null)
}
case class MatchedAuthors(
@BeanProperty var author: Author,
@BeanProperty var orcid: OrcidAutor,
@BeanProperty var `type`: String
)
class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String], log: Logger)
extends AbstractScalaApplication(propertyPath, args, log: Logger) {
@ -22,107 +47,132 @@ class SparkEnrichGraphWithOrcidAuthors(propertyPath: String, args: Array[String]
log.info(s"orcidPath is '$orcidPath'")
val targetPath = parser.get("targetPath")
log.info(s"targetPath is '$targetPath'")
val orcidPublication: Dataset[Row] = generateOrcidTable(spark, orcidPath)
// ModelSupport.entityTypes.entrySet().asScala.filter(k => k.getKey.getClass isInstance(Result))
enrichResult(
spark,
s"$graphPath/publication",
orcidPublication,
s"$targetPath/publication",
Encoders.bean(classOf[Publication])
)
enrichResult(
spark,
s"$graphPath/dataset",
orcidPublication,
s"$targetPath/dataset",
Encoders.bean(classOf[eu.dnetlib.dhp.schema.oaf.Dataset])
)
enrichResult(
spark,
s"$graphPath/software",
orcidPublication,
s"$targetPath/software",
Encoders.bean(classOf[Software])
)
enrichResult(
spark,
s"$graphPath/otherresearchproduct",
orcidPublication,
s"$targetPath/otherresearchproduct",
Encoders.bean(classOf[OtherResearchProduct])
)
createTemporaryData(graphPath, orcidPath, targetPath)
analisys(targetPath)
generateGraph(graphPath, targetPath)
}
private def enrichResult[T <: Result](
spark: SparkSession,
graphPath: String,
orcidPublication: Dataset[Row],
outputPath: String,
enc: Encoder[T]
): Unit = {
val entities = spark.read
.schema(enc.schema)
.json(graphPath)
.select(col("id"), col("datainfo"), col("instance"))
.where("datainfo.deletedbyinference != true")
.drop("datainfo")
.withColumn("instances", explode(col("instance")))
.withColumn("pids", explode(col("instances.pid")))
.select(
col("pids.qualifier.classid").alias("pid_schema"),
col("pids.value").alias("pid_value"),
col("id").alias("dnet_id")
)
private def generateGraph(graphPath: String, targetPath: String): Unit = {
ModelSupport.entityTypes.asScala
.filter(e => ModelSupport.isResult(e._1))
.foreach(e => {
val resultType = e._1.name()
val enc = Encoders.bean(e._2)
val matched = spark.read
.schema(Encoders.bean(classOf[ORCIDAuthorEnricherResult]).schema)
.parquet(s"${targetPath}/${resultType}_matched")
.selectExpr("id", "enriched_author")
spark.read
.schema(enc.schema)
.json(s"$graphPath/$resultType")
.join(matched, Seq("id"), "left")
.withColumn(
"author",
when(size(col("enriched_author")).gt(0), col("enriched_author"))
.otherwise(col("author"))
)
.drop("enriched_author")
.write
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(s"${targetPath}/${resultType}")
})
val orcidDnet = orcidPublication
.join(
entities,
lower(col("schema")).equalTo(lower(col("pid_schema"))) &&
lower(col("value")).equalTo(lower(col("pid_value"))),
"inner"
)
.groupBy(col("dnet_id"))
.agg(collect_set(orcidPublication("author")).alias("orcid_authors"))
.select("dnet_id", "orcid_authors")
.cache()
orcidDnet.count()
val result = spark.read.schema(enc.schema).json(graphPath).as[T](enc)
result
.joinWith(orcidDnet, result("id").equalTo(orcidDnet("dnet_id")), "left")
.map {
case (r: T, null) =>
r
case (p: T, r: Row) =>
p.setAuthor(AuthorMerger.enrichOrcid(p.getAuthor, AuthorEnricher.toOAFAuthor(r)))
p
}(enc)
.write
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath)
}
private def generateOrcidTable(spark: SparkSession, inputPath: String): Dataset[Row] = {
private def createTemporaryData(graphPath: String, orcidPath: String, targetPath: String): Unit = {
val orcidAuthors =
spark.read.load(s"$inputPath/Authors").select("orcid", "familyName", "givenName", "creditName", "otherNames")
spark.read.load(s"$orcidPath/Authors").select("orcid", "familyName", "givenName", "creditName", "otherNames")
val orcidWorks = spark.read
.load(s"$inputPath/Works")
.load(s"$orcidPath/Works")
.select(col("orcid"), explode(col("pids")).alias("identifier"))
.where(
"identifier.schema IN('doi','pmid','pmc','arxiv','handle')"
"identifier.schema IN('doi','pmid','pmc','arxiv','handle')" // scopus eid ?
)
val orcidPublication = orcidAuthors
.join(orcidWorks, orcidAuthors("orcid").equalTo(orcidWorks("orcid")))
val orcidWorksWithAuthors = orcidAuthors
.join(orcidWorks, Seq("orcid"))
.select(
col("identifier.schema").alias("schema"),
col("identifier.value").alias("value"),
struct(orcidAuthors("orcid").alias("orcid"), col("givenName"), col("familyName")).alias("author")
lower(col("identifier.schema")).alias("pid_schema"),
lower(col("identifier.value")).alias("pid_value"),
struct(
col("orcid"),
col("givenName"),
col("familyName"),
col("creditName"),
col("otherNames")
).alias("author")
)
orcidPublication.cache()
.cache()
ModelSupport.entityTypes.asScala
.filter(e => ModelSupport.isResult(e._1))
.foreach(e => {
val resultType = e._1.name()
val enc = Encoders.bean(e._2)
val oaEntities = spark.read
.schema(enc.schema)
.json(s"$graphPath/$resultType")
.select(col("id"), col("datainfo"), col("instance"))
.where("datainfo.deletedbyinference != true")
.drop("datainfo")
.withColumn("instances", explode(col("instance")))
.withColumn("pids", explode(col("instances.pid")))
.select(
lower(col("pids.qualifier.classid")).alias("pid_schema"),
lower(col("pids.value")).alias("pid_value"),
col("id")
)
val orcidDnet = orcidWorksWithAuthors
.join(
oaEntities,
Seq("pid_schema", "pid_value"),
"inner"
)
.groupBy(col("id"))
.agg(collect_set(col("author")).alias("orcid_authors"))
.select("id", "orcid_authors")
val result =
spark.read.schema(enc.schema).json(s"$graphPath/$resultType").selectExpr("id", "author as graph_authors")
result
.join(orcidDnet, Seq("id"))
.write
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.parquet(s"$targetPath/${resultType}_unmatched")
})
orcidWorksWithAuthors.unpersist()
}
private def analisys(targetPath: String): Unit = {
ModelSupport.entityTypes.asScala
.filter(e => ModelSupport.isResult(e._1))
.foreach(e => {
val resultType = e._1.name()
spark.read
.parquet(s"$targetPath/${resultType}_unmatched")
.where("size(graph_authors) > 0")
.as[MatchData](Encoders.bean(classOf[MatchData]))
.map(md => {
ORCIDAuthorEnricher.enrichOrcid(md.id, md.graph_authors, md.orcid_authors)
})(Encoders.bean(classOf[ORCIDAuthorEnricherResult]))
.write
.option("compression", "gzip")
.mode("overwrite")
.parquet(s"$targetPath/${resultType}_matched")
})
}
}

@ -1,10 +1,9 @@
package eu.dnetlib.oa.merge;
import static org.junit.jupiter.api.Assertions.*;
package eu.dnetlib.dhp.enrich.orcid;
import java.io.BufferedReader;
import java.io.InputStreamReader;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
@ -14,10 +13,9 @@ import org.junit.platform.commons.util.StringUtils;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.oa.merge.AuthorMerger;
import eu.dnetlib.dhp.schema.oaf.Author;
public class AuthorMergerTest {
public class ORCIDAuthorEnricherTest {
@Test
public void testEnrcichAuthor() throws Exception {
@ -26,12 +24,13 @@ public class AuthorMergerTest {
BufferedReader pr = new BufferedReader(new InputStreamReader(
Objects
.requireNonNull(
AuthorMergerTest.class
.getResourceAsStream("/eu/dnetlib/dhp/oa/merge/authors_publication_sample.json"))));
ORCIDAuthorEnricherTest.class
.getResourceAsStream("/eu/dnetlib/dhp/enrich/orcid/authors_publication_sample.json"))));
BufferedReader or = new BufferedReader(new InputStreamReader(
Objects
.requireNonNull(
AuthorMergerTest.class.getResourceAsStream("/eu/dnetlib/dhp/oa/merge/authors_orcid_sample.json"))));
ORCIDAuthorEnricherTest.class
.getResourceAsStream("/eu/dnetlib/dhp/enrich/orcid/authors_orcid_sample.json"))));
TypeReference<List<Author>> aclass = new TypeReference<List<Author>>() {
};
@ -67,7 +66,8 @@ public class AuthorMergerTest {
long start = System.currentTimeMillis();
// final List<Author> enrichedList = AuthorMerger.enrichOrcid(publicationAuthors, orcidAuthors);
final List<Author> enrichedList = AuthorMerger.enrichOrcid(publicationAuthors, orcidAuthors);
final List<Author> enrichedList = Collections.emptyList(); // SparkEnrichGraphWithOrcidAuthors.enrichOrcid(publicationAuthors,
// orcidAuthors);
long enrichedAuthorWithPid = enrichedList
.stream()
@ -91,24 +91,4 @@ public class AuthorMergerTest {
}
}
@Test
public void checkSimilarityTest() {
final Author left = new Author();
left.setName("Anand");
left.setSurname("Rachna");
left.setFullname("Anand, Rachna");
System.out.println(AuthorMerger.normalizeFullName(left.getFullname()));
final Author right = new Author();
right.setName("Rachna");
right.setSurname("Anand");
right.setFullname("Rachna, Anand");
// System.out.println(AuthorMerger.normalize(right.getFullname()));
boolean same = AuthorMerger.checkORCIDSimilarity(left, right);
assertTrue(same);
}
}

@ -196,6 +196,12 @@ class MappersTest {
assertEquals(aff1.getSource(), aff2.getTarget());
assertEquals(aff2.getSource(), aff1.getTarget());
// COUNTRIES
assertEquals(3, p.getCountry().size());
assertEquals("IT", p.getCountry().get(0).getClassid());
assertEquals("FR", p.getCountry().get(1).getClassid());
assertEquals("DE", p.getCountry().get(2).getClassid());
}
private void verifyRelation(Relation r) {
@ -867,6 +873,12 @@ class MappersTest {
assertValidId(p.getCollectedfrom().get(0).getKey());
System.out.println(p.getTitle().get(0).getValue());
assertTrue(StringUtils.isNotBlank(p.getTitle().get(0).getValue()));
// COUNTRIES
assertEquals(3, p.getCountry().size());
assertEquals("IT", p.getCountry().get(0).getClassid());
assertEquals("FR", p.getCountry().get(1).getClassid());
assertEquals("DE", p.getCountry().get(2).getClassid());
}
@Test

@ -71,6 +71,9 @@
subRelType="outcome"
targetType="project"
validationDate="2020-01-01">corda_______::226852</oaf:relation>
<oaf:country>IT</oaf:country>
<oaf:country>FR</oaf:country>
<oaf:country>DE</oaf:country>
</metadata>
<about xmlns:oai="http://www.openarchives.org/OAI/2.0/">
<provenance xmlns="http://www.openarchives.org/OAI/2.0/provenance" xsi:schemaLocation="http://www.openarchives.org/OAI/2.0/provenance http://www.openarchives.org/OAI/2.0/provenance.xsd">

@ -78,6 +78,9 @@
<oaf:projectid>corda_______::630786</oaf:projectid>
<oaf:hostedBy id="re3data_____::r3d100010386" name="LINDAT/CLARIN repository"/>
<oaf:collectedFrom id="re3data_____::r3d100010386" name="LINDAT/CLARIN repository"/>
<oaf:country>IT</oaf:country>
<oaf:country>FR</oaf:country>
<oaf:country>DE</oaf:country>
</metadata>
<about xmlns:dc="http://purl.org/dc/elements/1.1/"
xmlns:dri="http://www.driver-repository.eu/namespace/dri" xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance">

@ -0,0 +1,35 @@
package eu.dnetlib.dhp.enrich.orcid
import eu.dnetlib.dhp.enrich.orcid.ORCIDAuthorMatchers.matchOrderedTokenAndAbbreviations
import org.junit.jupiter.api.Assertions.{assertFalse, assertTrue}
import org.junit.jupiter.api.Test
class ORCIDAuthorMatchersTest {
@Test def testShortNames(): Unit = {
assertTrue(matchOrderedTokenAndAbbreviations("Lasagni Mariozzi Federico", "Lasagni F. Mariozzi"))
}
@Test def testInvertedNames(): Unit = {
assertTrue(matchOrderedTokenAndAbbreviations("Andrea, Paolo Marcello", "Marcello Paolo, Andrea"))
}
@Test def testHomonymy(): Unit = {
assertTrue(matchOrderedTokenAndAbbreviations("Jang Myung Lee", "J Lee"))
}
@Test def testAmbiguousShortNames(): Unit = {
assertFalse(matchOrderedTokenAndAbbreviations("P. Mariozzi", "M. Paolozzi"))
}
@Test def testNonMatches(): Unit = {
assertFalse(matchOrderedTokenAndAbbreviations("Giovanni Paolozzi", "Francesco Paolozzi"))
assertFalse(matchOrderedTokenAndAbbreviations("G. Paolozzi", "F. Paolozzi"))
}
@Test def testChineseNames(): Unit = {
assertTrue(matchOrderedTokenAndAbbreviations("孙林 Sun Lin", "Sun Lin"))
// assertTrue(AuthorsMatchRevised.compare("孙林 Sun Lin", "孙林")); // not yet implemented
}
}

@ -0,0 +1,52 @@
/*
* Copyright (c) 2024.
* SPDX-FileCopyrightText: © 2023 Consiglio Nazionale delle Ricerche
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
package eu.dnetlib.dhp.oa.provision;
import java.io.StringReader;
import java.io.StringWriter;
import javax.xml.transform.Transformer;
import javax.xml.transform.TransformerException;
import javax.xml.transform.stream.StreamResult;
import javax.xml.transform.stream.StreamSource;
import eu.dnetlib.dhp.utils.saxon.SaxonTransformerFactory;
public abstract class AbstractSolrRecordTransformJob {
protected static String toIndexRecord(Transformer tr, final String xmlRecord) {
final StreamResult res = new StreamResult(new StringWriter());
try {
tr.transform(new StreamSource(new StringReader(xmlRecord)), res);
return res.getWriter().toString();
} catch (TransformerException e) {
throw new IllegalArgumentException("XPathException on record: \n" + xmlRecord, e);
}
}
/**
* Creates the XSLT responsible for building the index xml records.
*
* @param format Metadata format name (DMF|TMF)
* @param xslt xslt for building the index record transformer
* @param fields the list of fields
* @return the javax.xml.transform.Transformer
* @throws TransformerException could happen
*/
protected static String getLayoutTransformer(String format, String fields, String xslt)
throws TransformerException {
final Transformer layoutTransformer = SaxonTransformerFactory.newInstance(xslt);
final StreamResult layoutToXsltXslt = new StreamResult(new StringWriter());
layoutTransformer.setParameter("format", format);
layoutTransformer.transform(new StreamSource(new StringReader(fields)), layoutToXsltXslt);
return layoutToXsltXslt.getWriter().toString();
}
}

@ -27,14 +27,7 @@ import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport;
import eu.dnetlib.dhp.oa.provision.model.RelatedEntity;
import eu.dnetlib.dhp.oa.provision.model.RelatedEntityWrapper;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.oaf.Datasource;
import eu.dnetlib.dhp.schema.oaf.Field;
import eu.dnetlib.dhp.schema.oaf.OafEntity;
import eu.dnetlib.dhp.schema.oaf.Organization;
import eu.dnetlib.dhp.schema.oaf.Project;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits;
import scala.Tuple2;
@ -156,10 +149,33 @@ public class CreateRelatedEntitiesJob_phase1 {
case software:
final Result result = (Result) entity;
if (result.getTitle() != null && !result.getTitle().isEmpty()) {
final StructuredProperty title = result.getTitle().stream().findFirst().get();
title.setValue(StringUtils.left(title.getValue(), ModelHardLimits.MAX_TITLE_LENGTH));
re.setTitle(title);
if (Objects.nonNull(result.getTitle()) && !result.getTitle().isEmpty()) {
result
.getTitle()
.stream()
.findFirst()
.map(StructuredProperty::getValue)
.ifPresent(
title -> re.getTitle().setValue(StringUtils.left(title, ModelHardLimits.MAX_TITLE_LENGTH)));
}
if (Objects.nonNull(result.getDescription()) && !result.getDescription().isEmpty()) {
result
.getDescription()
.stream()
.findFirst()
.map(Field::getValue)
.ifPresent(
d -> re.setDescription(StringUtils.left(d, ModelHardLimits.MAX_RELATED_ABSTRACT_LENGTH)));
}
if (Objects.nonNull(result.getAuthor()) && !result.getAuthor().isEmpty()) {
re
.setAuthor(
result
.getAuthor()
.stream()
.map(Author::getFullname)
.filter(StringUtils::isNotBlank)
.collect(Collectors.toList()));
}
re.setDateofacceptance(getValue(result.getDateofacceptance()));

@ -0,0 +1,144 @@
package eu.dnetlib.dhp.oa.provision;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.Optional;
import javax.xml.transform.TransformerException;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.solr.common.SolrInputDocument;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Encoder;
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.oa.provision.model.SerializableSolrInputDocument;
import eu.dnetlib.dhp.oa.provision.model.TupleWrapper;
import eu.dnetlib.dhp.oa.provision.utils.ISLookupClient;
import eu.dnetlib.dhp.oa.provision.utils.StreamingInputDocumentFactory;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.dhp.utils.saxon.SaxonTransformerFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
public class SolrRecordDumpJob extends AbstractSolrRecordTransformJob {
private static final Logger log = LoggerFactory.getLogger(SolrRecordDumpJob.class);
private static final Integer DEFAULT_BATCH_SIZE = 1000;
private final String inputPath;
private final String format;
private final String outputPath;
private final SparkSession spark;
public static void main(String[] args) throws Exception {
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
SolrRecordDumpJob.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/provision/input_params_solr_record_dump.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);
final String format = parser.get("format");
log.info("format: {}", format);
final String outputPath = Optional
.ofNullable(parser.get("outputPath"))
.map(StringUtils::trim)
.orElse(null);
log.info("outputPath: {}", outputPath);
final Integer batchSize = Optional
.ofNullable(parser.get("batchSize"))
.map(Integer::valueOf)
.orElse(DEFAULT_BATCH_SIZE);
log.info("batchSize: {}", batchSize);
final boolean shouldIndex = Optional
.ofNullable(parser.get("shouldIndex"))
.map(Boolean::valueOf)
.orElse(false);
log.info("shouldIndex: {}", shouldIndex);
final SparkConf conf = new SparkConf();
conf.registerKryoClasses(new Class[] {
SerializableSolrInputDocument.class
});
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
final String isLookupUrl = parser.get("isLookupUrl");
log.info("isLookupUrl: {}", isLookupUrl);
final ISLookupClient isLookup = new ISLookupClient(ISLookupClientFactory.getLookUpService(isLookupUrl));
new SolrRecordDumpJob(spark, inputPath, format, outputPath).run(isLookup);
});
}
public SolrRecordDumpJob(SparkSession spark, String inputPath, String format, String outputPath) {
this.spark = spark;
this.inputPath = inputPath;
this.format = format;
this.outputPath = outputPath;
}
public void run(ISLookupClient isLookup) throws ISLookUpException, TransformerException {
final String fields = isLookup.getLayoutSource(format);
log.info("fields: {}", fields);
final String xslt = isLookup.getLayoutTransformer();
final String dsId = isLookup.getDsId(format);
log.info("dsId: {}", dsId);
final String indexRecordXslt = getLayoutTransformer(format, fields, xslt);
log.info("indexRecordTransformer {}", indexRecordXslt);
final Encoder<TupleWrapper> encoder = Encoders.bean(TupleWrapper.class);
spark
.read()
.schema(encoder.schema())
.json(inputPath)
.as(encoder)
.map(
(MapFunction<TupleWrapper, TupleWrapper>) t -> new TupleWrapper(
toIndexRecord(SaxonTransformerFactory.newInstance(indexRecordXslt), t.getXml()),
t.getJson()),
Encoders.bean(TupleWrapper.class))
.map(
(MapFunction<TupleWrapper, SerializableSolrInputDocument>) t -> {
SolrInputDocument s = new StreamingInputDocumentFactory()
.parseDocument(t.getXml(), t.getJson());
return new SerializableSolrInputDocument(s);
},
Encoders.kryo(SerializableSolrInputDocument.class))
.write()
.mode(SaveMode.Overwrite)
.parquet(outputPath);
}
}

@ -3,6 +3,7 @@ package eu.dnetlib.dhp.oa.provision;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import static eu.dnetlib.dhp.utils.DHPUtils.toSeq;
import static org.apache.spark.sql.functions.*;
import java.util.List;
import java.util.Map;
@ -14,22 +15,31 @@ import org.apache.hadoop.io.compress.GzipCodec;
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
import org.apache.spark.SparkConf;
import org.apache.spark.SparkContext;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.PairFunction;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.*;
import org.apache.spark.sql.expressions.UserDefinedFunction;
import org.apache.spark.sql.types.DataTypes;
import org.apache.spark.util.LongAccumulator;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Maps;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
import eu.dnetlib.dhp.oa.provision.model.JoinedEntity;
import eu.dnetlib.dhp.oa.provision.model.ProvisionModelSupport;
import eu.dnetlib.dhp.oa.provision.model.TupleWrapper;
import eu.dnetlib.dhp.oa.provision.utils.ContextMapper;
import eu.dnetlib.dhp.oa.provision.utils.XmlRecordFactory;
import eu.dnetlib.dhp.schema.solr.SolrRecord;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import scala.Tuple2;
/**
@ -65,13 +75,20 @@ public class XmlConverterJob {
final String contextApiBaseUrl = parser.get("contextApiBaseUrl");
log.info("contextApiBaseUrl: {}", contextApiBaseUrl);
String isLookupUrl = parser.get("isLookupUrl");
log.info("isLookupUrl: {}", isLookupUrl);
ISLookUpService isLookup = ISLookupClientFactory.getLookUpService(isLookupUrl);
final SparkConf conf = new SparkConf();
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
conf.registerKryoClasses(ProvisionModelSupport.getModelClasses());
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
removeOutputDir(spark, outputPath);
convertToXml(spark, inputPath, outputPath, ContextMapper.fromAPI(contextApiBaseUrl));
convertToXml(
spark, inputPath, outputPath, ContextMapper.fromAPI(contextApiBaseUrl),
VocabularyGroup.loadVocsFromIS(isLookup));
});
}
@ -79,7 +96,8 @@ public class XmlConverterJob {
final SparkSession spark,
final String inputPath,
final String outputPath,
final ContextMapper contextMapper) {
final ContextMapper contextMapper,
final VocabularyGroup vocabularies) {
final XmlRecordFactory recordFactory = new XmlRecordFactory(
prepareAccumulators(spark.sparkContext()),
@ -92,20 +110,25 @@ public class XmlConverterJob {
log.info("Found paths: {}", String.join(",", paths));
final ObjectMapper mapper = new ObjectMapper();
mapper.setSerializationInclusion(JsonInclude.Include.NON_EMPTY);
spark
.read()
.load(toSeq(paths))
.as(Encoders.kryo(JoinedEntity.class))
.map(
(MapFunction<JoinedEntity, Tuple2<String, String>>) je -> new Tuple2<>(
je.getEntity().getId(),
recordFactory.build(je)),
Encoders.tuple(Encoders.STRING(), Encoders.STRING()))
.javaRDD()
.mapToPair(
(PairFunction<Tuple2<String, String>, Text, Text>) t -> new Tuple2<>(new Text(t._1()),
new Text(t._2())))
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, GzipCodec.class);
(MapFunction<JoinedEntity, Tuple2<String, SolrRecord>>) je -> new Tuple2<>(
recordFactory.build(je),
ProvisionModelSupport.transform(je, contextMapper, vocabularies)),
Encoders.tuple(Encoders.STRING(), Encoders.bean(SolrRecord.class)))
.map(
(MapFunction<Tuple2<String, SolrRecord>, TupleWrapper>) t -> new TupleWrapper(
t._1(), mapper.writeValueAsString(t._2())),
Encoders.bean(TupleWrapper.class))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath);
}
private static void removeOutputDir(final SparkSession spark, final String path) {

@ -3,26 +3,17 @@ package eu.dnetlib.dhp.oa.provision;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.StringReader;
import java.io.StringWriter;
import java.text.SimpleDateFormat;
import java.util.Date;
import java.util.Optional;
import javax.xml.transform.Transformer;
import javax.xml.transform.TransformerException;
import javax.xml.transform.stream.StreamResult;
import javax.xml.transform.stream.StreamSource;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.io.Text;
import org.apache.solr.common.SolrInputDocument;
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.Encoder;
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;
@ -31,34 +22,25 @@ import com.lucidworks.spark.util.SolrSupport;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.oa.provision.model.SerializableSolrInputDocument;
import eu.dnetlib.dhp.oa.provision.model.TupleWrapper;
import eu.dnetlib.dhp.oa.provision.utils.ISLookupClient;
import eu.dnetlib.dhp.oa.provision.utils.StreamingInputDocumentFactory;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.dhp.utils.saxon.SaxonTransformerFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
public class XmlIndexingJob {
public class XmlIndexingJob extends AbstractSolrRecordTransformJob {
private static final Logger log = LoggerFactory.getLogger(XmlIndexingJob.class);
public enum OutputFormat {
SOLR, HDFS
}
private static final Integer DEFAULT_BATCH_SIZE = 1000;
protected static final String DATE_FORMAT = "yyyy-MM-dd'T'hh:mm:ss'Z'";
private final String inputPath;
private final String format;
private final int batchSize;
private final OutputFormat outputFormat;
private final String outputPath;
private final SparkSession spark;
public static void main(String[] args) throws Exception {
@ -83,25 +65,14 @@ public class XmlIndexingJob {
final String format = parser.get("format");
log.info("format: {}", format);
final String outputPath = Optional
.ofNullable(parser.get("outputPath"))
.map(StringUtils::trim)
.orElse(null);
log.info("outputPath: {}", outputPath);
final Integer batchSize = Optional
.ofNullable(parser.get("batchSize"))
.map(Integer::valueOf)
.orElse(DEFAULT_BATCH_SIZE);
log.info("batchSize: {}", batchSize);
final OutputFormat outputFormat = Optional
.ofNullable(parser.get("outputFormat"))
.map(OutputFormat::valueOf)
.orElse(OutputFormat.SOLR);
log.info("outputFormat: {}", outputFormat);
final SparkConf conf = new SparkConf();
conf.registerKryoClasses(new Class[] {
SerializableSolrInputDocument.class
});
@ -113,19 +84,16 @@ public class XmlIndexingJob {
final String isLookupUrl = parser.get("isLookupUrl");
log.info("isLookupUrl: {}", isLookupUrl);
final ISLookupClient isLookup = new ISLookupClient(ISLookupClientFactory.getLookUpService(isLookupUrl));
new XmlIndexingJob(spark, inputPath, format, batchSize, outputFormat, outputPath).run(isLookup);
new XmlIndexingJob(spark, inputPath, format, batchSize)
.run(isLookup);
});
}
public XmlIndexingJob(SparkSession spark, String inputPath, String format, Integer batchSize,
OutputFormat outputFormat,
String outputPath) {
public XmlIndexingJob(SparkSession spark, String inputPath, String format, Integer batchSize) {
this.spark = spark;
this.inputPath = inputPath;
this.format = format;
this.batchSize = batchSize;
this.outputFormat = outputFormat;
this.outputPath = outputPath;
}
public void run(ISLookupClient isLookup) throws ISLookUpException, TransformerException {
@ -137,84 +105,31 @@ public class XmlIndexingJob {
final String dsId = isLookup.getDsId(format);
log.info("dsId: {}", dsId);
final String collection = ProvisionConstants.getCollectionName(format);
log.info("collection: {}", collection);
final String zkHost = isLookup.getZkHost();
log.info("zkHost: {}", zkHost);
final String version = getRecordDatestamp();
final String indexRecordXslt = getLayoutTransformer(format, fields, xslt);
log.info("indexRecordTransformer {}", indexRecordXslt);
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<SolrInputDocument> docs = sc
.sequenceFile(inputPath, Text.class, Text.class)
.map(t -> t._2().toString())
.map(s -> toIndexRecord(SaxonTransformerFactory.newInstance(indexRecordXslt), s))
.map(s -> new StreamingInputDocumentFactory().parseDocument(s));
switch (outputFormat) {
case SOLR:
final String collection = ProvisionConstants.getCollectionName(format);
// SparkSolr >= 4
// com.lucidworks.spark.BatchSizeType bt = com.lucidworks.spark.BatchSizeType.NUM_DOCS;
// SolrSupport.indexDocs(zkHost, collection, batchSize, bt, docs.rdd());
// SparkSolr < 4
SolrSupport.indexDocs(zkHost, collection, batchSize, docs.rdd());
break;
case HDFS:
spark
.createDataset(
docs.map(SerializableSolrInputDocument::new).rdd(),
Encoders.kryo(SerializableSolrInputDocument.class))
.write()
.mode(SaveMode.Overwrite)
.parquet(outputPath);
break;
default:
throw new IllegalArgumentException("invalid outputFormat: " + outputFormat);
}
}
protected static String toIndexRecord(Transformer tr, final String xmlRecord) {
final StreamResult res = new StreamResult(new StringWriter());
try {
tr.transform(new StreamSource(new StringReader(xmlRecord)), res);
return res.getWriter().toString();
} catch (TransformerException e) {
throw new IllegalArgumentException("XPathException on record: \n" + xmlRecord, e);
}
}
/**
* Creates the XSLT responsible for building the index xml records.
*
* @param format Metadata format name (DMF|TMF)
* @param xslt xslt for building the index record transformer
* @param fields the list of fields
* @return the javax.xml.transform.Transformer
* @throws TransformerException could happen
*/
protected static String getLayoutTransformer(String format, String fields, String xslt)
throws TransformerException {
final Transformer layoutTransformer = SaxonTransformerFactory.newInstance(xslt);
final StreamResult layoutToXsltXslt = new StreamResult(new StringWriter());
layoutTransformer.setParameter("format", format);
layoutTransformer.transform(new StreamSource(new StringReader(fields)), layoutToXsltXslt);
return layoutToXsltXslt.getWriter().toString();
}
/**
* method return a solr-compatible string representation of a date, used to mark all records as indexed today
*
* @return the parsed date
*/
public static String getRecordDatestamp() {
return new SimpleDateFormat(DATE_FORMAT).format(new Date());
final Encoder<TupleWrapper> encoder = Encoders.bean(TupleWrapper.class);
JavaRDD<SolrInputDocument> docs = spark
.read()
.schema(encoder.schema())
.json(inputPath)
.as(encoder)
.map(
(MapFunction<TupleWrapper, TupleWrapper>) t -> new TupleWrapper(
toIndexRecord(SaxonTransformerFactory.newInstance(indexRecordXslt), t.getXml()),
t.getJson()),
Encoders.bean(TupleWrapper.class))
.javaRDD()
.map(
t -> new StreamingInputDocumentFactory().parseDocument(t.getXml(), t.getJson()));
SolrSupport.indexDocs(zkHost, collection, batchSize, docs.rdd());
}
}

@ -5,11 +5,15 @@ import java.io.Serializable;
import java.util.LinkedList;
import java.util.List;
import eu.dnetlib.dhp.schema.oaf.OafEntity;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
public class JoinedEntity<E extends OafEntity> implements Serializable {
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.oaf.*;
private E entity;
public class JoinedEntity implements Serializable {
private OafEntity entity;
private List<RelatedEntityWrapper> links;
@ -17,16 +21,16 @@ public class JoinedEntity<E extends OafEntity> implements Serializable {
links = new LinkedList<>();
}
public JoinedEntity(E entity) {
public JoinedEntity(OafEntity entity) {
this();
this.entity = entity;
}
public E getEntity() {
public OafEntity getEntity() {
return entity;
}
public void setEntity(E entity) {
public void setEntity(OafEntity entity) {
this.entity = entity;
}

@ -1,13 +1,46 @@
package eu.dnetlib.dhp.oa.provision.model;
import java.util.List;
import static org.apache.commons.lang3.StringUtils.substringBefore;
import java.io.StringReader;
import java.util.*;
import java.util.stream.Collectors;
import org.apache.commons.lang3.StringUtils;
import org.dom4j.Document;
import org.dom4j.DocumentException;
import org.dom4j.io.SAXReader;
import org.jetbrains.annotations.Nullable;
import com.google.common.base.Splitter;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
import eu.dnetlib.dhp.common.vocabulary.VocabularyTerm;
import eu.dnetlib.dhp.oa.provision.RelationList;
import eu.dnetlib.dhp.oa.provision.SortableRelation;
import eu.dnetlib.dhp.oa.provision.utils.ContextDef;
import eu.dnetlib.dhp.oa.provision.utils.ContextMapper;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.dhp.schema.solr.*;
import eu.dnetlib.dhp.schema.solr.AccessRight;
import eu.dnetlib.dhp.schema.solr.Author;
import eu.dnetlib.dhp.schema.solr.Context;
import eu.dnetlib.dhp.schema.solr.Country;
import eu.dnetlib.dhp.schema.solr.Datasource;
import eu.dnetlib.dhp.schema.solr.EoscIfGuidelines;
import eu.dnetlib.dhp.schema.solr.Instance;
import eu.dnetlib.dhp.schema.solr.Journal;
import eu.dnetlib.dhp.schema.solr.OpenAccessColor;
import eu.dnetlib.dhp.schema.solr.OpenAccessRoute;
import eu.dnetlib.dhp.schema.solr.Organization;
import eu.dnetlib.dhp.schema.solr.Project;
import eu.dnetlib.dhp.schema.solr.Result;
import eu.dnetlib.dhp.schema.solr.Subject;
public class ProvisionModelSupport {
@ -28,4 +61,636 @@ public class ProvisionModelSupport {
RelationList.class));
return modelClasses.toArray(new Class[] {});
}
public static SolrRecord transform(JoinedEntity je, ContextMapper contextMapper, VocabularyGroup vocs) {
SolrRecord r = new SolrRecord();
final OafEntity e = je.getEntity();
final RecordType type = RecordType.valueOf(e.getClass().getSimpleName().toLowerCase());
final Boolean deletedbyinference = Optional
.ofNullable(e.getDataInfo())
.map(DataInfo::getDeletedbyinference)
.orElse(null);
r
.setHeader(
SolrRecordHeader
.newInstance(
e.getId(), e.getOriginalId(), type, deletedbyinference));
r.setCollectedfrom(asProvenance(e.getCollectedfrom()));
r.setContext(asContext(e.getContext(), contextMapper));
r.setPid(asPid(e.getPid()));
if (e instanceof eu.dnetlib.dhp.schema.oaf.Result) {
r.setResult(mapResult((eu.dnetlib.dhp.schema.oaf.Result) e));
} else if (e instanceof eu.dnetlib.dhp.schema.oaf.Datasource) {
r.setDatasource(mapDatasource((eu.dnetlib.dhp.schema.oaf.Datasource) e));
} else if (e instanceof eu.dnetlib.dhp.schema.oaf.Organization) {
r.setOrganization(mapOrganization((eu.dnetlib.dhp.schema.oaf.Organization) e));
} else if (e instanceof eu.dnetlib.dhp.schema.oaf.Project) {
r.setProject(mapProject((eu.dnetlib.dhp.schema.oaf.Project) e, vocs));
}
r
.setLinks(
Optional
.ofNullable(je.getLinks())
.map(
links -> links
.stream()
.map(rew -> mapRelatedRecord(rew, vocs))
.collect(Collectors.toList()))
.orElse(null));
return r;
}
private static RelatedRecord mapRelatedRecord(RelatedEntityWrapper rew, VocabularyGroup vocs) {
RelatedRecord rr = new RelatedRecord();
final RelatedEntity re = rew.getTarget();
final RecordType relatedRecordType = RecordType.valueOf(re.getType());
final Relation relation = rew.getRelation();
rr
.setHeader(
RelatedRecordHeader
.newInstance(
relation.getRelType(),
relation.getRelClass(),
relation.getTarget(), relatedRecordType));
rr.setAcronym(re.getAcronym());
rr.setCode(re.getCode());
rr.setContracttype(mapCodeLabel(re.getContracttype()));
rr.setCollectedfrom(asProvenance(re.getCollectedfrom()));
rr.setCodeRepositoryUrl(re.getCodeRepositoryUrl());
rr.setCountry(asCountry(re.getCountry()));
rr.setDatasourcetype(mapCodeLabel(re.getDatasourcetype()));
rr.setDatasourcetypeui(mapCodeLabel(re.getDatasourcetypeui()));
rr.setDateofacceptance(re.getDateofacceptance());
rr.setFunding(mapFunding(re.getFundingtree(), vocs));
rr.setInstances(mapInstances(re.getInstances()));
rr.setLegalname(re.getLegalname());
rr.setLegalshortname(re.getLegalshortname());
rr.setOfficialname(re.getOfficialname());
rr.setOpenairecompatibility(mapCodeLabel(re.getOpenairecompatibility()));
rr.setPid(asPid(re.getPid()));
rr.setProjectTitle(rr.getProjectTitle());
rr.setPublisher(re.getPublisher());
rr.setResulttype(mapQualifier(re.getResulttype()));
rr.setTitle(Optional.ofNullable(re.getTitle()).map(StructuredProperty::getValue).orElse(null));
return rr;
}
private static Project mapProject(eu.dnetlib.dhp.schema.oaf.Project p, VocabularyGroup vocs) {
Project ps = new Project();
ps.setAcronym(mapField(p.getAcronym()));
ps.setCode(mapField(p.getCode()));
ps.setContracttype(mapCodeLabel(p.getContracttype()));
ps.setCurrency(mapField(p.getCurrency()));
ps.setDuration(mapField(p.getDuration()));
ps.setCallidentifier(mapField(p.getCallidentifier()));
ps.setEcarticle29_3(mapField(p.getEcarticle29_3()));
ps.setEnddate(mapField(p.getEnddate()));
ps.setFundedamount(p.getFundedamount());
ps.setKeywords(mapField(p.getKeywords()));
ps.setStartdate(mapField(p.getStartdate()));
ps.setSubjects(asSubjectSP(p.getSubjects()));
ps.setSummary(mapField(p.getSummary()));
ps.setTitle(mapField(p.getTitle()));
ps.setTotalcost(p.getTotalcost());
ps.setWebsiteurl(mapField(p.getWebsiteurl()));
ps.setFunding(mapFundingField(p.getFundingtree(), vocs));
return ps;
}
private static Funding mapFunding(List<String> fundingtree, VocabularyGroup vocs) {
SAXReader reader = new SAXReader();
return Optional
.ofNullable(fundingtree)
.flatMap(
ftree -> ftree
.stream()
.map(ft -> {
try {
Document doc = reader.read(new StringReader(ft));
String countryCode = doc.valueOf("/fundingtree/funder/jurisdiction/text()");
Country country = vocs
.find("dnet:countries")
.map(voc -> voc.getTerm(countryCode))
.map(VocabularyTerm::getName)
.map(label -> Country.newInstance(countryCode, label))
.orElse(null);
String level0_id = doc.valueOf("//funding_level_0/id/text()");
String level1_id = doc.valueOf("//funding_level_1/id/text()");
String level2_id = doc.valueOf("//funding_level_2/id/text()");
return Funding
.newInstance(
Funder
.newInstance(
doc.valueOf("/fundingtree/funder/id/text()"),
doc.valueOf("/fundingtree/funder/shortname/text()"),
doc.valueOf("/fundingtree/funder/name/text()"),
country, new ArrayList<>()),
Optional
.ofNullable(level0_id)
.map(
id -> FundingLevel
.newInstance(
id,
doc.valueOf("//funding_level_0/description/text()"),
doc.valueOf("//funding_level_0/name/text()")))
.orElse(null),
Optional
.ofNullable(level1_id)
.map(
id -> FundingLevel
.newInstance(
id,
doc.valueOf("//funding_level_1/description/text()"),
doc.valueOf("//funding_level_1/name/text()")))
.orElse(null),
Optional
.ofNullable(level2_id)
.map(
id -> FundingLevel
.newInstance(
id,
doc.valueOf("//funding_level_2/description/text()"),
doc.valueOf("//funding_level_2/name/text()")))
.orElse(null));
} catch (DocumentException e) {
throw new IllegalArgumentException(e);
}
})
.findFirst())
.orElse(null);
}
private static Funding mapFundingField(List<Field<String>> fundingtree, VocabularyGroup vocs) {
return mapFunding(
Optional
.ofNullable(fundingtree)
.map(fts -> fts.stream().map(Field::getValue).collect(Collectors.toList()))
.orElse(null),
vocs);
}
private static Organization mapOrganization(eu.dnetlib.dhp.schema.oaf.Organization o) {
Organization org = new Organization();
org.setCountry(mapCodeLabel(o.getCountry()));
org.setLegalname(mapField(o.getLegalname()));
org.setLegalshortname(mapField(o.getLegalshortname()));
org.setAlternativeNames(mapFieldList(o.getAlternativeNames()));
org.setWebsiteurl(mapField(o.getWebsiteurl()));
org.setLogourl(mapField(o.getLogourl()));
org.setEcenterprise(mapField(o.getEcenterprise()));
org.setEchighereducation(mapField(o.getEchighereducation()));
org.setEclegalbody(mapField(o.getEclegalbody()));
org.setEcinternationalorganization(mapField(o.getEcinternationalorganization()));
org.setEcinternationalorganizationeurinterests(mapField(o.getEcinternationalorganizationeurinterests()));
org.setEclegalperson(mapField(o.getEclegalperson()));
org.setEcnonprofit(mapField(o.getEcnonprofit()));
org.setEcnutscode(mapField(o.getEcnutscode()));
org.setEcresearchorganization(mapField(o.getEcresearchorganization()));
org.setEcsmevalidated(mapField(o.getEcsmevalidated()));
return org;
}
private static Datasource mapDatasource(eu.dnetlib.dhp.schema.oaf.Datasource d) {
Datasource ds = new Datasource();
ds.setEnglishname(mapField(d.getEnglishname()));
ds.setOfficialname(mapField(d.getOfficialname()));
ds.setDescription(mapField(d.getDescription()));
ds.setJournal(mapJournal(d.getJournal()));
ds.setLogourl(mapField(d.getLogourl()));
ds.setAccessinfopackage(mapFieldList(d.getAccessinfopackage()));
ds.setCertificates(mapField(d.getCertificates()));
ds.setCitationguidelineurl(mapField(d.getCitationguidelineurl()));
ds.setConsenttermsofuse(d.getConsenttermsofuse());
ds.setConsenttermsofusedate(d.getConsenttermsofusedate());
ds.setContactemail(mapField(d.getContactemail()));
ds.setContentpolicies(mapCodeLabel(d.getContentpolicies()));
ds.setDatabaseaccessrestriction(mapField(d.getDatabaseaccessrestriction()));
ds.setDatabaseaccesstype(mapField(d.getDatabaseaccesstype()));
ds.setDataprovider(mapField(d.getDataprovider()));
ds.setDatasourcetype(mapCodeLabel(d.getDatasourcetype()));
ds.setDatasourcetypeui(mapCodeLabel(d.getDatasourcetypeui()));
ds.setDatauploadrestriction(mapField(d.getDatauploadrestriction()));
ds.setDatauploadtype(mapField(d.getDatauploadtype()));
ds.setDateofvalidation(mapField(d.getDateofvalidation()));
ds.setEoscdatasourcetype(mapCodeLabel(d.getEoscdatasourcetype()));
ds.setEosctype(mapCodeLabel(d.getEosctype()));
ds.setFulltextdownload(d.getFulltextdownload());
ds.setJurisdiction(mapCodeLabel(d.getJurisdiction()));
ds.setLanguages(d.getLanguages());
ds.setLatitude(mapField(d.getLatitude()));
ds.setLongitude(mapField(d.getLongitude()));
ds.setLastconsenttermsofusedate(d.getLastconsenttermsofusedate());
ds.setMissionstatementurl(mapField(d.getMissionstatementurl()));
ds.setNamespaceprefix(mapField(d.getNamespaceprefix()));
ds.setOdcontenttypes(mapFieldList(d.getOdcontenttypes()));
ds.setOdlanguages(mapFieldList(d.getOdlanguages()));
ds.setOdnumberofitems(mapField(d.getOdnumberofitems()));
ds.setOdnumberofitemsdate(mapField(d.getOdnumberofitemsdate()));
ds.setOdpolicies(mapField(d.getOdpolicies()));
ds.setOpenairecompatibility(mapCodeLabel(d.getOpenairecompatibility()));
ds.setPidsystems(mapField(d.getPidsystems()));
ds.setPolicies(mapCodeLabelKV(d.getPolicies()));
ds.setPreservationpolicyurl(d.getPreservationpolicyurl());
ds.setProvidedproducttypes(ds.getProvidedproducttypes());
ds.setReleaseenddate(mapField(d.getReleasestartdate()));
ds.setReleasestartdate(mapField(d.getReleasestartdate()));
ds.setResearchentitytypes(ds.getResearchentitytypes());
ds.setResearchproductaccesspolicies(d.getResearchproductaccesspolicies());
ds.setResearchproductmetadataaccesspolicies(d.getResearchproductmetadataaccesspolicies());
ds.setServiceprovider(mapField(d.getServiceprovider()));
ds.setSubjects(asSubjectSP(d.getSubjects()));
ds.setSubmissionpolicyurl(d.getSubmissionpolicyurl());
ds.setThematic(d.getThematic());
ds.setVersioncontrol(d.getVersioncontrol());
ds.setVersioning(mapField(d.getVersioning()));
return ds;
}
private static Result mapResult(eu.dnetlib.dhp.schema.oaf.Result r) {
Result rs = new Result();
rs.setResulttype(mapQualifier(r.getResulttype()));
rs.setAuthor(asAuthor(r.getAuthor()));
rs.setMaintitle(getMaintitle(r.getTitle()));
rs.setOtherTitles(getOtherTitles(r.getTitle()));
rs.setDescription(mapFieldList(r.getDescription()));
rs.setSubject(asSubject(r.getSubject()));
rs.setPublicationdate(mapField(r.getDateofacceptance()));
rs.setPublisher(mapField(r.getPublisher()));
rs.setEmbargoenddate(mapField(r.getEmbargoenddate()));
rs.setSource(mapFieldList(r.getSource()));
rs.setFormat(mapFieldList(r.getFormat()));
rs.setContributor(mapFieldList(r.getContributor()));
rs.setCoverage(mapFieldList(r.getCoverage()));
rs
.setBestaccessright(
BestAccessRight
.newInstance(r.getBestaccessright().getClassid(), r.getBestaccessright().getClassname()));
rs.setFulltext(mapFieldList(r.getFulltext()));
rs.setCountry(asCountry(r.getCountry()));
rs.setEoscifguidelines(asEOSCIF(r.getEoscifguidelines()));
rs.setGreen(r.getIsGreen());
rs
.setOpenAccessColor(
Optional
.ofNullable(r.getOpenAccessColor())
.map(color -> OpenAccessColor.valueOf(color.toString()))
.orElse(null));
rs.setInDiamondJournal(r.getIsInDiamondJournal());
rs.setPubliclyFunded(r.getPubliclyFunded());
rs.setTransformativeAgreement(r.getTransformativeAgreement());
rs.setInstance(mapInstances(r.getInstance()));
if (r instanceof Publication) {
Publication pub = (Publication) r;
rs.setJournal(mapJournal(pub.getJournal()));
} else if (r instanceof Dataset) {
Dataset d = (Dataset) r;
rs.setSize(mapField(d.getSize()));
rs.setVersion(mapField(d.getVersion()));
} else if (r instanceof Software) {
Software sw = (Software) r;
rs.setCodeRepositoryUrl(mapField(sw.getCodeRepositoryUrl()));
rs.setProgrammingLanguage(mapQualifier(sw.getProgrammingLanguage()));
rs.setDocumentationUrl(mapFieldList(sw.getDocumentationUrl()));
} else if (r instanceof OtherResearchProduct) {
OtherResearchProduct orp = (OtherResearchProduct) r;
rs.setContactperson(mapFieldList(orp.getContactperson()));
rs.setContactgroup(mapFieldList(orp.getContactgroup()));
rs.setTool(mapFieldList(orp.getTool()));
}
return rs;
}
@Nullable
private static List<String> getOtherTitles(List<StructuredProperty> titleList) {
return Optional
.ofNullable(titleList)
.map(
titles -> titles
.stream()
.filter(
t -> !"main title"
.equals(
Optional
.ofNullable(t.getQualifier())
.map(Qualifier::getClassid)
.orElse(null)))
.map(StructuredProperty::getValue)
.collect(Collectors.toList()))
.orElse(null);
}
private static String getMaintitle(List<StructuredProperty> titleList) {
return Optional
.ofNullable(titleList)
.flatMap(
titles -> titles
.stream()
.filter(
t -> "main title"
.equals(
Optional
.ofNullable(t.getQualifier())
.map(Qualifier::getClassid)
.orElse(null)))
.map(StructuredProperty::getValue)
.findFirst())
.orElse(null);
}
private static List<Instance> mapInstances(List<eu.dnetlib.dhp.schema.oaf.Instance> instanceList) {
return Optional
.ofNullable(instanceList)
.map(
instances -> instances
.stream()
.map(instance -> {
Instance i = new Instance();
i.setCollectedfrom(asProvenance(instance.getCollectedfrom()));
i.setHostedby(asProvenance(instance.getHostedby()));
i.setFulltext(i.getFulltext());
i.setPid(asPid(instance.getPid()));
i.setAlternateIdentifier(asPid(instance.getAlternateIdentifier()));
i.setAccessright(mapAccessRight(instance.getAccessright()));
i.setInstancetype(mapQualifier(instance.getInstancetype()));
i.setLicense(mapField(instance.getLicense()));
i.setUrl(instance.getUrl());
i.setRefereed(mapQualifier(instance.getRefereed()));
i.setDateofacceptance(mapField(instance.getDateofacceptance()));
i.setDistributionlocation(instance.getDistributionlocation());
i.setProcessingcharges(getProcessingcharges(instance));
return i;
})
.collect(Collectors.toList()))
.orElse(null);
}
private static APC getProcessingcharges(eu.dnetlib.dhp.schema.oaf.Instance instance) {
return Optional
.of(
APC
.newInstance(
mapField(instance.getProcessingchargecurrency()),
mapField(instance.getProcessingchargeamount())))
.filter(apc -> Objects.nonNull(apc.getAmount()) && Objects.nonNull(apc.getCurrency()))
.orElse(null);
}
private static AccessRight mapAccessRight(eu.dnetlib.dhp.schema.oaf.AccessRight accessright) {
return AccessRight
.newInstance(
mapQualifier(accessright),
Optional
.ofNullable(accessright.getOpenAccessRoute())
.map(route -> OpenAccessRoute.valueOf(route.toString()))
.orElse(null));
}
private static <T> T mapField(eu.dnetlib.dhp.schema.oaf.Field<T> f) {
return Optional.ofNullable(f).map(Field::getValue).orElse(null);
}
private static <T> List<T> mapFieldList(List<eu.dnetlib.dhp.schema.oaf.Field<T>> fl) {
return Optional
.ofNullable(fl)
.map(v -> v.stream().map(Field::getValue).collect(Collectors.toList()))
.orElse(null);
}
private static String mapQualifier(eu.dnetlib.dhp.schema.oaf.Qualifier q) {
return Optional.ofNullable(q).map(Qualifier::getClassid).orElse(null);
}
private static Journal mapJournal(eu.dnetlib.dhp.schema.oaf.Journal joaf) {
return Optional
.ofNullable(joaf)
.map(jo -> {
Journal j = new Journal();
j.setConferencedate(jo.getConferencedate());
j.setConferenceplace(jo.getConferenceplace());
j.setEdition(jo.getEdition());
j.setSp(jo.getSp());
j.setEp(jo.getEp());
j.setVol(jo.getVol());
j.setIss(jo.getEdition());
j.setName(jo.getName());
j.setIssnPrinted(jo.getIssnPrinted());
j.setIssnOnline(jo.getIssnOnline());
j.setIssnLinking(jo.getIssnLinking());
return j;
})
.orElse(null);
}
private static List<Provenance> asProvenance(List<KeyValue> keyValueList) {
return Optional
.ofNullable(keyValueList)
.map(
kvs -> kvs
.stream()
.map(ProvisionModelSupport::asProvenance)
.collect(Collectors.toList()))
.orElse(null);
}
private static Provenance asProvenance(KeyValue keyValue) {
return Optional.ofNullable(keyValue).map(cf -> Provenance.newInstance(cf.getKey(), cf.getValue())).orElse(null);
}
private static List<Context> asContext(List<eu.dnetlib.dhp.schema.oaf.Context> ctxList,
ContextMapper contextMapper) {
final Set<String> contexts = Optional
.ofNullable(ctxList)
.map(
ctx -> ctx
.stream()
.map(eu.dnetlib.dhp.schema.oaf.Context::getId)
.collect(Collectors.toCollection(HashSet::new)))
.orElse(new HashSet<>());
/* FIXME: Workaround for CLARIN mining issue: #3670#note-29 */
if (contexts.contains("dh-ch::subcommunity::2")) {
contexts.add("clarin");
}
return Optional
.ofNullable(contexts)
.map(
ctx -> ctx
.stream()
.map(contextPath -> {
Context context = new Context();
String id = "";
Map<String, Category> categoryMap = Maps.newHashMap();
for (final String token : Splitter.on("::").split(contextPath)) {
id += token;
final ContextDef def = contextMapper.get(id);
if (def == null) {
continue;
}
if (def.getName().equals("context")) {
context.setId(def.getId());
context.setLabel(def.getLabel());
context.setType(def.getType());
}
if (def.getName().equals("category")) {
Category category = Category.newInstance(def.getId(), def.getLabel());
if (Objects.isNull(context.getCategory())) {
context.setCategory(Lists.newArrayList());
}
context.getCategory().add(category);
categoryMap.put(def.getId(), category);
}
if (def.getName().equals("concept")) {
String parentId = StringUtils.substringBeforeLast(def.getId(), "::");
if (categoryMap.containsKey(parentId)) {
categoryMap
.get(parentId)
.getConcept()
.add(Concept.newInstance(def.getId(), def.getLabel()));
}
}
id += "::";
}
return context;
})
.collect(Collectors.toList()))
.orElse(null);
}
private static List<Pid> asPid(List<StructuredProperty> pidList) {
return Optional
.ofNullable(pidList)
.map(
pids -> pids
.stream()
.map(p -> Pid.newInstance(p.getQualifier().getClassid(), p.getValue()))
.collect(Collectors.toList()))
.orElse(null);
}
private static List<Author> asAuthor(List<eu.dnetlib.dhp.schema.oaf.Author> authorList) {
return Optional
.ofNullable(authorList)
.map(
authors -> authors
.stream()
.map(
a -> Author
.newInstance(a.getFullname(), a.getName(), a.getSurname(), a.getRank(), asPid(a.getPid())))
.collect(Collectors.toList()))
.orElse(null);
}
private static List<Subject> asSubject(List<eu.dnetlib.dhp.schema.oaf.Subject> subjectList) {
return Optional
.ofNullable(subjectList)
.map(
subjects -> subjects
.stream()
.filter(s -> Objects.nonNull(s.getQualifier()))
.filter(s -> Objects.nonNull(s.getQualifier().getClassid()))
.map(s -> Subject.newInstance(s.getValue(), s.getQualifier().getClassid()))
.collect(Collectors.toList()))
.orElse(null);
}
private static List<Subject> asSubjectSP(List<eu.dnetlib.dhp.schema.oaf.StructuredProperty> subjectList) {
return Optional
.ofNullable(subjectList)
.map(
subjects -> subjects
.stream()
.filter(s -> Objects.nonNull(s.getQualifier()))
.filter(s -> Objects.nonNull(s.getQualifier().getClassid()))
.map(s -> Subject.newInstance(s.getValue(), s.getQualifier().getClassid()))
.collect(Collectors.toList()))
.orElse(null);
}
private static Country asCountry(eu.dnetlib.dhp.schema.oaf.Qualifier country) {
return Optional
.ofNullable(country)
.filter(c -> Objects.nonNull(c.getClassid()) && Objects.nonNull(c.getClassname()))
.map(c -> Country.newInstance(c.getClassid(), c.getClassname()))
.orElse(null);
}
private static List<Country> asCountry(List<eu.dnetlib.dhp.schema.oaf.Country> countryList) {
return Optional
.ofNullable(countryList)
.map(
countries -> countries
.stream()
.map(c -> Country.newInstance(c.getClassid(), c.getClassname()))
.collect(Collectors.toList()))
.orElse(null);
}
private static List<EoscIfGuidelines> asEOSCIF(List<eu.dnetlib.dhp.schema.oaf.EoscIfGuidelines> eoscIfGuidelines) {
return Optional
.ofNullable(eoscIfGuidelines)
.map(
eoscif -> eoscif
.stream()
.map(
e -> EoscIfGuidelines
.newInstance(e.getCode(), e.getLabel(), e.getUrl(), e.getSemanticRelation()))
.collect(Collectors.toList()))
.orElse(null);
}
private static List<CodeLabel> mapCodeLabelKV(List<KeyValue> kvList) {
return Optional
.ofNullable(kvList)
.map(
kvs -> kvs
.stream()
.map(ProvisionModelSupport::mapCodeLabel)
.collect(Collectors.toList()))
.orElse(null);
}
private static List<CodeLabel> mapCodeLabel(List<Qualifier> qualifiers) {
return Optional
.ofNullable(qualifiers)
.map(
list -> list
.stream()
.map(ProvisionModelSupport::mapCodeLabel)
.collect(Collectors.toList()))
.orElse(null);
}
private static CodeLabel mapCodeLabel(Qualifier qualifier) {
return Optional
.ofNullable(qualifier)
.map(q -> CodeLabel.newInstance(q.getClassid(), q.getClassname()))
.orElse(null);
}
private static CodeLabel mapCodeLabel(KeyValue kv) {
return Optional
.ofNullable(kv)
.map(q -> CodeLabel.newInstance(kv.getKey(), kv.getValue()))
.orElse(null);
}
}

@ -13,6 +13,8 @@ import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
public class RelatedEntity implements Serializable {
private static final long serialVersionUID = -4982643490443810597L;
private String id;
private String type;
@ -21,6 +23,8 @@ public class RelatedEntity implements Serializable {
private String websiteurl; // datasource, organizations, projects
// results
private String description;
private List<String> author;
private String dateofacceptance;
private String publisher;
private List<StructuredProperty> pid;
@ -75,6 +79,22 @@ public class RelatedEntity implements Serializable {
return websiteurl;
}
public String getDescription() {
return description;
}
public void setDescription(String description) {
this.description = description;
}
public List<String> getAuthor() {
return author;
}
public void setAuthor(List<String> author) {
this.author = author;
}
public void setWebsiteurl(String websiteurl) {
this.websiteurl = websiteurl;
}

@ -9,6 +9,8 @@ import eu.dnetlib.dhp.schema.oaf.Relation;
public class RelatedEntityWrapper implements Serializable {
private static final long serialVersionUID = -2624854064081757234L;
private Relation relation;
private RelatedEntity target;

@ -0,0 +1,41 @@
/*
* Copyright (c) 2024.
* SPDX-FileCopyrightText: © 2023 Consiglio Nazionale delle Ricerche
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
package eu.dnetlib.dhp.oa.provision.model;
import java.io.Serializable;
public class TupleWrapper implements Serializable {
private static final long serialVersionUID = -1418439827125577822L;
private String xml;
private String json;
public TupleWrapper() {
}
public TupleWrapper(String xml, String json) {
this.xml = xml;
this.json = json;
}
public String getXml() {
return xml;
}
public void setXml(String xml) {
this.xml = xml;
}
public String getJson() {
return json;
}
public void setJson(String json) {
this.json = json;
}
}

@ -1,6 +1,7 @@
package eu.dnetlib.dhp.oa.provision.utils;
import java.io.Serializable;
import java.io.StringReader;
import java.io.StringWriter;
import java.util.HashMap;
@ -32,7 +33,7 @@ import com.google.common.collect.Lists;
*
* @author claudio
*/
public class StreamingInputDocumentFactory {
public class StreamingInputDocumentFactory implements Serializable {
private static final String INDEX_FIELD_PREFIX = "__";
@ -40,6 +41,8 @@ public class StreamingInputDocumentFactory {
private static final String INDEX_RESULT = INDEX_FIELD_PREFIX + RESULT;
private static final String INDEX_JSON_RESULT = INDEX_FIELD_PREFIX + "json";
private static final String INDEX_RECORD_ID = INDEX_FIELD_PREFIX + "indexrecordidentifier";
private static final String DEFAULTDNETRESULT = "dnetResult";
@ -71,13 +74,17 @@ public class StreamingInputDocumentFactory {
this.resultName = resultName;
}
public SolrInputDocument parseDocument(final String inputDocument) {
public SolrInputDocument parseDocument(final String xml) {
return parseDocument(xml, "");
}
public SolrInputDocument parseDocument(final String xml, final String json) {
final StringWriter results = new StringWriter();
final List<Namespace> nsList = Lists.newLinkedList();
try {
XMLEventReader parser = inputFactory.get().createXMLEventReader(new StringReader(inputDocument));
XMLEventReader parser = inputFactory.get().createXMLEventReader(new StringReader(xml));
final SolrInputDocument indexDocument = new SolrInputDocument(new HashMap<>());
@ -95,13 +102,13 @@ public class StreamingInputDocumentFactory {
} else if (TARGETFIELDS.equals(localName)) {
parseTargetFields(indexDocument, parser);
} else if (resultName.equals(localName)) {
copyResult(indexDocument, results, parser, nsList, resultName);
copyResult(indexDocument, json, results, parser, nsList, resultName);
}
}
}
if (!indexDocument.containsKey(INDEX_RECORD_ID)) {
throw new IllegalStateException("cannot extract record ID from: " + inputDocument);
throw new IllegalStateException("cannot extract record ID from: " + xml);
}
return indexDocument;
@ -171,6 +178,7 @@ public class StreamingInputDocumentFactory {
*/
protected void copyResult(
final SolrInputDocument indexDocument,
final String json,
final StringWriter results,
final XMLEventReader parser,
final List<Namespace> nsList,
@ -205,6 +213,7 @@ public class StreamingInputDocumentFactory {
}
writer.close();
indexDocument.addField(INDEX_RESULT, results.toString());
indexDocument.addField(INDEX_JSON_RESULT, json);
} finally {
outputFactory.remove();
eventFactory.remove();

@ -100,13 +100,17 @@ public class TemplateFactory {
public String getInstance(
final List<String> instancemetadata, final String url) {
return getInstance(instancemetadata, Lists.newArrayList(url));
}
public String getInstance(
final List<String> instancemetadata, final List<String> url) {
return getTemplate(resources.getInstance())
.add("metadata", instancemetadata)
.add(
"webresources",
Optional
.ofNullable(url)
.map(u -> Lists.newArrayList(url))
.orElse(Lists.newArrayList())
.stream()
.filter(StringUtils::isNotBlank)

@ -49,6 +49,7 @@ import eu.dnetlib.dhp.schema.common.*;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory;
import eu.dnetlib.dhp.schema.oaf.utils.ModelHardLimits;
import scala.Tuple2;
public class XmlRecordFactory implements Serializable {
@ -365,6 +366,7 @@ public class XmlRecordFactory implements Serializable {
.getDescription()
.stream()
.filter(Objects::nonNull)
.limit(ModelHardLimits.MAX_ABSTRACTS)
.map(c -> XmlSerializationUtils.asXmlElement("description", c.getValue()))
.collect(Collectors.toCollection(HashSet::new)));
}
@ -1057,7 +1059,8 @@ public class XmlRecordFactory implements Serializable {
return kv != null && StringUtils.isNotBlank(kv.getKey()) && StringUtils.isNotBlank(kv.getValue());
}
private List<String> mapFields(final RelatedEntityWrapper link, final Set<String> contexts) {
private List<String> mapFields(final TemplateFactory templateFactory, final RelatedEntityWrapper link,
final Set<String> contexts) {
final Relation rel = link.getRelation();
final RelatedEntity re = link.getTarget();
final String targetType = link.getTarget().getType();
@ -1071,6 +1074,18 @@ public class XmlRecordFactory implements Serializable {
if (re.getTitle() != null && isNotBlank(re.getTitle().getValue())) {
metadata.add(XmlSerializationUtils.mapStructuredProperty("title", re.getTitle()));
}
if (StringUtils.isNotBlank(re.getDescription())) {
metadata.add(XmlSerializationUtils.asXmlElement("description", re.getDescription()));
}
if (re.getAuthor() != null) {
metadata
.addAll(
re
.getAuthor()
.stream()
.map(author -> XmlSerializationUtils.asXmlElement("creator", author))
.collect(Collectors.toList()));
}
if (isNotBlank(re.getDateofacceptance())) {
metadata
.add(XmlSerializationUtils.asXmlElement("dateofacceptance", re.getDateofacceptance()));
@ -1104,6 +1119,54 @@ public class XmlRecordFactory implements Serializable {
.map(p -> XmlSerializationUtils.mapStructuredProperty("pid", p))
.collect(Collectors.toList()));
}
if (re.getInstances() != null) {
re
.getInstances()
.forEach(i -> {
final List<String> instanceFields = Lists.newArrayList();
if (i.getAccessright() != null && !i.getAccessright().isBlank()) {
instanceFields
.add(XmlSerializationUtils.mapQualifier("accessright", i.getAccessright()));
}
if (i.getHostedby() != null) {
instanceFields.add(XmlSerializationUtils.mapKeyValue("hostedby", i.getHostedby()));
}
if (i.getDateofacceptance() != null && isNotBlank(i.getDateofacceptance().getValue())) {
instanceFields
.add(
XmlSerializationUtils
.asXmlElement("dateofacceptance", i.getDateofacceptance().getValue()));
}
if (i.getInstancetype() != null && !i.getInstancetype().isBlank()) {
instanceFields
.add(XmlSerializationUtils.mapQualifier("instancetype", i.getInstancetype()));
}
if (i.getRefereed() != null && !i.getRefereed().isBlank()) {
instanceFields.add(XmlSerializationUtils.mapQualifier("refereed", i.getRefereed()));
}
if (i.getLicense() != null && isNotBlank(i.getLicense().getValue())) {
instanceFields
.add(XmlSerializationUtils.asXmlElement("license", i.getLicense().getValue()));
}
if (isNotBlank(i.getFulltext())) {
instanceFields.add(XmlSerializationUtils.asXmlElement("fulltext", i.getFulltext()));
}
if (i.getUrl() != null && !i.getUrl().isEmpty()) {
instanceFields
.addAll(
i
.getUrl()
.stream()
.filter(StringUtils::isNotBlank)
.map(url -> XmlSerializationUtils.asXmlElement("url", url))
.collect(Collectors.toList()));
}
metadata.add(templateFactory.getInstance(instanceFields, i.getUrl()));
});
}
break;
case datasource:
if (isNotBlank(re.getOfficialname())) {
@ -1133,6 +1196,9 @@ public class XmlRecordFactory implements Serializable {
if (re.getCountry() != null && !re.getCountry().isBlank()) {
metadata.add(XmlSerializationUtils.mapQualifier("country", re.getCountry()));
}
if (StringUtils.isNotBlank(re.getWebsiteurl())) {
metadata.add(XmlSerializationUtils.asXmlElement("websiteurl", re.getWebsiteurl()));
}
break;
case project:
if (isNotBlank(re.getProjectTitle())) {
@ -1182,7 +1248,7 @@ public class XmlRecordFactory implements Serializable {
throw new IllegalArgumentException(
String.format("missing scheme for: <%s - %s>", type, targetType));
}
final HashSet<String> fields = Sets.newHashSet(mapFields(link, contexts));
final HashSet<String> fields = Sets.newHashSet(mapFields(templateFactory, link, contexts));
if (rel.getValidated() == null) {
rel.setValidated(false);
}
@ -1206,7 +1272,7 @@ public class XmlRecordFactory implements Serializable {
.map(link -> {
final String targetType = link.getTarget().getType();
final String name = ModelSupport.getMainType(EntityType.valueOf(targetType));
final HashSet<String> fields = Sets.newHashSet(mapFields(link, null));
final HashSet<String> fields = Sets.newHashSet(mapFields(templateFactory, link, null));
return templateFactory
.getChild(name, link.getTarget().getId(), Lists.newArrayList(fields));
})

@ -0,0 +1,26 @@
[
{
"paramName": "is",
"paramLongName": "isLookupUrl",
"paramDescription": "URL of the isLookUp Service",
"paramRequired": true
},
{
"paramName": "i",
"paramLongName": "inputPath",
"paramDescription": "the path of the sequence file to read the XML records",
"paramRequired": true
},
{
"paramName": "f",
"paramLongName": "format",
"paramDescription": "MDFormat name found in the IS profile",
"paramRequired": true
},
{
"paramName": "op",
"paramLongName": "outputPath",
"paramDescription": "path on hdfs activating an alternative output for the SolrInputDocuments",
"paramRequired": false
}
]

@ -22,17 +22,5 @@
"paramLongName": "batchSize",
"paramDescription": "size of the batch of documents sent to solr",
"paramRequired": false
},
{
"paramName": "of",
"paramLongName": "outputFormat",
"paramDescription": "decides the job output format, SOLR | HDFS",
"paramRequired": false
},
{
"paramName": "op",
"paramLongName": "outputPath",
"paramDescription": "path on hdfs activating an alternative output for the SolrInputDocuments",
"paramRequired": false
}
]

@ -16,5 +16,11 @@
"paramLongName": "contextApiBaseUrl",
"paramDescription": "URL of the context API",
"paramRequired": true
},
{
"paramName": "isu",
"paramLongName": "isLookupUrl",
"paramDescription": "URL of the context ISLookup Service",
"paramRequired": true
}
]

@ -592,8 +592,9 @@
--conf spark.network.timeout=${sparkNetworkTimeout}
</spark-opts>
<arg>--inputPath</arg><arg>${workingDir}/join_entities</arg>
<arg>--outputPath</arg><arg>${workingDir}/xml</arg>
<arg>--outputPath</arg><arg>${workingDir}/xml_json</arg>
<arg>--contextApiBaseUrl</arg><arg>${contextApiBaseUrl}</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
</spark>
<ok to="should_index"/>
<error to="Kill"/>
@ -602,8 +603,8 @@
<decision name="should_index">
<switch>
<case to="drop_solr_collection">${wf:conf('shouldIndex') eq 'true'}</case>
<case to="End">${wf:conf('shouldIndex') eq 'false'}</case>
<default to="drop_solr_collection"/>
<case to="dump_solr_records_hdfs">${wf:conf('shouldIndex') eq 'false'}</case>
<default to="dump_solr_records_hdfs"/>
</switch>
</decision>
@ -646,12 +647,10 @@
--conf spark.hadoop.mapreduce.map.speculative=false
--conf spark.hadoop.mapreduce.reduce.speculative=false
</spark-opts>
<arg>--inputPath</arg><arg>${workingDir}/xml</arg>
<arg>--inputPath</arg><arg>${workingDir}/xml_json</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--format</arg><arg>${format}</arg>
<arg>--batchSize</arg><arg>${batchSize}</arg>
<arg>--outputFormat</arg><arg>${outputFormat}</arg>
<arg>--outputPath</arg><arg>${workingDir}/solr_documents</arg>
</spark>
<ok to="commit_solr_collection"/>
<error to="Kill"/>
@ -674,5 +673,30 @@
<error to="Kill"/>
</action>
<action name="dump_solr_records_hdfs">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>dump_solr_records_hdfs</name>
<class>eu.dnetlib.dhp.oa.provision.SolrRecordDumpJob</class>
<jar>dhp-graph-provision-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCoresForJoining}
--executor-memory=${sparkExecutorMemoryForJoining}
--driver-memory=${sparkDriverMemoryForJoining}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--inputPath</arg><arg>${workingDir}/xml_json</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--format</arg><arg>${format}</arg>
<arg>--outputPath</arg><arg>${workingDir}/solr_documents</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

@ -57,7 +57,7 @@ public class EOSCFuture_Test {
IOUtils.toString(getClass().getResourceAsStream("eosc-future/photic-zone.json")),
OtherResearchProduct.class);
final String xml = xmlRecordFactory.build(new JoinedEntity<>(p));
final String xml = xmlRecordFactory.build(new JoinedEntity(p));
assertNotNull(xml);

@ -63,7 +63,7 @@ public class IndexRecordTransformerTest {
final Project pj = load("project.json", Project.class);
final Relation rel = load("relToValidatedProject.json", Relation.class);
final JoinedEntity je = new JoinedEntity<>(p);
final JoinedEntity je = new JoinedEntity(p);
je
.setLinks(
Lists
@ -86,7 +86,7 @@ public class IndexRecordTransformerTest {
final Publication p = load("publication.json", Publication.class);
final JoinedEntity<Publication> je = new JoinedEntity<>(p);
final JoinedEntity je = new JoinedEntity(p);
final String record = xmlRecordFactory.build(je);
assertNotNull(record);
SolrInputDocument solrDoc = testRecordTransformation(record);
@ -102,7 +102,7 @@ public class IndexRecordTransformerTest {
final Publication p = load("riunet.json", Publication.class);
final JoinedEntity je = new JoinedEntity<>(p);
final JoinedEntity je = new JoinedEntity(p);
final String record = xmlRecordFactory.build(je);
assertNotNull(record);
testRecordTransformation(record);

@ -0,0 +1,51 @@
/*
* Copyright (c) 2024.
* SPDX-FileCopyrightText: © 2023 Consiglio Nazionale delle Ricerche
* SPDX-License-Identifier: AGPL-3.0-or-later
*/
package eu.dnetlib.dhp.oa.provision;
import java.io.IOException;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.oa.provision.model.JoinedEntity;
import eu.dnetlib.dhp.schema.oaf.Field;
import eu.dnetlib.dhp.schema.oaf.Journal;
import eu.dnetlib.dhp.schema.oaf.Organization;
import eu.dnetlib.dhp.schema.oaf.Publication;
class JoinedEntityTest {
private static final Logger log = LoggerFactory.getLogger(JoinedEntityTest.class);
@Test
void test_serialisation() throws IOException {
Publication p = new Publication();
p.setId("p1");
Journal j = new Journal();
j.setIss("1234-5678");
p.setJournal(j);
Organization o = new Organization();
o.setId("o1");
Field<String> lName = new Field<>();
lName.setValue("CNR");
o.setLegalname(lName);
ObjectMapper mapper = new ObjectMapper();
mapper.setSerializationInclusion(JsonInclude.Include.NON_EMPTY);
final String json = mapper.writeValueAsString(new JoinedEntity(p));
log.info(json);
}
}

@ -86,8 +86,7 @@ public class SolrConfigExploreTest extends SolrExploreTest {
String inputPath = "src/test/resources/eu/dnetlib/dhp/oa/provision/xml";
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize, XmlIndexingJob.OutputFormat.SOLR, null)
.run(isLookupClient);
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize).run(isLookupClient);
Assertions.assertEquals(0, miniCluster.getSolrClient().commit().getStatus());
String[] queryStrings = {

@ -95,7 +95,7 @@ public class SolrConfigTest extends SolrTest {
String inputPath = "src/test/resources/eu/dnetlib/dhp/oa/provision/xml";
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize, XmlIndexingJob.OutputFormat.SOLR, null)
new XmlIndexingJob(spark, inputPath, FORMAT, batchSize)
.run(isLookupClient);
Assertions.assertEquals(0, miniCluster.getSolrClient().commit().getStatus());

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

Loading…
Cancel
Save