Merge branch 'kubernetes' of code-repo.d4science.org:D-Net/dnet-hadoop into kubernetes

# Conflicts:
#	dhp-shade-package/dependency-reduced-pom.xml
#	dhp-shade-package/pom.xml
#	dhp-workflows/dhp-aggregation/src/main/java/eu/dnetlib/dhp/actionmanager/personentity/ExtractPerson.java
#	dhp-workflows/pom.xml
This commit is contained in:
Sandro La Bruzzo 2024-12-10 09:16:49 +01:00
commit 9cb8755124
132 changed files with 4428 additions and 1344 deletions

4
.gitignore vendored
View File

@ -27,4 +27,6 @@ spark-warehouse
/**/.factorypath
/**/.scalafmt.conf
/.java-version
/dhp-shade-package/dependency-reduced-pom.xml
/**/dependency-reduced-pom.xml
/**/job.properties

View File

@ -10,6 +10,11 @@ public class Constants {
public static final Map<String, String> accessRightsCoarMap = Maps.newHashMap();
public static final Map<String, String> coarCodeLabelMap = Maps.newHashMap();
public static final String RAID_NS_PREFIX = "raid________";
public static final String END_DATE = "endDate";
public static final String START_DATE = "startDate";
public static final String ROR_NS_PREFIX = "ror_________";
public static final String ROR_OPENAIRE_ID = "10|openaire____::993a7ae7a863813cf95028b50708e222";

View File

@ -1,5 +1,5 @@
package eu.dnetlib.dhp.actionmanager.personentity;
package eu.dnetlib.dhp.common.person;
import java.util.Arrays;
import java.util.Iterator;
@ -61,7 +61,7 @@ public class CoAuthorshipIterator implements Iterator<Relation> {
private Relation getRelation(String orcid1, String orcid2) {
String source = PERSON_PREFIX + IdentifierFactory.md5(orcid1);
String target = PERSON_PREFIX + IdentifierFactory.md5(orcid2);
return OafMapperUtils
Relation relation = OafMapperUtils
.getRelation(
source, target, ModelConstants.PERSON_PERSON_RELTYPE,
ModelConstants.PERSON_PERSON_SUBRELTYPE,
@ -76,5 +76,7 @@ public class CoAuthorshipIterator implements Iterator<Relation> {
ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS),
"0.91"),
null);
relation.setValidated(true);
return relation;
}
}

View File

@ -1,12 +1,9 @@
package eu.dnetlib.dhp.actionmanager.personentity;
package eu.dnetlib.dhp.common.person;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
import eu.dnetlib.dhp.schema.oaf.Relation;
public class Coauthors implements Serializable {
private List<String> coauthors;

View File

@ -2,8 +2,7 @@
package eu.dnetlib.dhp.oa.merge;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import static org.apache.spark.sql.functions.col;
import static org.apache.spark.sql.functions.when;
import static org.apache.spark.sql.functions.*;
import java.util.Map;
import java.util.Optional;
@ -135,7 +134,9 @@ public class GroupEntitiesSparkJob {
.applyCoarVocabularies(entity, vocs),
OAFENTITY_KRYO_ENC)
.groupByKey((MapFunction<OafEntity, String>) OafEntity::getId, Encoders.STRING())
.mapGroups((MapGroupsFunction<String, OafEntity, OafEntity>) MergeUtils::mergeById, OAFENTITY_KRYO_ENC)
.mapGroups(
(MapGroupsFunction<String, OafEntity, OafEntity>) (key, group) -> MergeUtils.mergeById(group, vocs),
OAFENTITY_KRYO_ENC)
.map(
(MapFunction<OafEntity, Tuple2<String, OafEntity>>) t -> new Tuple2<>(
t.getClass().getName(), t),

View File

@ -2,7 +2,6 @@
package eu.dnetlib.dhp.schema.oaf.utils;
import static eu.dnetlib.dhp.schema.common.ModelConstants.*;
import static eu.dnetlib.dhp.schema.common.ModelConstants.OPENAIRE_META_RESOURCE_TYPE;
import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.getProvenance;
import java.net.MalformedURLException;
@ -363,6 +362,8 @@ public class GraphCleaningFunctions extends CleaningFunctions {
// nothing to clean here
} else if (value instanceof Project) {
// nothing to clean here
} else if (value instanceof Person) {
// nothing to clean here
} else if (value instanceof Organization) {
Organization o = (Organization) value;
if (Objects.isNull(o.getCountry()) || StringUtils.isBlank(o.getCountry().getClassid())) {
@ -694,6 +695,7 @@ public class GraphCleaningFunctions extends CleaningFunctions {
}
}
// set ORCID_PENDING to all orcid values that are not coming from ORCID provenance
for (Author a : r.getAuthor()) {
if (Objects.isNull(a.getPid())) {
a.setPid(Lists.newArrayList());
@ -750,6 +752,40 @@ public class GraphCleaningFunctions extends CleaningFunctions {
.collect(Collectors.toList()));
}
}
// Identify clashing ORCIDS:that is same ORCID associated to multiple authors in this result
Map<String, Integer> clashing_orcid = new HashMap<>();
for (Author a : r.getAuthor()) {
a
.getPid()
.stream()
.filter(
p -> StringUtils
.contains(StringUtils.lowerCase(p.getQualifier().getClassid()), ORCID_PENDING))
.map(StructuredProperty::getValue)
.distinct()
.forEach(orcid -> clashing_orcid.compute(orcid, (k, v) -> (v == null) ? 1 : v + 1));
}
Set<String> clashing = clashing_orcid
.entrySet()
.stream()
.filter(ee -> ee.getValue() > 1)
.map(Map.Entry::getKey)
.collect(Collectors.toSet());
// filter out clashing orcids
for (Author a : r.getAuthor()) {
a
.setPid(
a
.getPid()
.stream()
.filter(p -> !clashing.contains(p.getValue()))
.collect(Collectors.toList()));
}
}
if (value instanceof Publication) {
@ -808,7 +844,7 @@ public class GraphCleaningFunctions extends CleaningFunctions {
return author;
}
private static Optional<String> cleanDateField(Field<String> dateofacceptance) {
public static Optional<String> cleanDateField(Field<String> dateofacceptance) {
return Optional
.ofNullable(dateofacceptance)
.map(Field::getValue)

View File

@ -204,6 +204,7 @@ public class IdentifierFactory implements Serializable {
.map(
pp -> pp
.stream()
.filter(p -> StringUtils.isNotBlank(p.getValue()))
// filter away PIDs provided by a DS that is not considered an authority for the
// given PID Type
.filter(p -> shouldFilterPidByCriteria(collectedFrom, p, mapHandles))

View File

@ -23,24 +23,30 @@ import org.apache.commons.lang3.tuple.Pair;
import com.github.sisyphsu.dateparser.DateParserUtils;
import com.google.common.base.Joiner;
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup;
import eu.dnetlib.dhp.oa.merge.AuthorMerger;
import eu.dnetlib.dhp.schema.common.AccessRightComparator;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
public class MergeUtils {
public static <T extends Oaf> T mergeById(String s, Iterator<T> oafEntityIterator) {
return mergeGroup(s, oafEntityIterator, true);
public static <T extends Oaf> T mergeById(Iterator<T> oafEntityIterator, VocabularyGroup vocs) {
return mergeGroup(oafEntityIterator, true, vocs);
}
public static <T extends Oaf> T mergeGroup(String s, Iterator<T> oafEntityIterator) {
return mergeGroup(s, oafEntityIterator, false);
public static <T extends Oaf> T mergeGroup(Iterator<T> oafEntityIterator) {
return mergeGroup(oafEntityIterator, false);
}
public static <T extends Oaf> T mergeGroup(String s, Iterator<T> oafEntityIterator,
boolean checkDelegateAuthority) {
public static <T extends Oaf> T mergeGroup(Iterator<T> oafEntityIterator, boolean checkDelegateAuthority) {
return mergeGroup(oafEntityIterator, checkDelegateAuthority, null);
}
public static <T extends Oaf> T mergeGroup(Iterator<T> oafEntityIterator,
boolean checkDelegateAuthority, VocabularyGroup vocs) {
ArrayList<T> sortedEntities = new ArrayList<>();
oafEntityIterator.forEachRemaining(sortedEntities::add);
@ -49,13 +55,55 @@ public class MergeUtils {
Iterator<T> it = sortedEntities.iterator();
T merged = it.next();
if (!it.hasNext() && merged instanceof Result && vocs != null) {
return enforceResultType(vocs, (Result) merged);
} else {
while (it.hasNext()) {
merged = checkedMerge(merged, it.next(), checkDelegateAuthority);
}
}
return merged;
}
private static <T extends Oaf> T enforceResultType(VocabularyGroup vocs, Result mergedResult) {
if (Optional.ofNullable(mergedResult.getInstance()).map(List::isEmpty).orElse(true)) {
return (T) mergedResult;
} else {
final Instance i = mergedResult.getInstance().get(0);
if (!vocs.vocabularyExists(ModelConstants.DNET_RESULT_TYPOLOGIES)) {
return (T) mergedResult;
} else {
final String expectedResultType = Optional
.ofNullable(
vocs
.lookupTermBySynonym(
ModelConstants.DNET_RESULT_TYPOLOGIES, i.getInstancetype().getClassid()))
.orElse(ModelConstants.ORP_DEFAULT_RESULTTYPE)
.getClassid();
// there is a clash among the result types
if (!expectedResultType.equals(mergedResult.getResulttype().getClassid())) {
Result result = (Result) Optional
.ofNullable(ModelSupport.oafTypes.get(expectedResultType))
.map(r -> {
try {
return r.newInstance();
} catch (InstantiationException | IllegalAccessException e) {
throw new IllegalStateException(e);
}
})
.orElse(new OtherResearchProduct());
result.setId(mergedResult.getId());
return (T) mergeResultFields(result, mergedResult);
} else {
return (T) mergedResult;
}
}
}
}
public static <T extends Oaf> T checkedMerge(final T left, final T right, boolean checkDelegateAuthority) {
return (T) merge(left, right, checkDelegateAuthority);
}
@ -106,7 +154,7 @@ public class MergeUtils {
return mergeSoftware((Software) left, (Software) right);
}
return mergeResultFields((Result) left, (Result) right);
return left;
} else if (sameClass(left, right, Datasource.class)) {
// TODO
final int trust = compareTrust(left, right);
@ -654,16 +702,9 @@ public class MergeUtils {
}
private static Field<String> selectOldestDate(Field<String> d1, Field<String> d2) {
if (d1 == null || StringUtils.isBlank(d1.getValue())) {
if (!GraphCleaningFunctions.cleanDateField(d1).isPresent()) {
return d2;
} else if (d2 == null || StringUtils.isBlank(d2.getValue())) {
return d1;
}
if (StringUtils.contains(d1.getValue(), "null")) {
return d2;
}
if (StringUtils.contains(d2.getValue(), "null")) {
} else if (!GraphCleaningFunctions.cleanDateField(d2).isPresent()) {
return d1;
}
@ -715,7 +756,11 @@ public class MergeUtils {
private static String spKeyExtractor(StructuredProperty sp) {
return Optional
.ofNullable(sp)
.map(s -> Joiner.on("||").join(qualifierKeyExtractor(s.getQualifier()), s.getValue()))
.map(
s -> Joiner
.on("||")
.useForNull("")
.join(qualifierKeyExtractor(s.getQualifier()), s.getValue()))
.orElse(null);
}

View File

@ -1,6 +1,12 @@
package eu.dnetlib.dhp.schema.oaf.utils;
import java.util.Map;
import com.google.common.collect.Maps;
import eu.dnetlib.dhp.schema.common.ModelConstants;
public class ModelHardLimits {
private ModelHardLimits() {
@ -12,6 +18,7 @@ public class ModelHardLimits {
public static final int MAX_EXTERNAL_ENTITIES = 50;
public static final int MAX_AUTHORS = 200;
public static final int MAX_RELATED_AUTHORS = 20;
public static final int MAX_AUTHOR_FULLNAME_LENGTH = 1000;
public static final int MAX_TITLE_LENGTH = 5000;
public static final int MAX_TITLES = 10;
@ -19,6 +26,12 @@ public class ModelHardLimits {
public static final int MAX_ABSTRACT_LENGTH = 150000;
public static final int MAX_RELATED_ABSTRACT_LENGTH = 500;
public static final int MAX_INSTANCES = 10;
public static final Map<String, Long> MAX_RELATIONS_BY_RELCLASS = Maps.newHashMap();
static {
MAX_RELATIONS_BY_RELCLASS.put(ModelConstants.PERSON_PERSON_HASCOAUTHORED, 500L);
MAX_RELATIONS_BY_RELCLASS.put(ModelConstants.RESULT_PERSON_HASAUTHORED, 500L);
}
public static String getCollectionName(String format) {
return format + SEPARATOR + LAYOUT + SEPARATOR + INTERPRETATION;

View File

@ -26,7 +26,7 @@ public class PidCleaner {
String value = Optional
.ofNullable(pidValue)
.map(String::trim)
.orElseThrow(() -> new IllegalArgumentException("PID value cannot be empty"));
.orElseThrow(() -> new IllegalArgumentException("PID (" + pidType + ") value cannot be empty"));
switch (pidType) {

View File

@ -179,7 +179,7 @@ class OafMapperUtilsTest {
assertEquals(
ModelConstants.DATASET_RESULTTYPE_CLASSID,
((Result) MergeUtils
.merge(p2, d1))
.merge(p2, d1, true))
.getResulttype()
.getClassid());
}

View File

@ -38,7 +38,7 @@ public class NumAuthorsTitleSuffixPrefixChain extends AbstractClusteringFunction
@Override
protected Collection<String> doApply(Config conf, String s) {
return suffixPrefixChain(cleanup(s), param("mod"));
return suffixPrefixChain(cleanup(s), paramOrDefault("mod", 10));
}
private Collection<String> suffixPrefixChain(String s, int mod) {

View File

@ -90,7 +90,7 @@ public class AbstractPaceFunctions extends PaceCommonUtils {
inferFrom = normalize(inferFrom);
inferFrom = filterAllStopWords(inferFrom);
Set<String> cities = getCities(inferFrom, 4);
return citiesToCountry(cities).stream().findFirst().orElse("UNKNOWN");
return citiesToCountry(cities).stream().filter(Objects::nonNull).findFirst().orElse("UNKNOWN");
}
public static String cityInference(String original) {

View File

@ -54,6 +54,22 @@ public class FieldDef implements Serializable {
public FieldDef() {
}
public FieldDef clone() {
FieldDef fieldDef = new FieldDef();
fieldDef.setName(this.name);
fieldDef.setPath(this.path);
fieldDef.setType(this.type);
fieldDef.setOverrideMatch(this.overrideMatch);
fieldDef.setSize(this.size);
fieldDef.setLength(this.length);
fieldDef.setFilter(this.filter);
fieldDef.setSorted(this.sorted);
fieldDef.setClean(this.clean);
fieldDef.setInfer(this.infer);
fieldDef.setInferenceFrom(this.inferenceFrom);
return fieldDef;
}
public String getInferenceFrom() {
return inferenceFrom;
}

View File

@ -19,48 +19,10 @@ case class SparkDeduper(conf: DedupConfig) extends Serializable {
val model: SparkModel = SparkModel(conf)
val dedup: (Dataset[Row] => Dataset[Row]) = df => {
df.transform(filterAndCleanup)
.transform(generateClustersWithCollect)
df.transform(generateClustersWithCollect)
.transform(processBlocks)
}
val filterAndCleanup: (Dataset[Row] => Dataset[Row]) = df => {
val df_with_filters = conf.getPace.getModel.asScala.foldLeft(df)((res, fdef) => {
if (conf.blacklists.containsKey(fdef.getName)) {
res.withColumn(
fdef.getName + "_filtered",
filterColumnUDF(fdef).apply(new Column(fdef.getName))
)
} else {
res
}
})
df_with_filters
}
def filterColumnUDF(fdef: FieldDef): UserDefinedFunction = {
val blacklist: Predicate[String] = conf.blacklists().get(fdef.getName)
if (blacklist == null) {
throw new IllegalArgumentException("Column: " + fdef.getName + " does not have any filter")
} else {
fdef.getType match {
case Type.List | Type.JSON =>
udf[Array[String], Array[String]](values => {
values.filter((v: String) => !blacklist.test(v))
})
case _ =>
udf[String, String](v => {
if (blacklist.test(v)) ""
else v
})
}
}
}
val generateClustersWithCollect: (Dataset[Row] => Dataset[Row]) = df_with_filters => {
var df_with_clustering_keys: Dataset[Row] = null

View File

@ -5,12 +5,12 @@ import eu.dnetlib.pace.common.AbstractPaceFunctions
import eu.dnetlib.pace.config.{DedupConfig, Type}
import eu.dnetlib.pace.util.{MapDocumentUtil, SparkCompatUtils}
import org.apache.commons.lang3.StringUtils
import org.apache.spark.sql.catalyst.encoders.RowEncoder
import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema
import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType}
import org.apache.spark.sql.{Dataset, Row}
import java.util.Locale
import java.util.function.Predicate
import java.util.regex.Pattern
import scala.collection.JavaConverters._
@ -29,8 +29,20 @@ case class SparkModel(conf: DedupConfig) {
identifier.setName(identifierFieldName)
identifier.setType(Type.String)
// create fields for blacklist
val filtered = conf.getPace.getModel.asScala.flatMap(fdef => {
if (conf.blacklists().containsKey(fdef.getName)) {
val fdef_filtered = fdef.clone()
fdef_filtered.setName(fdef.getName + "_filtered")
Seq(fdef, fdef_filtered)
}
else {
Seq(fdef)
}
})
// Construct a Spark StructType representing the schema of the model
(Seq(identifier) ++ conf.getPace.getModel.asScala)
(Seq(identifier) ++ filtered)
.foldLeft(
new StructType()
)((resType, fieldDef) => {
@ -44,7 +56,6 @@ case class SparkModel(conf: DedupConfig) {
})
})
}
val identityFieldPosition: Int = schema.fieldIndex(identifierFieldName)
@ -52,7 +63,8 @@ case class SparkModel(conf: DedupConfig) {
val orderingFieldPosition: Int = schema.fieldIndex(orderingFieldName)
val parseJsonDataset: (Dataset[String] => Dataset[Row]) = df => {
df.map(r => rowFromJson(r))(SparkCompatUtils.encoderFor(schema))
df
.map(r => rowFromJson(r))(SparkCompatUtils.encoderFor(schema))
}
def rowFromJson(json: String): Row = {
@ -64,9 +76,11 @@ case class SparkModel(conf: DedupConfig) {
schema.fieldNames.zipWithIndex.foldLeft(values) {
case ((res, (fname, index))) =>
val fdef = conf.getPace.getModelMap.get(fname)
val fdef = conf.getPace.getModelMap.get(fname.split("_filtered")(0))
if (fdef != null) {
if (!fname.contains("_filtered")) { //process fields with no blacklist
res(index) = fdef.getType match {
case Type.String | Type.Int =>
MapDocumentUtil.truncateValue(
@ -99,6 +113,26 @@ case class SparkModel(conf: DedupConfig) {
case Type.DoubleArray =>
MapDocumentUtil.getJPathArray(fdef.getPath, json)
}
}
else { //process fields with blacklist
val blacklist: Predicate[String] = conf.blacklists().get(fdef.getName)
res(index) = fdef.getType match {
case Type.List | Type.JSON =>
MapDocumentUtil.truncateList(
MapDocumentUtil.getJPathList(fdef.getPath, documentContext, fdef.getType),
fdef.getSize
).asScala.filter((v: String) => !blacklist.test(v))
case _ =>
val value: String = MapDocumentUtil.truncateValue(
MapDocumentUtil.getJPathString(fdef.getPath, documentContext),
fdef.getLength
)
if (blacklist.test(value)) "" else value
}
}
val filter = fdef.getFilter
@ -125,13 +159,12 @@ case class SparkModel(conf: DedupConfig) {
}
if (StringUtils.isNotBlank(fdef.getInfer)) {
val inferFrom : String = if (StringUtils.isNotBlank(fdef.getInferenceFrom)) fdef.getInferenceFrom else fdef.getPath
val inferFrom: String = if (StringUtils.isNotBlank(fdef.getInferenceFrom)) fdef.getInferenceFrom else fdef.getPath
res(index) = res(index) match {
case x: Seq[String] => x.map(inference(_, MapDocumentUtil.getJPathString(inferFrom, documentContext), fdef.getInfer))
case _ => inference(res(index).toString, MapDocumentUtil.getJPathString(inferFrom, documentContext), fdef.getInfer)
}
}
}
res
@ -139,6 +172,7 @@ case class SparkModel(conf: DedupConfig) {
}
new GenericRowWithSchema(values, schema)
}
def clean(value: String, cleantype: String) : String = {

View File

@ -21,7 +21,7 @@ public class CodeMatch extends AbstractStringComparator {
public CodeMatch(Map<String, String> params) {
super(params);
this.params = params;
this.CODE_REGEX = Pattern.compile(params.getOrDefault("codeRegex", "[a-zA-Z]::\\d+"));
this.CODE_REGEX = Pattern.compile(params.getOrDefault("codeRegex", "[a-zA-Z]+::\\d+"));
}
public Set<String> getRegexList(String input) {

View File

@ -0,0 +1,67 @@
package eu.dnetlib.pace.tree;
import java.time.DateTimeException;
import java.time.LocalDate;
import java.time.Period;
import java.time.format.DateTimeFormatter;
import java.util.Locale;
import java.util.Map;
import com.wcohen.ss.AbstractStringDistance;
import eu.dnetlib.pace.config.Config;
import eu.dnetlib.pace.tree.support.AbstractStringComparator;
import eu.dnetlib.pace.tree.support.ComparatorClass;
@ComparatorClass("dateRange")
public class DateRange extends AbstractStringComparator {
int YEAR_RANGE;
public DateRange(Map<String, String> params) {
super(params, new com.wcohen.ss.JaroWinkler());
YEAR_RANGE = Integer.parseInt(params.getOrDefault("year_range", "3"));
}
public DateRange(final double weight) {
super(weight, new com.wcohen.ss.JaroWinkler());
}
protected DateRange(final double weight, final AbstractStringDistance ssalgo) {
super(weight, ssalgo);
}
public static boolean isNumeric(String str) {
return str.matches("\\d+"); // match a number with optional '-' and decimal.
}
@Override
public double distance(final String a, final String b, final Config conf) {
if (a.isEmpty() || b.isEmpty()) {
return -1.0; // return -1 if a field is missing
}
try {
DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd", Locale.ENGLISH);
LocalDate d1 = LocalDate.parse(a, formatter);
LocalDate d2 = LocalDate.parse(b, formatter);
Period period = Period.between(d1, d2);
return period.getYears() <= YEAR_RANGE ? 1.0 : 0.0;
} catch (DateTimeException e) {
return -1.0;
}
}
@Override
public double getWeight() {
return super.weight;
}
@Override
protected double normalize(final double d) {
return d;
}
}

View File

@ -41,21 +41,38 @@ public class JsonListMatch extends AbstractListComparator {
return -1;
}
final Set<String> ca = sa.stream().map(this::toComparableString).collect(Collectors.toSet());
final Set<String> cb = sb.stream().map(this::toComparableString).collect(Collectors.toSet());
Set<String> ca = sa.stream().map(this::toComparableString).collect(Collectors.toSet());
Set<String> cb = sb.stream().map(this::toComparableString).collect(Collectors.toSet());
switch (MODE) {
case "count":
return Sets.intersection(ca, cb).size();
case "percentage":
int incommon = Sets.intersection(ca, cb).size();
int simDiff = Sets.symmetricDifference(ca, cb).size();
if (incommon + simDiff == 0) {
return 0.0;
}
if (MODE.equals("percentage"))
return (double) incommon / (incommon + simDiff);
else
return incommon;
case "type":
Set<String> typesA = ca.stream().map(s -> s.split("::")[0]).collect(Collectors.toSet());
Set<String> typesB = cb.stream().map(s -> s.split("::")[0]).collect(Collectors.toSet());
Set<String> types = Sets.intersection(typesA, typesB);
if (types.isEmpty()) // if no common type, it is impossible to compare
return -1;
ca = ca.stream().filter(s -> types.contains(s.split("::")[0])).collect(Collectors.toSet());
cb = cb.stream().filter(s -> types.contains(s.split("::")[0])).collect(Collectors.toSet());
return (double) Sets.intersection(ca, cb).size() / types.size();
default:
return -1;
}
}
// converts every json into a comparable string basing on parameters
@ -69,7 +86,7 @@ public class JsonListMatch extends AbstractListComparator {
// for each path in the param list
for (String key : params.keySet().stream().filter(k -> k.contains("jpath")).collect(Collectors.toList())) {
String path = params.get(key);
String value = MapDocumentUtil.getJPathString(path, documentContext);
String value = MapDocumentUtil.getJPathString(path, documentContext).toLowerCase();
if (value == null || value.isEmpty())
value = "";
st.append(value);

View File

@ -227,4 +227,17 @@ public class ClusteringFunctionTest extends AbstractPaceTest {
System.out.println(cf.apply(conf, Lists.newArrayList(s)));
}
@Test
public void testNumAuthorsTitleSuffixPrefixChain() {
final ClusteringFunction cf = new NumAuthorsTitleSuffixPrefixChain(params);
params.put("mod", 10);
final String title = "PARP-2 Regulates SIRT1 Expression and Whole-Body Energy Expenditure";
final String num_authors = "10";
System.out.println("title = " + title);
System.out.println("num_authors = " + num_authors);
System.out.println(cf.apply(conf, Lists.newArrayList(num_authors, title)));
}
}

View File

@ -1,8 +1,7 @@
package eu.dnetlib.pace.common;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.*;
import org.junit.jupiter.api.*;
@ -54,8 +53,17 @@ public class PaceFunctionTest extends AbstractPaceFunctions {
System.out.println("Fixed aliases : " + fixAliases(TEST_STRING));
}
@Test()
public void countryInferenceTest_NPE() {
assertThrows(
NullPointerException.class,
() -> countryInference("UNKNOWN", null),
"Expected countryInference() to throw an NPE");
}
@Test
public void countryInferenceTest() {
assertEquals("UNKNOWN", countryInference("UNKNOWN", ""));
assertEquals("IT", countryInference("UNKNOWN", "Università di Bologna"));
assertEquals("UK", countryInference("UK", "Università di Bologna"));
assertEquals("IT", countryInference("UNKNOWN", "Universiteé de Naples"));

View File

@ -65,6 +65,43 @@ public class ComparatorTest extends AbstractPaceTest {
}
@Test
public void datasetVersionCodeMatchTest() {
params.put("codeRegex", "(?=[\\w-]*[a-zA-Z])(?=[\\w-]*\\d)[\\w-]+");
CodeMatch codeMatch = new CodeMatch(params);
// names have different codes
assertEquals(
0.0,
codeMatch
.distance(
"physical oceanography at ctd station june 1998 ev02a",
"physical oceanography at ctd station june 1998 ir02", conf));
// names have same code
assertEquals(
1.0,
codeMatch
.distance(
"physical oceanography at ctd station june 1998 ev02a",
"physical oceanography at ctd station june 1998 ev02a", conf));
// code is not in both names
assertEquals(
-1,
codeMatch
.distance(
"physical oceanography at ctd station june 1998",
"physical oceanography at ctd station june 1998 ev02a", conf));
assertEquals(
1.0,
codeMatch
.distance(
"physical oceanography at ctd station june 1998", "physical oceanography at ctd station june 1998",
conf));
}
@Test
public void listContainsMatchTest() {
@ -257,15 +294,15 @@ public class ComparatorTest extends AbstractPaceTest {
List<String> a = createFieldList(
Arrays
.asList(
"{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":null,\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:actionset\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"doi\",\"classname\":\"Digital Object Identifier\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"10.1111/pbi.12655\"}"),
"{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":null,\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:actionset\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"grid\",\"classname\":\"GRID Identifier\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"grid_1\"}",
"{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":null,\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:actionset\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"ror\",\"classname\":\"Research Organization Registry\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"ror_1\"}"),
"authors");
List<String> b = createFieldList(
Arrays
.asList(
"{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":\"\",\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:crosswalk:repository\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"pmc\",\"classname\":\"PubMed Central ID\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"PMC5399005\"}",
"{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":\"\",\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:crosswalk:repository\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"pmid\",\"classname\":\"PubMed ID\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"27775869\"}",
"{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":\"\",\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"user:claim\",\"classname\":\"Linked by user\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"doi\",\"classname\":\"Digital Object Identifier\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"10.1111/pbi.12655\"}",
"{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":\"\",\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:crosswalk:repository\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"handle\",\"classname\":\"Handle\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"1854/LU-8523529\"}"),
"{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":\"\",\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:crosswalk:repository\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"grid\",\"classname\":\"GRID Identifier\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"grid_1\"}",
"{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":\"\",\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"sysimport:crosswalk:repository\",\"classname\":\"Harvested\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"ror\",\"classname\":\"Research Organization Registry\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"ror_2\"}",
"{\"datainfo\":{\"deletedbyinference\":false,\"inferenceprovenance\":\"\",\"inferred\":false,\"invisible\":false,\"provenanceaction\":{\"classid\":\"user:claim\",\"classname\":\"Linked by user\",\"schemeid\":\"dnet:provenanceActions\",\"schemename\":\"dnet:provenanceActions\"},\"trust\":\"0.9\"},\"qualifier\":{\"classid\":\"isni\",\"classname\":\"ISNI Identifier\",\"schemeid\":\"dnet:pid_types\",\"schemename\":\"dnet:pid_types\"},\"value\":\"isni_1\"}"),
"authors");
double result = jsonListMatch.compare(a, b, conf);
@ -277,6 +314,13 @@ public class ComparatorTest extends AbstractPaceTest {
result = jsonListMatch.compare(a, b, conf);
assertEquals(1.0, result);
params.put("mode", "type");
jsonListMatch = new JsonListMatch(params);
result = jsonListMatch.compare(a, b, conf);
assertEquals(0.5, result);
}
@Test
@ -327,4 +371,34 @@ public class ComparatorTest extends AbstractPaceTest {
}
@Test
public void dateMatch() {
DateRange dateRange = new DateRange(params);
double result = dateRange.distance("2021-05-13", "2023-05-13", conf);
assertEquals(1.0, result);
result = dateRange.distance("2021-05-13", "2025-05-13", conf);
assertEquals(0.0, result);
result = dateRange.distance("", "2020-05-05", conf);
assertEquals(-1.0, result);
result = dateRange.distance("invalid date", "2021-05-02", conf);
assertEquals(-1.0, result);
}
@Test
public void titleVersionMatchTest() {
TitleVersionMatch titleVersionMatch = new TitleVersionMatch(params);
double result = titleVersionMatch
.compare(
"parp 2 regulates sirt 1 expression and whole body energy expenditure",
"parp 2 regulates sirt 1 expression and whole body energy expenditure", conf);
assertEquals(1.0, result);
}
}

View File

@ -1,123 +0,0 @@
<?xml version="1.0" encoding="UTF-8"?>
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<parent>
<artifactId>dhp</artifactId>
<groupId>eu.dnetlib.dhp</groupId>
<version>1.2.5-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>dhp-shade-package</artifactId>
<description>This module create a jar of all module dependencies</description>
<build>
<plugins>
<plugin>
<artifactId>maven-shade-plugin</artifactId>
<executions>
<execution>
<phase>package</phase>
<goals>
<goal>shade</goal>
</goals>
<configuration>
<transformers>
<transformer>
<mainClass>eu.dnetlib.dhp.oa.dedup.SparkCreateSimRels</mainClass>
</transformer>
<transformer />
<transformer>
<resource>META-INF/cxf/bus-extensions.txt</resource>
</transformer>
</transformers>
<filters>
<filter>
<artifact>*:*</artifact>
<excludes>
<exclude>META-INF/maven/**</exclude>
<exclude>META-INF/*.SF</exclude>
<exclude>META-INF/*.DSA</exclude>
<exclude>META-INF/*.RSA</exclude>
</excludes>
</filter>
</filters>
<relocations>
<relocation>
<pattern>com</pattern>
<shadedPattern>repackaged.com.google.common</shadedPattern>
<includes>
<include>com.google.common.**</include>
</includes>
</relocation>
</relocations>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
<dependencies>
<dependency>
<groupId>org.projectlombok</groupId>
<artifactId>lombok</artifactId>
<version>1.18.28</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.junit.jupiter</groupId>
<artifactId>junit-jupiter</artifactId>
<version>5.6.1</version>
<scope>test</scope>
<exclusions>
<exclusion>
<artifactId>junit-jupiter-api</artifactId>
<groupId>org.junit.jupiter</groupId>
</exclusion>
<exclusion>
<artifactId>junit-jupiter-params</artifactId>
<groupId>org.junit.jupiter</groupId>
</exclusion>
<exclusion>
<artifactId>junit-jupiter-engine</artifactId>
<groupId>org.junit.jupiter</groupId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-core</artifactId>
<version>3.3.3</version>
<scope>test</scope>
<exclusions>
<exclusion>
<artifactId>byte-buddy</artifactId>
<groupId>net.bytebuddy</groupId>
</exclusion>
<exclusion>
<artifactId>byte-buddy-agent</artifactId>
<groupId>net.bytebuddy</groupId>
</exclusion>
<exclusion>
<artifactId>objenesis</artifactId>
<groupId>org.objenesis</groupId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-junit-jupiter</artifactId>
<version>3.3.3</version>
<scope>test</scope>
<exclusions>
<exclusion>
<artifactId>junit-jupiter-api</artifactId>
<groupId>org.junit.jupiter</groupId>
</exclusion>
</exclusions>
</dependency>
</dependencies>
<distributionManagement>
<site>
<id>DHPSite</id>
<url>${dhp.site.stage.path}/dhp-common</url>
</site>
</distributionManagement>
</project>

View File

@ -25,17 +25,16 @@
<dependencies>
<!-- <dependency>-->
<!-- <groupId>eu.dnetlib.dhp</groupId>-->
<!-- <artifactId>dhp-actionmanager</artifactId>-->
<!-- <version>${project.version}</version>-->
<!-- </dependency>-->
<!-- <dependency>-->
<!-- <groupId>eu.dnetlib.dhp</groupId>-->
<!-- <artifactId>dhp-aggregation</artifactId>-->
<!-- <version>${project.version}</version>-->
<!-- </dependency>-->
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-aggregation</artifactId>
<version>${project.version}</version>
</dependency>
<!-- <dependency>-->
<!-- <groupId>eu.dnetlib.dhp</groupId>-->
<!-- <artifactId>dhp-blacklist</artifactId>-->
@ -46,11 +45,11 @@
<!-- <artifactId>dhp-broker-events</artifactId>-->
<!-- <version>${project.version}</version>-->
<!-- </dependency>-->
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-dedup-openaire</artifactId>
<version>${project.version}</version>
</dependency>
<!-- <dependency>-->
<!-- <groupId>eu.dnetlib.dhp</groupId>-->
<!-- <artifactId>dhp-dedup-openaire</artifactId>-->
<!-- <version>${project.version}</version>-->
<!-- </dependency>-->
<!-- <dependency>-->
<!-- <groupId>eu.dnetlib.dhp</groupId>-->
<!-- <artifactId>dhp-enrichment</artifactId>-->

View File

@ -151,12 +151,17 @@ public class PromoteActionPayloadForGraphTableJob {
SparkSession spark, String path, Class<G> rowClazz) {
logger.info("Reading graph table from path: {}", path);
if (HdfsSupport.exists(path, spark.sparkContext().hadoopConfiguration())) {
return spark
.read()
.textFile(path)
.map(
(MapFunction<String, G>) value -> OBJECT_MAPPER.readValue(value, rowClazz),
Encoders.bean(rowClazz));
} else {
logger.info("Found empty graph table from path: {}", path);
return spark.emptyDataset(Encoders.bean(rowClazz));
}
}
private static <A extends Oaf> Dataset<A> readActionPayload(
@ -223,7 +228,7 @@ public class PromoteActionPayloadForGraphTableJob {
rowClazz,
actionPayloadClazz);
if (shouldGroupById) {
if (Boolean.TRUE.equals(shouldGroupById)) {
return PromoteActionPayloadFunctions
.groupGraphTableByIdAndMerge(
joinedAndMerged, rowIdFn, mergeRowsAndGetFn, zeroFn, isNotZeroFn, rowClazz);
@ -250,6 +255,8 @@ public class PromoteActionPayloadForGraphTableJob {
return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Relation());
case "eu.dnetlib.dhp.schema.oaf.Software":
return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Software());
case "eu.dnetlib.dhp.schema.oaf.Person":
return () -> clazz.cast(new eu.dnetlib.dhp.schema.oaf.Person());
default:
throw new RuntimeException("unknown class: " + clazz.getCanonicalName());
}

View File

@ -50,7 +50,7 @@ public class PromoteActionPayloadFunctions {
PromoteAction.Strategy promoteActionStrategy,
Class<G> rowClazz,
Class<A> actionPayloadClazz) {
if (!isSubClass(rowClazz, actionPayloadClazz)) {
if (Boolean.FALSE.equals(isSubClass(rowClazz, actionPayloadClazz))) {
throw new RuntimeException(
"action payload type must be the same or be a super type of table row type");
}

View File

@ -7,3 +7,4 @@ promote_action_payload_for_project_table classpath eu/dnetlib/dhp/actionmanager/
promote_action_payload_for_publication_table classpath eu/dnetlib/dhp/actionmanager/wf/publication/oozie_app
promote_action_payload_for_relation_table classpath eu/dnetlib/dhp/actionmanager/wf/relation/oozie_app
promote_action_payload_for_software_table classpath eu/dnetlib/dhp/actionmanager/wf/software/oozie_app
promote_action_payload_for_person_table classpath eu/dnetlib/dhp/actionmanager/wf/person/oozie_app

View File

@ -135,21 +135,10 @@
<arg>--outputPath</arg><arg>${workingDir}/action_payload_by_type</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
</spark>
<ok to="ForkPromote"/>
<ok to="PromoteActionPayloadForDatasetTable"/>
<error to="Kill"/>
</action>
<fork name="ForkPromote">
<path start="PromoteActionPayloadForDatasetTable"/>
<path start="PromoteActionPayloadForDatasourceTable"/>
<path start="PromoteActionPayloadForOrganizationTable"/>
<path start="PromoteActionPayloadForOtherResearchProductTable"/>
<path start="PromoteActionPayloadForProjectTable"/>
<path start="PromoteActionPayloadForPublicationTable"/>
<path start="PromoteActionPayloadForRelationTable"/>
<path start="PromoteActionPayloadForSoftwareTable"/>
</fork>
<action name="PromoteActionPayloadForDatasetTable">
<sub-workflow>
<app-path>${wf:appPath()}/promote_action_payload_for_dataset_table</app-path>
@ -161,7 +150,7 @@
</property>
</configuration>
</sub-workflow>
<ok to="JoinPromote"/>
<ok to="PromoteActionPayloadForDatasourceTable"/>
<error to="Kill"/>
</action>
@ -176,7 +165,7 @@
</property>
</configuration>
</sub-workflow>
<ok to="JoinPromote"/>
<ok to="PromoteActionPayloadForOrganizationTable"/>
<error to="Kill"/>
</action>
@ -191,7 +180,7 @@
</property>
</configuration>
</sub-workflow>
<ok to="JoinPromote"/>
<ok to="PromoteActionPayloadForOtherResearchProductTable"/>
<error to="Kill"/>
</action>
@ -206,7 +195,7 @@
</property>
</configuration>
</sub-workflow>
<ok to="JoinPromote"/>
<ok to="PromoteActionPayloadForProjectTable"/>
<error to="Kill"/>
</action>
@ -221,7 +210,7 @@
</property>
</configuration>
</sub-workflow>
<ok to="JoinPromote"/>
<ok to="PromoteActionPayloadForPublicationTable"/>
<error to="Kill"/>
</action>
@ -236,7 +225,7 @@
</property>
</configuration>
</sub-workflow>
<ok to="JoinPromote"/>
<ok to="PromoteActionPayloadForRelationTable"/>
<error to="Kill"/>
</action>
@ -251,7 +240,7 @@
</property>
</configuration>
</sub-workflow>
<ok to="JoinPromote"/>
<ok to="PromoteActionPayloadForSoftwareTable"/>
<error to="Kill"/>
</action>
@ -266,11 +255,9 @@
</property>
</configuration>
</sub-workflow>
<ok to="JoinPromote"/>
<ok to="End"/>
<error to="Kill"/>
</action>
<join name="JoinPromote" to="End"/>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,129 @@
<workflow-app name="promote_action_payload_for_person_table" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>activePromotePersonActionPayload</name>
<description>when true will promote actions with eu.dnetlib.dhp.schema.oaf.Person payload</description>
</property>
<property>
<name>inputGraphRootPath</name>
<description>root location of input materialized graph</description>
</property>
<property>
<name>inputActionPayloadRootPath</name>
<description>root location of action payloads to promote</description>
</property>
<property>
<name>outputGraphRootPath</name>
<description>root location for output materialized graph</description>
</property>
<property>
<name>mergeAndGetStrategy</name>
<description>strategy for merging graph table objects with action payload instances, MERGE_FROM_AND_GET or SELECT_NEWER_AND_GET</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>oozie.action.sharelib.for.spark</name>
<value>${oozieActionShareLibForSpark2}</value>
</property>
</configuration>
</global>
<start to="DecisionPromotePersonActionPayload"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<decision name="DecisionPromotePersonActionPayload">
<switch>
<case to="PromotePersonActionPayloadForPersonTable">
${(activePromotePersonActionPayload eq "true") and
(fs:exists(concat(concat(concat(concat(wf:conf('nameNode'),'/'),wf:conf('inputActionPayloadRootPath')),'/'),'clazz=eu.dnetlib.dhp.schema.oaf.Person')) eq "true")}
</case>
<default to="SkipPromotePersonActionPayloadForPersonTable"/>
</switch>
</decision>
<action name="PromotePersonActionPayloadForPersonTable">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn-cluster</master>
<mode>cluster</mode>
<name>PromotePersonActionPayloadForPersonTable</name>
<class>eu.dnetlib.dhp.actionmanager.promote.PromoteActionPayloadForGraphTableJob</class>
<jar>dhp-actionmanager-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.executor.memoryOverhead=${sparkExecutorMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--inputGraphTablePath</arg><arg>${inputGraphRootPath}/person</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Person</arg>
<arg>--inputActionPayloadPath</arg><arg>${inputActionPayloadRootPath}/clazz=eu.dnetlib.dhp.schema.oaf.Person</arg>
<arg>--actionPayloadClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Person</arg>
<arg>--outputGraphTablePath</arg><arg>${outputGraphRootPath}/person</arg>
<arg>--mergeAndGetStrategy</arg><arg>${mergeAndGetStrategy}</arg>
<arg>--promoteActionStrategy</arg><arg>${promoteActionStrategy}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<action name="SkipPromotePersonActionPayloadForPersonTable">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<prepare>
<delete path="${outputGraphRootPath}/person"/>
</prepare>
<arg>-pb</arg>
<arg>${inputGraphRootPath}/person</arg>
<arg>${outputGraphRootPath}/person</arg>
</distcp>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -13,6 +13,8 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.Instance;
import eu.dnetlib.dhp.schema.oaf.Qualifier;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
import eu.dnetlib.dhp.schema.oaf.Subject;
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;

View File

@ -34,7 +34,7 @@ import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
import scala.Tuple2;
/**
* Creates action sets for Crossref affiliation relations inferred by BIP!
* Creates action sets for Crossref affiliation relations inferred by OpenAIRE
*/
public class PrepareAffiliationRelations implements Serializable {
@ -104,22 +104,22 @@ public class PrepareAffiliationRelations implements Serializable {
.listKeyValues(OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME);
JavaPairRDD<Text, Text> crossrefRelations = prepareAffiliationRelationsNewModel(
spark, crossrefInputPath, collectedfromOpenAIRE);
spark, crossrefInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + ":crossref");
JavaPairRDD<Text, Text> pubmedRelations = prepareAffiliationRelations(
spark, pubmedInputPath, collectedfromOpenAIRE);
spark, pubmedInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + ":pubmed");
JavaPairRDD<Text, Text> openAPCRelations = prepareAffiliationRelationsNewModel(
spark, openapcInputPath, collectedfromOpenAIRE);
spark, openapcInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + ":openapc");
JavaPairRDD<Text, Text> dataciteRelations = prepareAffiliationRelations(
spark, dataciteInputPath, collectedfromOpenAIRE);
JavaPairRDD<Text, Text> dataciteRelations = prepareAffiliationRelationsNewModel(
spark, dataciteInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + ":datacite");
JavaPairRDD<Text, Text> webCrawlRelations = prepareAffiliationRelations(
spark, webcrawlInputPath, collectedfromOpenAIRE);
JavaPairRDD<Text, Text> webCrawlRelations = prepareAffiliationRelationsNewModel(
spark, webcrawlInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + ":rawaff");
JavaPairRDD<Text, Text> publisherRelations = prepareAffiliationRelationFromPublisher(
spark, publisherlInputPath, collectedfromOpenAIRE);
JavaPairRDD<Text, Text> publisherRelations = prepareAffiliationRelationFromPublisherNewModel(
spark, publisherlInputPath, collectedfromOpenAIRE, BIP_INFERENCE_PROVENANCE + ":webcrawl");
crossrefRelations
.union(pubmedRelations)
@ -133,7 +133,8 @@ public class PrepareAffiliationRelations implements Serializable {
private static JavaPairRDD<Text, Text> prepareAffiliationRelationFromPublisherNewModel(SparkSession spark,
String inputPath,
List<KeyValue> collectedfrom) {
List<KeyValue> collectedfrom,
String dataprovenance) {
Dataset<Row> df = spark
.read()
@ -142,12 +143,13 @@ public class PrepareAffiliationRelations implements Serializable {
.json(inputPath)
.where("DOI is not null");
return getTextTextJavaPairRDD(collectedfrom, df.selectExpr("DOI", "Organizations as Matchings"));
return getTextTextJavaPairRDDNew(
collectedfrom, df.selectExpr("DOI", "Organizations as Matchings"), dataprovenance);
}
private static JavaPairRDD<Text, Text> prepareAffiliationRelationFromPublisher(SparkSession spark, String inputPath,
List<KeyValue> collectedfrom) {
List<KeyValue> collectedfrom, String dataprovenance) {
Dataset<Row> df = spark
.read()
@ -155,13 +157,14 @@ public class PrepareAffiliationRelations implements Serializable {
.json(inputPath)
.where("DOI is not null");
return getTextTextJavaPairRDD(collectedfrom, df.selectExpr("DOI", "Organizations as Matchings"));
return getTextTextJavaPairRDD(
collectedfrom, df.selectExpr("DOI", "Organizations as Matchings"), dataprovenance);
}
private static <I extends Result> JavaPairRDD<Text, Text> prepareAffiliationRelations(SparkSession spark,
String inputPath,
List<KeyValue> collectedfrom) {
List<KeyValue> collectedfrom, String dataprovenance) {
// load and parse affiliation relations from HDFS
Dataset<Row> df = spark
@ -170,12 +173,12 @@ public class PrepareAffiliationRelations implements Serializable {
.json(inputPath)
.where("DOI is not null");
return getTextTextJavaPairRDD(collectedfrom, df);
return getTextTextJavaPairRDD(collectedfrom, df, dataprovenance);
}
private static <I extends Result> JavaPairRDD<Text, Text> prepareAffiliationRelationsNewModel(SparkSession spark,
String inputPath,
List<KeyValue> collectedfrom) {
List<KeyValue> collectedfrom, String dataprovenance) {
// load and parse affiliation relations from HDFS
Dataset<Row> df = spark
.read()
@ -184,10 +187,11 @@ public class PrepareAffiliationRelations implements Serializable {
.json(inputPath)
.where("DOI is not null");
return getTextTextJavaPairRDDNew(collectedfrom, df);
return getTextTextJavaPairRDDNew(collectedfrom, df, dataprovenance);
}
private static JavaPairRDD<Text, Text> getTextTextJavaPairRDD(List<KeyValue> collectedfrom, Dataset<Row> df) {
private static JavaPairRDD<Text, Text> getTextTextJavaPairRDD(List<KeyValue> collectedfrom, Dataset<Row> df,
String dataprovenance) {
// unroll nested arrays
df = df
.withColumn("matching", functions.explode(new Column("Matchings")))
@ -219,7 +223,7 @@ public class PrepareAffiliationRelations implements Serializable {
DataInfo dataInfo = OafMapperUtils
.dataInfo(
false,
BIP_INFERENCE_PROVENANCE,
dataprovenance,
true,
false,
qualifier,
@ -235,7 +239,8 @@ public class PrepareAffiliationRelations implements Serializable {
new Text(OBJECT_MAPPER.writeValueAsString(aa))));
}
private static JavaPairRDD<Text, Text> getTextTextJavaPairRDDNew(List<KeyValue> collectedfrom, Dataset<Row> df) {
private static JavaPairRDD<Text, Text> getTextTextJavaPairRDDNew(List<KeyValue> collectedfrom, Dataset<Row> df,
String dataprovenance) {
// unroll nested arrays
df = df
.withColumn("matching", functions.explode(new Column("Matchings")))
@ -276,7 +281,7 @@ public class PrepareAffiliationRelations implements Serializable {
DataInfo dataInfo = OafMapperUtils
.dataInfo(
false,
BIP_INFERENCE_PROVENANCE,
dataprovenance,
true,
false,
qualifier,

View File

@ -2,22 +2,31 @@
package eu.dnetlib.dhp.actionmanager.personentity;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import static org.apache.spark.sql.functions.*;
import java.io.BufferedWriter;
import java.io.IOException;
import java.io.OutputStreamWriter;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.util.*;
import java.util.stream.Collectors;
import org.apache.commons.cli.ParseException;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.compress.BZip2Codec;
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.*;
import org.apache.spark.sql.*;
import org.apache.spark.sql.Dataset;
import org.jetbrains.annotations.NotNull;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -28,13 +37,14 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.collection.orcid.model.Author;
import eu.dnetlib.dhp.collection.orcid.model.Employment;
import eu.dnetlib.dhp.collection.orcid.model.Work;
import eu.dnetlib.dhp.common.DbClient;
import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.common.person.CoAuthorshipIterator;
import eu.dnetlib.dhp.common.person.Coauthors;
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.KeyValue;
import eu.dnetlib.dhp.schema.oaf.Person;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory;
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
import eu.dnetlib.dhp.schema.oaf.utils.PidCleaner;
@ -44,7 +54,7 @@ import scala.Tuple2;
public class ExtractPerson implements Serializable {
private static final Logger log = LoggerFactory.getLogger(ExtractPerson.class);
private static final String QUERY = "SELECT * FROM project_person WHERE pid_type = 'ORCID'";
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static final String OPENAIRE_PREFIX = "openaire____";
private static final String SEPARATOR = "::";
@ -58,9 +68,48 @@ public class ExtractPerson implements Serializable {
private static final String PMCID_PREFIX = "50|pmcid_______::";
private static final String ROR_PREFIX = "20|ror_________::";
private static final String PERSON_PREFIX = ModelSupport.getIdPrefix(Person.class) + "|orcid_______";
private static final String PERSON_PREFIX = ModelSupport.getIdPrefix(Person.class)
+ IdentifierFactory.ID_PREFIX_SEPARATOR + ModelConstants.ORCID + "_______";
private static final String PROJECT_ID_PREFIX = ModelSupport.getIdPrefix(Project.class)
+ IdentifierFactory.ID_PREFIX_SEPARATOR;
public static final String ORCID_AUTHORS_CLASSID = "sysimport:crosswalk:orcid";
public static final String ORCID_AUTHORS_CLASSNAME = "Imported from ORCID";
public static final String FUNDER_AUTHORS_CLASSID = "sysimport:crosswalk:funderdatabase";
public static final String FUNDER_AUTHORS_CLASSNAME = "Imported from Funder Database";
public static final String OPENAIRE_DATASOURCE_ID = "10|infrastruct_::f66f1bd369679b5b077dcdf006089556";
public static final String OPENAIRE_DATASOURCE_NAME = "OpenAIRE";
public static List<KeyValue> collectedfromOpenAIRE = OafMapperUtils
.listKeyValues(OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME);
public static final DataInfo ORCIDDATAINFO = OafMapperUtils
.dataInfo(
false,
null,
false,
false,
OafMapperUtils
.qualifier(
ORCID_AUTHORS_CLASSID,
ORCID_AUTHORS_CLASSNAME,
ModelConstants.DNET_PROVENANCE_ACTIONS,
ModelConstants.DNET_PROVENANCE_ACTIONS),
"0.91");
public static final DataInfo FUNDERDATAINFO = OafMapperUtils
.dataInfo(
false,
null,
false,
false,
OafMapperUtils
.qualifier(
FUNDER_AUTHORS_CLASSID,
FUNDER_AUTHORS_CLASSNAME,
ModelConstants.DNET_PROVENANCE_ACTIONS,
ModelConstants.DNET_PROVENANCE_ACTIONS),
"0.91");
public static void main(final String[] args) throws IOException, ParseException {
@ -91,19 +140,130 @@ public class ExtractPerson implements Serializable {
final String workingDir = parser.get("workingDir");
log.info("workingDir {}", workingDir);
final String dbUrl = parser.get("postgresUrl");
final String dbUser = parser.get("postgresUser");
final String dbPassword = parser.get("postgresPassword");
final String hdfsNameNode = parser.get("hdfsNameNode");
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
HdfsSupport.remove(outputPath, spark.sparkContext().hadoopConfiguration());
createActionSet(spark, inputPath, outputPath, workingDir);
extractInfoForActionSetFromORCID(spark, inputPath, workingDir);
extractInfoForActionSetFromProjects(
spark, inputPath, workingDir, dbUrl, dbUser, dbPassword, workingDir + "/project", hdfsNameNode);
createActionSet(spark, outputPath, workingDir);
});
}
private static void createActionSet(SparkSession spark, String inputPath, String outputPath, String workingDir) {
private static void extractInfoForActionSetFromProjects(SparkSession spark, String inputPath, String workingDir,
String dbUrl, String dbUser, String dbPassword, String hdfsPath, String hdfsNameNode) throws IOException {
Configuration conf = new Configuration();
conf.set("fs.defaultFS", hdfsNameNode);
FileSystem fileSystem = FileSystem.get(conf);
Path hdfsWritePath = new Path(hdfsPath);
FSDataOutputStream fos = fileSystem.create(hdfsWritePath);
try (DbClient dbClient = new DbClient(dbUrl, dbUser, dbPassword)) {
try (BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fos, StandardCharsets.UTF_8))) {
dbClient.processResults(QUERY, rs -> writeRelation(getRelationWithProject(rs), writer));
}
} catch (IOException e) {
throw new RuntimeException(e);
}
}
public static Relation getRelationWithProject(ResultSet rs) {
try {
return getProjectRelation(
rs.getString("project"), rs.getString("pid"),
rs.getString("role"));
} catch (final SQLException e) {
throw new RuntimeException(e);
}
}
private static Relation getProjectRelation(String project, String orcid, String role) {
String source = PERSON_PREFIX + "::" + IdentifierFactory.md5(orcid);
String target = PROJECT_ID_PREFIX + StringUtils.substringBefore(project, "::") + "::"
+ IdentifierFactory.md5(StringUtils.substringAfter(project, "::"));
List<KeyValue> properties = new ArrayList<>();
Relation relation = OafMapperUtils
.getRelation(
source, target, ModelConstants.PROJECT_PERSON_RELTYPE, ModelConstants.PROJECT_PERSON_SUBRELTYPE,
ModelConstants.PROJECT_PERSON_PARTICIPATES,
collectedfromOpenAIRE,
FUNDERDATAINFO,
null);
relation.setValidated(true);
if (StringUtils.isNotBlank(role)) {
KeyValue kv = new KeyValue();
kv.setKey("role");
kv.setValue(role);
properties.add(kv);
}
if (!properties.isEmpty())
relation.setProperties(properties);
return relation;
}
protected static void writeRelation(final Relation relation, BufferedWriter writer) {
try {
writer.write(OBJECT_MAPPER.writeValueAsString(relation));
writer.newLine();
} catch (final IOException e) {
throw new RuntimeException(e);
}
}
private static void createActionSet(SparkSession spark, String outputPath, String workingDir) {
Dataset<Person> people;
people = spark
.read()
.textFile(workingDir + "/people")
.map(
(MapFunction<String, Person>) value -> OBJECT_MAPPER
.readValue(value, Person.class),
Encoders.bean(Person.class));
people
.toJavaRDD()
.map(p -> new AtomicAction(p.getClass(), p))
.union(
getRelations(spark, workingDir + "/authorship").toJavaRDD().map(r -> new AtomicAction(r.getClass(), r)))
.union(
getRelations(spark, workingDir + "/coauthorship")
.toJavaRDD()
.map(r -> new AtomicAction(r.getClass(), r)))
.union(
getRelations(spark, workingDir + "/affiliation")
.toJavaRDD()
.map(r -> new AtomicAction(r.getClass(), r)))
.union(
getRelations(spark, workingDir + "/project")
.toJavaRDD()
.map(r -> new AtomicAction(r.getClass(), r)))
.mapToPair(
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
.saveAsHadoopFile(
outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, BZip2Codec.class);
}
private static void extractInfoForActionSetFromORCID(SparkSession spark, String inputPath, String workingDir) {
Dataset<Author> authors = spark
.read()
.parquet(inputPath + "Authors")
@ -129,18 +289,13 @@ public class ExtractPerson implements Serializable {
.parquet(inputPath + "Employments")
.as(Encoders.bean(Employment.class));
Dataset<Author> peopleToMap = authors
.joinWith(works, authors.col("orcid").equalTo(works.col("orcid")))
.map((MapFunction<Tuple2<Author, Work>, Author>) t2 -> t2._1(), Encoders.bean(Author.class))
.groupByKey((MapFunction<Author, String>) a -> a.getOrcid(), Encoders.STRING())
.mapGroups((MapGroupsFunction<String, Author, Author>) (k, it) -> it.next(), Encoders.bean(Author.class));
Dataset<Employment> employment = employmentDataset
.joinWith(peopleToMap, employmentDataset.col("orcid").equalTo(peopleToMap.col("orcid")))
.joinWith(authors, employmentDataset.col("orcid").equalTo(authors.col("orcid")))
.map((MapFunction<Tuple2<Employment, Author>, Employment>) t2 -> t2._1(), Encoders.bean(Employment.class));
Dataset<Person> people;
peopleToMap.map((MapFunction<Author, Person>) op -> {
// Mapping all the orcid profiles even if the profile has no visible works
authors.map((MapFunction<Author, Person>) op -> {
Person person = new Person();
person.setId(DHPUtils.generateIdentifier(op.getOrcid(), PERSON_PREFIX));
person
@ -190,9 +345,23 @@ public class ExtractPerson implements Serializable {
OafMapperUtils
.structuredProperty(
op.getOrcid(), ModelConstants.ORCID, ModelConstants.ORCID_CLASSNAME,
ModelConstants.DNET_PID_TYPES, ModelConstants.DNET_PID_TYPES, null));
ModelConstants.DNET_PID_TYPES, ModelConstants.DNET_PID_TYPES,
OafMapperUtils
.dataInfo(
false,
null,
false,
false,
OafMapperUtils
.qualifier(
ModelConstants.SYSIMPORT_CROSSWALK_ENTITYREGISTRY,
ModelConstants.SYSIMPORT_CROSSWALK_ENTITYREGISTRY,
ModelConstants.DNET_PID_TYPES,
ModelConstants.DNET_PID_TYPES),
"0.91")));
person.setDateofcollection(op.getLastModifiedDate());
person.setOriginalId(Arrays.asList(op.getOrcid()));
person.setDataInfo(ORCIDDATAINFO);
return person;
}, Encoders.bean(Person.class))
.write()
@ -246,34 +415,6 @@ public class ExtractPerson implements Serializable {
.option("compression", "gzip")
.mode(SaveMode.Overwrite)
.json(workingDir + "/affiliation");
people = spark
.read()
.textFile(workingDir + "/people")
.map(
(MapFunction<String, Person>) value -> OBJECT_MAPPER
.readValue(value, Person.class),
Encoders.bean(Person.class));
people.show(false);
people
.toJavaRDD()
.map(p -> new AtomicAction(p.getClass(), p))
.union(
getRelations(spark, workingDir + "/authorship").toJavaRDD().map(r -> new AtomicAction(r.getClass(), r)))
.union(
getRelations(spark, workingDir + "/coauthorship")
.toJavaRDD()
.map(r -> new AtomicAction(r.getClass(), r)))
.union(
getRelations(spark, workingDir + "/affiliation")
.toJavaRDD()
.map(r -> new AtomicAction(r.getClass(), r)))
.mapToPair(
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
.saveAsHadoopFile(
outputPath, Text.class, Text.class, SequenceFileOutputFormat.class, BZip2Codec.class);
}
private static Dataset<Relation> getRelations(SparkSession spark, String path) {
@ -307,15 +448,9 @@ public class ExtractPerson implements Serializable {
source, target, ModelConstants.ORG_PERSON_RELTYPE, ModelConstants.ORG_PERSON_SUBRELTYPE,
ModelConstants.ORG_PERSON_PARTICIPATES,
Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)),
OafMapperUtils
.dataInfo(
false, null, false, false,
OafMapperUtils
.qualifier(
ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, ModelConstants.DNET_PROVENANCE_ACTIONS,
ModelConstants.DNET_PROVENANCE_ACTIONS),
"0.91"),
ORCIDDATAINFO,
null);
relation.setValidated(true);
if (Optional.ofNullable(row.getStartDate()).isPresent() && StringUtils.isNotBlank(row.getStartDate())) {
KeyValue kv = new KeyValue();
@ -336,45 +471,6 @@ public class ExtractPerson implements Serializable {
}
private static Collection<? extends Relation> getCoAuthorshipRelations(String orcid1, String orcid2) {
String source = PERSON_PREFIX + "::" + IdentifierFactory.md5(orcid1);
String target = PERSON_PREFIX + "::" + IdentifierFactory.md5(orcid2);
return Arrays
.asList(
OafMapperUtils
.getRelation(
source, target, ModelConstants.PERSON_PERSON_RELTYPE,
ModelConstants.PERSON_PERSON_SUBRELTYPE,
ModelConstants.PERSON_PERSON_HASCOAUTHORED,
Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)),
OafMapperUtils
.dataInfo(
false, null, false, false,
OafMapperUtils
.qualifier(
ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME,
ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS),
"0.91"),
null),
OafMapperUtils
.getRelation(
target, source, ModelConstants.PERSON_PERSON_RELTYPE,
ModelConstants.PERSON_PERSON_SUBRELTYPE,
ModelConstants.PERSON_PERSON_HASCOAUTHORED,
Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)),
OafMapperUtils
.dataInfo(
false, null, false, false,
OafMapperUtils
.qualifier(
ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME,
ModelConstants.DNET_PROVENANCE_ACTIONS, ModelConstants.DNET_PROVENANCE_ACTIONS),
"0.91"),
null));
}
private static @NotNull Iterator<Relation> getAuthorshipRelationIterator(Work w) {
if (Optional.ofNullable(w.getPids()).isPresent())
@ -417,21 +513,15 @@ public class ExtractPerson implements Serializable {
default:
return null;
}
return OafMapperUtils
Relation relation = OafMapperUtils
.getRelation(
source, target, ModelConstants.RESULT_PERSON_RELTYPE,
ModelConstants.RESULT_PERSON_SUBRELTYPE,
ModelConstants.RESULT_PERSON_HASAUTHORED,
Arrays.asList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)),
OafMapperUtils
.dataInfo(
false, null, false, false,
OafMapperUtils
.qualifier(
ORCID_AUTHORS_CLASSID, ORCID_AUTHORS_CLASSNAME, ModelConstants.DNET_PROVENANCE_ACTIONS,
ModelConstants.DNET_PROVENANCE_ACTIONS),
"0.91"),
ORCIDDATAINFO,
null);
relation.setValidated(true);
return relation;
}
}

View File

@ -0,0 +1,203 @@
package eu.dnetlib.dhp.actionmanager.raid;
import static eu.dnetlib.dhp.actionmanager.personentity.ExtractPerson.OPENAIRE_DATASOURCE_ID;
import static eu.dnetlib.dhp.actionmanager.personentity.ExtractPerson.OPENAIRE_DATASOURCE_NAME;
import static eu.dnetlib.dhp.common.Constants.*;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import static eu.dnetlib.dhp.schema.common.ModelConstants.*;
import static eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils.*;
import java.util.*;
import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.SequenceFileOutputFormat;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
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.raid.model.RAiDEntity;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.schema.action.AtomicAction;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
import eu.dnetlib.dhp.utils.DHPUtils;
import scala.Tuple2;
public class GenerateRAiDActionSetJob {
private static final Logger log = LoggerFactory
.getLogger(eu.dnetlib.dhp.actionmanager.raid.GenerateRAiDActionSetJob.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static final List<KeyValue> RAID_COLLECTED_FROM = listKeyValues(
OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME);
private static final Qualifier RAID_QUALIFIER = qualifier(
"0049", "Research Activity Identifier", DNET_PUBLICATION_RESOURCE, DNET_PUBLICATION_RESOURCE);
private static final Qualifier RAID_INFERENCE_QUALIFIER = qualifier(
"raid:openaireinference", "Inferred by OpenAIRE", DNET_PROVENANCE_ACTIONS, DNET_PROVENANCE_ACTIONS);
private static final DataInfo RAID_DATA_INFO = dataInfo(
false, OPENAIRE_DATASOURCE_NAME, true, false, RAID_INFERENCE_QUALIFIER, "0.92");
public static void main(final String[] args) throws Exception {
final String jsonConfiguration = IOUtils
.toString(
eu.dnetlib.dhp.actionmanager.raid.GenerateRAiDActionSetJob.class
.getResourceAsStream("/eu/dnetlib/dhp/actionmanager/raid/action_set_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
final 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 SparkConf conf = new SparkConf();
runWithSparkSession(conf, isSparkSessionManaged, spark -> {
removeOutputDir(spark, outputPath);
processRAiDEntities(spark, inputPath, outputPath);
});
}
private static void removeOutputDir(final SparkSession spark, final String path) {
HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration());
}
static void processRAiDEntities(final SparkSession spark,
final String inputPath,
final String outputPath) {
readInputPath(spark, inputPath)
.map(GenerateRAiDActionSetJob::prepareRAiD)
.flatMap(List::iterator)
.mapToPair(
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
.saveAsHadoopFile(outputPath, Text.class, Text.class, SequenceFileOutputFormat.class);
}
protected static List<AtomicAction<? extends Oaf>> prepareRAiD(final RAiDEntity r) {
final Date now = new Date();
final OtherResearchProduct orp = new OtherResearchProduct();
final List<AtomicAction<? extends Oaf>> res = new ArrayList<>();
String raidId = calculateOpenaireId(r.getRaid());
orp.setId(raidId);
orp.setCollectedfrom(RAID_COLLECTED_FROM);
orp.setDataInfo(RAID_DATA_INFO);
orp
.setTitle(
Collections
.singletonList(
structuredProperty(
r.getTitle(),
qualifier("main title", "main title", DNET_DATACITE_TITLE, DNET_DATACITE_TITLE),
RAID_DATA_INFO)));
orp.setDescription(listFields(RAID_DATA_INFO, r.getSummary()));
Instance instance = new Instance();
instance.setInstancetype(RAID_QUALIFIER);
orp.setInstance(Collections.singletonList(instance));
orp
.setSubject(
r
.getSubjects()
.stream()
.map(
s -> subject(
s,
qualifier(
DNET_SUBJECT_KEYWORD, DNET_SUBJECT_KEYWORD, DNET_SUBJECT_TYPOLOGIES,
DNET_SUBJECT_TYPOLOGIES),
RAID_DATA_INFO))
.collect(Collectors.toList()));
orp
.setRelevantdate(
Arrays
.asList(
structuredProperty(
r.getEndDate(), qualifier(END_DATE, END_DATE, DNET_DATACITE_DATE, DNET_DATACITE_DATE),
RAID_DATA_INFO),
structuredProperty(
r.getStartDate(),
qualifier(START_DATE, START_DATE, DNET_DATACITE_DATE, DNET_DATACITE_DATE),
RAID_DATA_INFO)));
orp.setLastupdatetimestamp(now.getTime());
orp.setDateofacceptance(field(r.getStartDate(), RAID_DATA_INFO));
res.add(new AtomicAction<>(OtherResearchProduct.class, orp));
for (String resultId : r.getIds()) {
Relation rel1 = OafMapperUtils
.getRelation(
raidId,
resultId,
ModelConstants.RESULT_RESULT,
PART,
HAS_PART,
orp);
Relation rel2 = OafMapperUtils
.getRelation(
resultId,
raidId,
ModelConstants.RESULT_RESULT,
PART,
IS_PART_OF,
orp);
res.add(new AtomicAction<>(Relation.class, rel1));
res.add(new AtomicAction<>(Relation.class, rel2));
}
return res;
}
public static String calculateOpenaireId(final String raid) {
return String.format("50|%s::%s", RAID_NS_PREFIX, DHPUtils.md5(raid));
}
public static List<Author> createAuthors(final List<String> author) {
return author.stream().map(s -> {
Author a = new Author();
a.setFullname(s);
return a;
}).collect(Collectors.toList());
}
private static JavaRDD<RAiDEntity> readInputPath(
final SparkSession spark,
final String path) {
return spark
.read()
.json(path)
.as(Encoders.bean(RAiDEntity.class))
.toJavaRDD();
}
}

View File

@ -0,0 +1,5 @@
package eu.dnetlib.dhp.actionmanager.raid.model;
public class GenerateRAiDActionSetJob {
}

View File

@ -0,0 +1,106 @@
package eu.dnetlib.dhp.actionmanager.raid.model;
import java.io.Serializable;
import java.util.List;
public class RAiDEntity implements Serializable {
String raid;
List<String> authors;
String startDate;
String endDate;
List<String> subjects;
List<String> titles;
List<String> ids;
String title;
String summary;
public RAiDEntity() {
}
public RAiDEntity(String raid, List<String> authors, String startDate, String endDate, List<String> subjects,
List<String> titles, List<String> ids, String title, String summary) {
this.raid = raid;
this.authors = authors;
this.startDate = startDate;
this.endDate = endDate;
this.subjects = subjects;
this.titles = titles;
this.ids = ids;
this.title = title;
this.summary = summary;
}
public String getRaid() {
return raid;
}
public void setRaid(String raid) {
this.raid = raid;
}
public List<String> getAuthors() {
return authors;
}
public void setAuthors(List<String> authors) {
this.authors = authors;
}
public String getStartDate() {
return startDate;
}
public void setStartDate(String startDate) {
this.startDate = startDate;
}
public String getEndDate() {
return endDate;
}
public void setEndDate(String endDate) {
this.endDate = endDate;
}
public List<String> getSubjects() {
return subjects;
}
public void setSubjects(List<String> subjects) {
this.subjects = subjects;
}
public List<String> getTitles() {
return titles;
}
public void setTitles(List<String> titles) {
this.titles = titles;
}
public List<String> getIds() {
return ids;
}
public void setIds(List<String> ids) {
this.ids = ids;
}
public String getTitle() {
return title;
}
public void setTitle(String title) {
this.title = title;
}
public String getSummary() {
return summary;
}
public void setSummary(String summary) {
this.summary = summary;
}
}

View File

@ -44,13 +44,7 @@ import eu.dnetlib.dhp.common.Constants;
import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.schema.action.AtomicAction;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.Field;
import eu.dnetlib.dhp.schema.oaf.KeyValue;
import eu.dnetlib.dhp.schema.oaf.Oaf;
import eu.dnetlib.dhp.schema.oaf.Organization;
import eu.dnetlib.dhp.schema.oaf.Qualifier;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.dhp.utils.DHPUtils;
import scala.Tuple2;

View File

@ -28,6 +28,7 @@ import eu.dnetlib.dhp.collection.plugin.mongodb.MongoDbDumpCollectorPlugin;
import eu.dnetlib.dhp.collection.plugin.oai.OaiCollectorPlugin;
import eu.dnetlib.dhp.collection.plugin.osf.OsfPreprintsCollectorPlugin;
import eu.dnetlib.dhp.collection.plugin.rest.RestCollectorPlugin;
import eu.dnetlib.dhp.collection.plugin.zenodo.CollectZenodoDumpCollectorPlugin;
import eu.dnetlib.dhp.common.aggregation.AggregatorReport;
import eu.dnetlib.dhp.common.collection.CollectorException;
import eu.dnetlib.dhp.common.collection.HttpClientParams;
@ -129,6 +130,8 @@ public class CollectorWorker extends ReportingJob {
return new Gtr2PublicationsCollectorPlugin(this.clientParams);
case osfPreprints:
return new OsfPreprintsCollectorPlugin(this.clientParams);
case zenodoDump:
return new CollectZenodoDumpCollectorPlugin();
case other:
final CollectorPlugin.NAME.OTHER_NAME plugin = Optional
.ofNullable(this.api.getParams().get("other_plugin_type"))

View File

@ -154,7 +154,6 @@ public class ORCIDExtractor extends Thread {
extractedItem++;
if (extractedItem % 100000 == 0) {
log.info("Thread {}: Extracted {} items", id, extractedItem);
break;
}
}
}

View File

@ -11,7 +11,7 @@ public interface CollectorPlugin {
enum NAME {
oai, other, rest_json2xml, file, fileGzip, baseDump, gtr2Publications, osfPreprints;
oai, other, rest_json2xml, file, fileGzip, baseDump, gtr2Publications, osfPreprints, zenodoDump;
public enum OTHER_NAME {
mdstore_mongodb_dump, mdstore_mongodb

View File

@ -1,6 +1,9 @@
package eu.dnetlib.dhp.collection.plugin.gtr2;
import java.nio.charset.StandardCharsets;
import java.time.LocalDate;
import java.time.format.DateTimeFormatter;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
@ -16,9 +19,6 @@ import org.dom4j.Document;
import org.dom4j.DocumentException;
import org.dom4j.DocumentHelper;
import org.dom4j.Element;
import org.joda.time.DateTime;
import org.joda.time.format.DateTimeFormat;
import org.joda.time.format.DateTimeFormatter;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -33,7 +33,7 @@ public class Gtr2PublicationsIterator implements Iterator<String> {
private static final Logger log = LoggerFactory.getLogger(Gtr2PublicationsIterator.class);
private final HttpConnector2 connector;
private static final DateTimeFormatter simpleDateTimeFormatter = DateTimeFormat.forPattern("yyyy-MM-dd");
private static final DateTimeFormatter simpleDateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd");
private static final int MAX_ATTEMPTS = 10;
@ -41,7 +41,7 @@ public class Gtr2PublicationsIterator implements Iterator<String> {
private int currPage;
private int endPage;
private boolean incremental = false;
private DateTime fromDate;
private LocalDate fromDate;
private final Map<String, String> cache = new HashMap<>();
@ -188,28 +188,28 @@ public class Gtr2PublicationsIterator implements Iterator<String> {
private Document loadURL(final String cleanUrl, final int attempt) {
try {
log.debug(" * Downloading Url: " + cleanUrl);
final byte[] bytes = this.connector.getInputSource(cleanUrl).getBytes("UTF-8");
log.debug(" * Downloading Url: {}", cleanUrl);
final byte[] bytes = this.connector.getInputSource(cleanUrl).getBytes(StandardCharsets.UTF_8);
return DocumentHelper.parseText(new String(bytes));
} catch (final Throwable e) {
log.error("Error dowloading url: " + cleanUrl + ", attempt = " + attempt, e);
log.error("Error dowloading url: {}, attempt = {}", cleanUrl, attempt, e);
if (attempt >= MAX_ATTEMPTS) {
throw new RuntimeException("Error dowloading url: " + cleanUrl, e);
throw new RuntimeException("Error downloading url: " + cleanUrl, e);
}
try {
Thread.sleep(60000); // I wait for a minute
} catch (final InterruptedException e1) {
throw new RuntimeException("Error dowloading url: " + cleanUrl, e);
throw new RuntimeException("Error downloading url: " + cleanUrl, e);
}
return loadURL(cleanUrl, attempt + 1);
}
}
private DateTime parseDate(final String s) {
return DateTime.parse(s.contains("T") ? s.substring(0, s.indexOf("T")) : s, simpleDateTimeFormatter);
private LocalDate parseDate(final String s) {
return LocalDate.parse(s.contains("T") ? s.substring(0, s.indexOf("T")) : s, simpleDateTimeFormatter);
}
private boolean isAfter(final String d, final DateTime fromDate) {
private boolean isAfter(final String d, final LocalDate fromDate) {
return StringUtils.isNotBlank(d) && parseDate(d).isAfter(fromDate);
}
}

View File

@ -6,7 +6,7 @@ import java.util.Queue;
import java.util.concurrent.PriorityBlockingQueue;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.math.NumberUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.http.Header;
@ -27,27 +27,27 @@ public class ResearchFiIterator implements Iterator<String> {
private final String baseUrl;
private final String authToken;
private int currPage;
private int nPages;
private String nextUrl;
private int nCalls = 0;
private final Queue<String> queue = new PriorityBlockingQueue<>();
public ResearchFiIterator(final String baseUrl, final String authToken) {
this.baseUrl = baseUrl;
this.authToken = authToken;
this.currPage = 0;
this.nPages = 0;
this.nextUrl = null;
}
private void verifyStarted() {
if (this.currPage == 0) {
try {
nextCall();
if (this.nCalls == 0) {
this.nextUrl = invokeUrl(this.baseUrl);
}
} catch (final CollectorException e) {
throw new IllegalStateException(e);
}
}
}
@Override
public boolean hasNext() {
@ -62,9 +62,9 @@ public class ResearchFiIterator implements Iterator<String> {
synchronized (this.queue) {
verifyStarted();
final String res = this.queue.poll();
while (this.queue.isEmpty() && (this.currPage < this.nPages)) {
while (this.queue.isEmpty() && StringUtils.isNotBlank(this.nextUrl)) {
try {
nextCall();
this.nextUrl = invokeUrl(this.nextUrl);
} catch (final CollectorException e) {
throw new IllegalStateException(e);
}
@ -73,18 +73,11 @@ public class ResearchFiIterator implements Iterator<String> {
}
}
private void nextCall() throws CollectorException {
private String invokeUrl(final String url) throws CollectorException {
this.currPage += 1;
this.nCalls += 1;
String next = null;
final String url;
if (!this.baseUrl.contains("?")) {
url = String.format("%s?PageNumber=%d&PageSize=%d", this.baseUrl, this.currPage, PAGE_SIZE);
} else if (!this.baseUrl.contains("PageSize=")) {
url = String.format("%s&PageNumber=%d&PageSize=%d", this.baseUrl, this.currPage, PAGE_SIZE);
} else {
url = String.format("%s&PageNumber=%d", this.baseUrl, this.currPage);
}
log.info("Calling url: " + url);
try (final CloseableHttpClient client = HttpClients.createDefault()) {
@ -94,11 +87,15 @@ public class ResearchFiIterator implements Iterator<String> {
try (final CloseableHttpResponse response = client.execute(req)) {
for (final Header header : response.getAllHeaders()) {
log.debug("HEADER: " + header.getName() + " = " + header.getValue());
if ("x-page-count".equals(header.getName())) {
final int totalPages = NumberUtils.toInt(header.getValue());
if (this.nPages != totalPages) {
this.nPages = NumberUtils.toInt(header.getValue());
log.info("Total pages: " + totalPages);
if ("link".equals(header.getName())) {
final String s = StringUtils.substringBetween(header.getValue(), "<", ">");
final String token = StringUtils
.substringBefore(StringUtils.substringAfter(s, "NextPageToken="), "&");
if (this.baseUrl.contains("?")) {
next = this.baseUrl + "&NextPageToken=" + token;
} else {
next = this.baseUrl + "?NextPageToken=" + token;
}
}
}
@ -108,6 +105,9 @@ public class ResearchFiIterator implements Iterator<String> {
jsonArray.forEach(obj -> this.queue.add(JsonUtils.convertToXML(obj.toString())));
}
return next;
} catch (final Throwable e) {
log.warn("Error calling url: " + url, e);
throw new CollectorException("Error calling url: " + url, e);

View File

@ -0,0 +1,96 @@
package eu.dnetlib.dhp.collection.plugin.zenodo;
import static eu.dnetlib.dhp.utils.DHPUtils.getHadoopConfiguration;
import java.io.IOException;
import java.io.InputStream;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.CompressionCodecFactory;
import org.apache.http.client.config.RequestConfig;
import org.apache.http.client.methods.CloseableHttpResponse;
import org.apache.http.client.methods.HttpGet;
import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.http.impl.client.HttpClientBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.collection.ApiDescriptor;
import eu.dnetlib.dhp.collection.plugin.CollectorPlugin;
import eu.dnetlib.dhp.common.aggregation.AggregatorReport;
import eu.dnetlib.dhp.common.collection.CollectorException;
public class CollectZenodoDumpCollectorPlugin implements CollectorPlugin {
final private Logger log = LoggerFactory.getLogger(getClass());
private void downloadItem(final String name, final String itemURL, final String basePath,
final FileSystem fileSystem) {
try {
final Path hdfsWritePath = new Path(String.format("%s/%s", basePath, name));
final FSDataOutputStream fsDataOutputStream = fileSystem.create(hdfsWritePath, true);
final HttpGet request = new HttpGet(itemURL);
final int timeout = 60; // seconds
final RequestConfig config = RequestConfig
.custom()
.setConnectTimeout(timeout * 1000)
.setConnectionRequestTimeout(timeout * 1000)
.setSocketTimeout(timeout * 1000)
.build();
log.info("Downloading url {} into {}", itemURL, hdfsWritePath.getName());
try (CloseableHttpClient client = HttpClientBuilder.create().setDefaultRequestConfig(config).build();
CloseableHttpResponse response = client.execute(request)) {
int responseCode = response.getStatusLine().getStatusCode();
log.info("Response code is {}", responseCode);
if (responseCode >= 200 && responseCode < 400) {
IOUtils.copy(response.getEntity().getContent(), fsDataOutputStream);
}
} catch (Throwable eu) {
throw new RuntimeException(eu);
}
} catch (Throwable e) {
throw new RuntimeException(e);
}
}
@Override
public Stream<String> collect(ApiDescriptor api, AggregatorReport report) throws CollectorException {
try {
final String zenodoURL = api.getBaseUrl();
final String hdfsURI = api.getParams().get("hdfsURI");
final FileSystem fileSystem = FileSystem.get(getHadoopConfiguration(hdfsURI));
downloadItem("zenodoDump.tar.gz", zenodoURL, "/tmp", fileSystem);
CompressionCodecFactory factory = new CompressionCodecFactory(fileSystem.getConf());
Path sourcePath = new Path("/tmp/zenodoDump.tar.gz");
CompressionCodec codec = factory.getCodec(sourcePath);
InputStream gzipInputStream = null;
try {
gzipInputStream = codec.createInputStream(fileSystem.open(sourcePath));
return iterateTar(gzipInputStream);
} catch (IOException e) {
throw new CollectorException(e);
} finally {
log.info("Closing gzip stream");
org.apache.hadoop.io.IOUtils.closeStream(gzipInputStream);
}
} catch (Exception e) {
throw new CollectorException(e);
}
}
private Stream<String> iterateTar(InputStream gzipInputStream) throws Exception {
Iterable<String> iterable = () -> new ZenodoTarIterator(gzipInputStream);
return StreamSupport.stream(iterable.spliterator(), false);
}
}

View File

@ -0,0 +1,59 @@
package eu.dnetlib.dhp.collection.plugin.zenodo;
import java.io.Closeable;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.util.Iterator;
import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
import org.apache.commons.io.IOUtils;
public class ZenodoTarIterator implements Iterator<String>, Closeable {
private final InputStream gzipInputStream;
private final StringBuilder currentItem = new StringBuilder();
private TarArchiveInputStream tais;
private boolean hasNext;
public ZenodoTarIterator(InputStream gzipInputStream) {
this.gzipInputStream = gzipInputStream;
tais = new TarArchiveInputStream(gzipInputStream);
hasNext = getNextItem();
}
private boolean getNextItem() {
try {
TarArchiveEntry entry;
while ((entry = tais.getNextTarEntry()) != null) {
if (entry.isFile()) {
currentItem.setLength(0);
currentItem.append(IOUtils.toString(new InputStreamReader(tais)));
return true;
}
}
return false;
} catch (Throwable e) {
throw new RuntimeException(e);
}
}
@Override
public boolean hasNext() {
return hasNext;
}
@Override
public String next() {
final String data = currentItem.toString();
hasNext = getNextItem();
return data;
}
@Override
public void close() throws IOException {
gzipInputStream.close();
}
}

View File

@ -0,0 +1,39 @@
package eu.dnetlib.dhp.sx.bio.pubmed;
/**
* The type Pubmed Affiliation.
*
* @author Sandro La Bruzzo
*/
public class PMAffiliation {
private String name;
private PMIdentifier identifier;
public PMAffiliation() {
}
public PMAffiliation(String name, PMIdentifier identifier) {
this.name = name;
this.identifier = identifier;
}
public String getName() {
return name;
}
public void setName(String name) {
this.name = name;
}
public PMIdentifier getIdentifier() {
return identifier;
}
public void setIdentifier(PMIdentifier identifier) {
this.identifier = identifier;
}
}

View File

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

View File

@ -12,6 +12,8 @@ public class PMAuthor implements Serializable {
private String lastName;
private String foreName;
private PMIdentifier identifier;
private PMAffiliation affiliation;
/**
* Gets last name.
@ -59,4 +61,40 @@ public class PMAuthor implements Serializable {
.format("%s, %s", this.foreName != null ? this.foreName : "", this.lastName != null ? this.lastName : "");
}
/**
* Gets identifier.
*
* @return the identifier
*/
public PMIdentifier getIdentifier() {
return identifier;
}
/**
* Sets identifier.
*
* @param identifier the identifier
*/
public void setIdentifier(PMIdentifier identifier) {
this.identifier = identifier;
}
/**
* Gets affiliation.
*
* @return the affiliation
*/
public PMAffiliation getAffiliation() {
return affiliation;
}
/**
* Sets affiliation.
*
* @param affiliation the affiliation
*/
public void setAffiliation(PMAffiliation affiliation) {
this.affiliation = affiliation;
}
}

View File

@ -0,0 +1,53 @@
package eu.dnetlib.dhp.sx.bio.pubmed;
public class PMIdentifier {
private String pid;
private String type;
public PMIdentifier(String pid, String type) {
this.pid = cleanPid(pid);
this.type = type;
}
public PMIdentifier() {
}
private String cleanPid(String pid) {
if (pid == null) {
return null;
}
// clean ORCID ID in the form 0000000163025705 to 0000-0001-6302-5705
if (pid.matches("[0-9]{15}[0-9X]")) {
return pid.replaceAll("(.{4})(.{4})(.{4})(.{4})", "$1-$2-$3-$4");
}
// clean ORCID in the form http://orcid.org/0000-0001-8567-3543 to 0000-0001-8567-3543
if (pid.matches("http://orcid.org/[0-9]{4}-[0-9]{4}-[0-9]{4}-[0-9]{4}")) {
return pid.replaceAll("http://orcid.org/", "");
}
return pid;
}
public String getPid() {
return pid;
}
public PMIdentifier setPid(String pid) {
this.pid = cleanPid(pid);
return this;
}
public String getType() {
return type;
}
public PMIdentifier setType(String type) {
this.type = type;
return this;
}
}

View File

@ -31,9 +31,11 @@ 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/crossref-data.json
pubmedInputPath=/data/bip-affiliations/pubmed-data.json
openapcInputPath=/data/bip-affiliations/openapc-data.json
dataciteInputPath=/data/bip-affiliations/datacite-data.json
crossrefInputPath=/data/openaire-affiliations/crossref-data.json
pubmedInputPath=/data/openaire-affiliations/pubmed-data-v4.json
openapcInputPath=/data/openaire-affiliations/openapc-data.json
dataciteInputPath=/data/openaire-affiliations/datacite-data.json
webCrawlInputPath=/data/openaire-affiliations/webCrawl
publisherInputPath=/data/openaire-affiliations/publishers
outputPath=/tmp/crossref-affiliations-output-v5
outputPath=/tmp/affRoAS

View File

@ -1,4 +1,4 @@
<workflow-app name="BipAffiliations" xmlns="uri:oozie:workflow:0.5">
<workflow-app name="OpenAIREAffiliations" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
@ -21,6 +21,10 @@
<name>webCrawlInputPath</name>
<description>the path where to find the inferred affiliation relations from webCrawl</description>
</property>
<property>
<name>publisherInputPath</name>
<description>the path where to find the inferred affiliation relations from publisher websites</description>
</property>
<property>
<name>outputPath</name>
<description>the path where to store the actionset</description>
@ -99,7 +103,7 @@
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Produces the atomic action with the inferred by BIP! affiliation relations (from Crossref and Pubmed)</name>
<name>Produces the atomic action with the inferred by OpenAIRE affiliation relations</name>
<class>eu.dnetlib.dhp.actionmanager.bipaffiliations.PrepareAffiliationRelations</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
@ -117,6 +121,7 @@
<arg>--openapcInputPath</arg><arg>${openapcInputPath}</arg>
<arg>--dataciteInputPath</arg><arg>${dataciteInputPath}</arg>
<arg>--webCrawlInputPath</arg><arg>${webCrawlInputPath}</arg>
<arg>--publisherInputPath</arg><arg>${publisherInputPath}</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
</spark>
<ok to="End"/>

View File

@ -21,5 +21,30 @@
"paramLongName": "workingDir",
"paramDescription": "the hdfs name node",
"paramRequired": false
},
{
"paramName": "pu",
"paramLongName": "postgresUrl",
"paramDescription": "the hdfs name node",
"paramRequired": false
},
{
"paramName": "ps",
"paramLongName": "postgresUser",
"paramDescription": "the hdfs name node",
"paramRequired": false
},
{
"paramName": "pp",
"paramLongName": "postgresPassword",
"paramDescription": "the hdfs name node",
"paramRequired": false
},{
"paramName": "nn",
"paramLongName": "hdfsNameNode",
"paramDescription": "the hdfs name node",
"paramRequired": false
}
]

View File

@ -1,2 +1,5 @@
inputPath=/data/orcid_2023/tables/
outputPath=/user/miriam.baglioni/peopleAS
postgresUrl=jdbc:postgresql://beta.services.openaire.eu:5432/dnet_openaireplus
postgresUser=dnet
postgresPassword=dnetPwd

View File

@ -9,6 +9,18 @@
<name>outputPath</name>
<description>the path where to store the actionset</description>
</property>
<property>
<name>postgresUrl</name>
<description>the path where to store the actionset</description>
</property>
<property>
<name>postgresUser</name>
<description>the path where to store the actionset</description>
</property>
<property>
<name>postgresPassword</name>
<description>the path where to store the actionset</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
@ -102,6 +114,10 @@
<arg>--inputPath</arg><arg>${inputPath}</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
<arg>--workingDir</arg><arg>${workingDir}</arg>
<arg>--hdfsNameNode</arg><arg>${nameNode}</arg>
<arg>--postgresUrl</arg><arg>${postgresUrl}</arg>
<arg>--postgresUser</arg><arg>${postgresUser}</arg>
<arg>--postgresPassword</arg><arg>${postgresPassword}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>

View File

@ -0,0 +1,14 @@
[
{
"paramName": "i",
"paramLongName": "inputPath",
"paramDescription": "the path of the input json",
"paramRequired": true
},
{
"paramName": "o",
"paramLongName": "outputPath",
"paramDescription": "the path of the new ActionSet",
"paramRequired": true
}
]

View File

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

View File

@ -0,0 +1,53 @@
<workflow-app name="Update_RAiD_action_set" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>raidJsonInputPath</name>
<description>the path of the json</description>
</property>
<property>
<name>raidActionSetPath</name>
<description>path where to store the action set</description>
</property>
</parameters>
<start to="deleteoutputpath"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="deleteoutputpath">
<fs>
<delete path='${raidActionSetPath}'/>
<mkdir path='${raidActionSetPath}'/>
</fs>
<ok to="processRAiDFile"/>
<error to="Kill"/>
</action>
<action name="processRAiDFile">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>ProcessRAiDFile</name>
<class>eu.dnetlib.dhp.actionmanager.raid.GenerateRAiDActionSetJob</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
--executor-memory=${sparkExecutorMemory}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>--inputPath</arg><arg>${raidJsonInputPath}</arg>
<arg>--outputPath</arg><arg>${raidActionSetPath}</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -24,7 +24,7 @@
<decision name="resume_from">
<switch>
<case to="download">${wf:conf('resumeFrom') eq 'DownloadDump'}</case>
<case to="reset_workingDir">${wf:conf('resumeFrom') eq 'DownloadDump'}</case>
<default to="create_actionset"/> <!-- first action to be done when downloadDump is to be performed -->
</switch>
</decision>
@ -33,6 +33,14 @@
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="reset_workingDir">
<fs>
<delete path="${workingDir}"/>
<mkdir path="${workingDir}"/>
</fs>
<ok to="download"/>
<error to="Kill"/>
</action>
<action name="download">
<shell xmlns="uri:oozie:shell-action:0.2">
<job-tracker>${jobTracker}</job-tracker>

View File

@ -1,8 +1,7 @@
[
{"paramName":"mt", "paramLongName":"master", "paramDescription": "should be local or yarn", "paramRequired": true},
{"paramName":"i", "paramLongName":"isLookupUrl", "paramDescription": "isLookupUrl", "paramRequired": true},
{"paramName":"w", "paramLongName":"workingPath", "paramDescription": "the path of the sequencial file to read", "paramRequired": true},
{"paramName":"mo", "paramLongName":"mdstoreOutputVersion", "paramDescription": "the oaf path ", "paramRequired": true},
{"paramName":"s", "paramLongName":"skipUpdate", "paramDescription": "skip update ", "paramRequired": false},
{"paramName":"h", "paramLongName":"hdfsServerUri", "paramDescription": "the working path ", "paramRequired": true}
{"paramName":"s", "paramLongName":"sourcePath", "paramDescription": "the baseline path", "paramRequired": true},
{"paramName":"mo", "paramLongName":"mdstoreOutputVersion", "paramDescription": "the mdstore path to save", "paramRequired": true}
]

View File

@ -1,4 +1,4 @@
<workflow-app name="Download_Transform_Pubmed_Workflow" xmlns="uri:oozie:workflow:0.5">
<workflow-app name="Transform_Pubmed_Workflow" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>baselineWorkingPath</name>
@ -16,11 +16,6 @@
<name>mdStoreManagerURI</name>
<description>the path of the cleaned mdstore</description>
</property>
<property>
<name>skipUpdate</name>
<value>false</value>
<description>The request block size</description>
</property>
</parameters>
<start to="StartTransaction"/>
@ -44,16 +39,16 @@
<arg>--mdStoreManagerURI</arg><arg>${mdStoreManagerURI}</arg>
<capture-output/>
</java>
<ok to="ConvertDataset"/>
<ok to="TransformPubMed"/>
<error to="RollBack"/>
</action>
<action name="ConvertDataset">
<action name="TransformPubMed">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Convert Baseline to OAF Dataset</name>
<class>eu.dnetlib.dhp.sx.bio.ebi.SparkCreateBaselineDataFrame</class>
<name>Convert Baseline Pubmed to OAF Dataset</name>
<class>eu.dnetlib.dhp.sx.bio.ebi.SparkCreatePubmedDump</class>
<jar>dhp-aggregation-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
@ -65,12 +60,10 @@
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
</spark-opts>
<arg>--workingPath</arg><arg>${baselineWorkingPath}</arg>
<arg>--sourcePath</arg><arg>${baselineWorkingPath}</arg>
<arg>--mdstoreOutputVersion</arg><arg>${wf:actionData('StartTransaction')['mdStoreVersion']}</arg>
<arg>--master</arg><arg>yarn</arg>
<arg>--isLookupUrl</arg><arg>${isLookupUrl}</arg>
<arg>--hdfsServerUri</arg><arg>${nameNode}</arg>
<arg>--skipUpdate</arg><arg>${skipUpdate}</arg>
</spark>
<ok to="CommitVersion"/>
<error to="RollBack"/>

View File

@ -37,7 +37,7 @@ case class mappingAuthor(
family: Option[String],
sequence: Option[String],
ORCID: Option[String],
affiliation: Option[mappingAffiliation]
affiliation: Option[List[mappingAffiliation]]
) {}
case class funderInfo(id: String, uri: String, name: String, synonym: List[String]) {}
@ -457,15 +457,14 @@ case object Crossref2Oaf {
}
//Mapping Author
val authorList: List[mappingAuthor] =
(json \ "author").extract[List[mappingAuthor]].filter(a => a.family.isDefined)
val authorList: List[mappingAuthor] = (json \ "author").extract[List[mappingAuthor]].filter(a => a.family.isDefined)
val sorted_list = authorList.sortWith((a: mappingAuthor, b: mappingAuthor) =>
a.sequence.isDefined && a.sequence.get.equalsIgnoreCase("first")
)
result.setAuthor(sorted_list.zipWithIndex.map { case (a, index) =>
generateAuhtor(a.given.orNull, a.family.get, a.ORCID.orNull, index)
generateAuthor(a.given.orNull, a.family.get, a.ORCID.orNull, index, a.affiliation)
}.asJava)
// Mapping instance
@ -504,19 +503,6 @@ case object Crossref2Oaf {
)
}
val is_review = json \ "relation" \ "is-review-of" \ "id"
if (is_review != JNothing) {
instance.setInstancetype(
OafMapperUtils.qualifier(
"0015",
"peerReviewed",
ModelConstants.DNET_REVIEW_LEVELS,
ModelConstants.DNET_REVIEW_LEVELS
)
)
}
if (doi.startsWith("10.3410") || doi.startsWith("10.12703"))
instance.setHostedby(
OafMapperUtils.keyValue(OafMapperUtils.createOpenaireId(10, "openaire____::H1Connect", true), "H1Connect")
@ -574,12 +560,23 @@ case object Crossref2Oaf {
s"50|doiboost____|$id"
}
def generateAuhtor(given: String, family: String, orcid: String, index: Int): Author = {
private def generateAuthor(
given: String,
family: String,
orcid: String,
index: Int,
affiliation: Option[List[mappingAffiliation]]
): Author = {
val a = new Author
a.setName(given)
a.setSurname(family)
a.setFullname(s"$given $family")
a.setRank(index + 1)
// Adding Raw affiliation if it's defined
if (affiliation.isDefined) {
a.setRawAffiliationString(affiliation.get.map(a => a.name).asJava)
}
if (StringUtils.isNotBlank(orcid))
a.setPid(
List(
@ -673,11 +670,11 @@ case object Crossref2Oaf {
val doi = input.getString(0)
val rorId = input.getString(1)
val pubId = s"50|${PidType.doi.toString.padTo(12, "_")}::${DoiCleaningRule.clean(doi)}"
val pubId = IdentifierFactory.idFromPid("50", "doi", DoiCleaningRule.clean(doi), true)
val affId = GenerateRorActionSetJob.calculateOpenaireId(rorId)
val r: Relation = new Relation
DoiCleaningRule.clean(doi)
r.setSource(pubId)
r.setTarget(affId)
r.setRelType(ModelConstants.RESULT_ORGANIZATION)
@ -705,7 +702,15 @@ case object Crossref2Oaf {
val objectType = (json \ "type").extractOrElse[String](null)
if (objectType == null)
return resultList
val typology = getTypeQualifier(objectType, vocabularies)
// If the item has a relations is-review-of, then we force it to a peer-review
val is_review = json \ "relation" \ "is-review-of" \ "id"
var force_to_review = false
if (is_review != JNothing) {
force_to_review = true
}
val typology = getTypeQualifier(if (force_to_review) "peer-review" else objectType, vocabularies)
if (typology == null)
return List()
@ -757,33 +762,6 @@ case object Crossref2Oaf {
else
resultList
}
// if (uw != null) {
// result.getCollectedfrom.add(createUnpayWallCollectedFrom())
// val i: Instance = new Instance()
// i.setCollectedfrom(createUnpayWallCollectedFrom())
// if (uw.best_oa_location != null) {
//
// i.setUrl(List(uw.best_oa_location.url).asJava)
// if (uw.best_oa_location.license.isDefined) {
// i.setLicense(field[String](uw.best_oa_location.license.get, null))
// }
//
// val colour = get_unpaywall_color(uw.oa_status)
// if (colour.isDefined) {
// val a = new AccessRight
// a.setClassid(ModelConstants.ACCESS_RIGHT_OPEN)
// a.setClassname(ModelConstants.ACCESS_RIGHT_OPEN)
// a.setSchemeid(ModelConstants.DNET_ACCESS_MODES)
// a.setSchemename(ModelConstants.DNET_ACCESS_MODES)
// a.setOpenAccessRoute(colour.get)
// i.setAccessright(a)
// }
// i.setPid(result.getPid)
// result.getInstance().add(i)
// }
// }
}
private def createCiteRelation(source: Result, targetPid: String, targetPidType: String): List[Relation] = {
@ -978,7 +956,26 @@ case object Crossref2Oaf {
case "10.13039/501100010790" =>
generateSimpleRelationFromAward(funder, "erasmusplus_", a => a)
case _ => logger.debug("no match for " + funder.DOI.get)
//Add for Danish funders
//Independent Research Fund Denmark (IRFD)
case "10.13039/501100004836" =>
generateSimpleRelationFromAward(funder, "irfd________", a => a)
val targetId = getProjectId("irfd________", "1e5e62235d094afd01cd56e65112fc63")
queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY)
queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES)
//Carlsberg Foundation (CF)
case "10.13039/501100002808" =>
generateSimpleRelationFromAward(funder, "cf__________", a => a)
val targetId = getProjectId("cf__________", "1e5e62235d094afd01cd56e65112fc63")
queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY)
queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES)
//Novo Nordisk Foundation (NNF)
case "10.13039/501100009708" =>
generateSimpleRelationFromAward(funder, "nnf___________", a => a)
val targetId = getProjectId("nnf_________", "1e5e62235d094afd01cd56e65112fc63")
queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY)
queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES)
case _ => logger.debug("no match for " + funder.DOI.get)
}
} else {

View File

@ -0,0 +1,104 @@
package eu.dnetlib.dhp.sx.bio.ebi
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.application.AbstractScalaApplication
import eu.dnetlib.dhp.common.Constants
import eu.dnetlib.dhp.common.Constants.{MDSTORE_DATA_PATH, MDSTORE_SIZE_PATH}
import eu.dnetlib.dhp.common.vocabulary.VocabularyGroup
import eu.dnetlib.dhp.schema.mdstore.MDStoreVersion
import eu.dnetlib.dhp.sx.bio.pubmed.{PMArticle, PMParser2, PubMedToOaf}
import eu.dnetlib.dhp.transformation.TransformSparkJobNode
import eu.dnetlib.dhp.utils.DHPUtils.writeHdfsFile
import eu.dnetlib.dhp.utils.ISLookupClientFactory
import org.apache.spark.sql.{Encoder, Encoders, SparkSession}
import org.slf4j.{Logger, LoggerFactory}
class SparkCreatePubmedDump(propertyPath: String, args: Array[String], log: Logger)
extends AbstractScalaApplication(propertyPath, args, log: Logger) {
/** Here all the spark applications runs this method
* where the whole logic of the spark node is defined
*/
override def run(): Unit = {
val isLookupUrl: String = parser.get("isLookupUrl")
log.info("isLookupUrl: {}", isLookupUrl)
val sourcePath = parser.get("sourcePath")
log.info(s"SourcePath is '$sourcePath'")
val mdstoreOutputVersion = parser.get("mdstoreOutputVersion")
log.info(s"mdstoreOutputVersion is '$mdstoreOutputVersion'")
val mapper = new ObjectMapper()
val cleanedMdStoreVersion = mapper.readValue(mdstoreOutputVersion, classOf[MDStoreVersion])
val outputBasePath = cleanedMdStoreVersion.getHdfsPath
log.info(s"outputBasePath is '$outputBasePath'")
val isLookupService = ISLookupClientFactory.getLookUpService(isLookupUrl)
val vocabularies = VocabularyGroup.loadVocsFromIS(isLookupService)
createPubmedDump(spark, sourcePath, outputBasePath, vocabularies)
}
/** This method creates a dump of the pubmed articles
* @param spark the spark session
* @param sourcePath the path of the source file
* @param targetPath the path of the target file
* @param vocabularies the vocabularies
*/
def createPubmedDump(
spark: SparkSession,
sourcePath: String,
targetPath: String,
vocabularies: VocabularyGroup
): Unit = {
require(spark != null)
implicit val PMEncoder: Encoder[PMArticle] = Encoders.bean(classOf[PMArticle])
import spark.implicits._
val df = spark.read.option("lineSep", "</PubmedArticle>").text(sourcePath)
val mapper = new ObjectMapper()
df.as[String]
.map(s => {
val id = s.indexOf("<PubmedArticle>")
if (id >= 0) s"${s.substring(id)}</PubmedArticle>" else null
})
.filter(s => s != null)
.map { i =>
//remove try catch
try {
new PMParser2().parse(i)
} catch {
case _: Exception => {
throw new RuntimeException(s"Error parsing article: $i")
}
}
}
.dropDuplicates("pmid")
.map { a =>
val oaf = PubMedToOaf.convert(a, vocabularies)
if (oaf != null)
mapper.writeValueAsString(oaf)
else
null
}
.as[String]
.filter(s => s != null)
.write
.option("compression", "gzip")
.mode("overwrite")
.text(targetPath + MDSTORE_DATA_PATH)
val mdStoreSize = spark.read.text(targetPath + MDSTORE_DATA_PATH).count
writeHdfsFile(spark.sparkContext.hadoopConfiguration, "" + mdStoreSize, targetPath + MDSTORE_SIZE_PATH)
}
}
object SparkCreatePubmedDump {
def main(args: Array[String]): Unit = {
val log: Logger = LoggerFactory.getLogger(getClass)
new SparkCreatePubmedDump("/eu/dnetlib/dhp/sx/bio/ebi/baseline_to_oaf_params.json", args, log).initialize().run()
}
}

View File

@ -0,0 +1,277 @@
package eu.dnetlib.dhp.sx.bio.pubmed
import org.apache.commons.lang3.StringUtils
import javax.xml.stream.XMLEventReader
import scala.collection.JavaConverters._
import scala.xml.{MetaData, NodeSeq}
import scala.xml.pull.{EvElemEnd, EvElemStart, EvText}
class PMParser2 {
/** Extracts the value of an attribute from a MetaData object.
* @param attrs the MetaData object
* @param key the key of the attribute
* @return the value of the attribute or null if the attribute is not found
*/
private def extractAttributes(attrs: MetaData, key: String): String = {
val res = attrs.get(key)
if (res.isDefined) {
val s = res.get
if (s != null && s.nonEmpty)
s.head.text
else
null
} else null
}
/** Validates and formats a date given the year, month, and day as strings.
*
* @param year the year as a string
* @param month the month as a string
* @param day the day as a string
* @return the formatted date as "YYYY-MM-DD" or null if the date is invalid
*/
private def validate_Date(year: String, month: String, day: String): String = {
try {
f"${year.toInt}-${month.toInt}%02d-${day.toInt}%02d"
} catch {
case _: Throwable => null
}
}
/** Extracts the grant information from a NodeSeq object.
*
* @param gNode the NodeSeq object
* @return the grant information or an empty list if the grant information is not found
*/
private def extractGrant(gNode: NodeSeq): List[PMGrant] = {
gNode
.map(node => {
val grantId = (node \ "GrantID").text
val agency = (node \ "Agency").text
val country = (node \ "Country").text
new PMGrant(grantId, agency, country)
})
.toList
}
/** Extracts the journal information from a NodeSeq object.
*
* @param jNode the NodeSeq object
* @return the journal information or null if the journal information is not found
*/
private def extractJournal(jNode: NodeSeq): PMJournal = {
val journal = new PMJournal
journal.setTitle((jNode \ "Title").text)
journal.setIssn((jNode \ "ISSN").text)
journal.setVolume((jNode \ "JournalIssue" \ "Volume").text)
journal.setIssue((jNode \ "JournalIssue" \ "Issue").text)
if (journal.getTitle != null && StringUtils.isNotEmpty(journal.getTitle))
journal
else
null
}
private def extractAuthors(aNode: NodeSeq): List[PMAuthor] = {
aNode
.map(author => {
val a = new PMAuthor
a.setLastName((author \ "LastName").text)
a.setForeName((author \ "ForeName").text)
val id = (author \ "Identifier").text
val idType = (author \ "Identifier" \ "@Source").text
if (id != null && id.nonEmpty && idType != null && idType.nonEmpty) {
a.setIdentifier(new PMIdentifier(id, idType))
}
val affiliation = (author \ "AffiliationInfo" \ "Affiliation").text
val affiliationId = (author \ "AffiliationInfo" \ "Identifier").text
val affiliationIdType = (author \ "AffiliationInfo" \ "Identifier" \ "@Source").text
if (affiliation != null && affiliation.nonEmpty) {
val aff = new PMAffiliation()
aff.setName(affiliation)
if (
affiliationId != null && affiliationId.nonEmpty && affiliationIdType != null && affiliationIdType.nonEmpty
) {
aff.setIdentifier(new PMIdentifier(affiliationId, affiliationIdType))
}
a.setAffiliation(aff)
}
a
})
.toList
}
def parse(input: String): PMArticle = {
val xml = scala.xml.XML.loadString(input)
val article = new PMArticle
val grantNodes = xml \ "MedlineCitation" \\ "Grant"
article.setGrants(extractGrant(grantNodes).asJava)
val journal = xml \ "MedlineCitation" \ "Article" \ "Journal"
article.setJournal(extractJournal(journal))
val authors = xml \ "MedlineCitation" \ "Article" \ "AuthorList" \ "Author"
article.setAuthors(
extractAuthors(authors).asJava
)
val pmId = xml \ "MedlineCitation" \ "PMID"
val articleIds = xml \ "PubmedData" \ "ArticleIdList" \ "ArticleId"
articleIds.foreach(articleId => {
val idType = (articleId \ "@IdType").text
val id = articleId.text
if ("doi".equalsIgnoreCase(idType)) article.setDoi(id)
if ("pmc".equalsIgnoreCase(idType)) article.setPmcId(id)
})
article.setPmid(pmId.text)
val pubMedPubDate = xml \ "MedlineCitation" \ "DateCompleted"
val currentDate =
validate_Date((pubMedPubDate \ "Year").text, (pubMedPubDate \ "Month").text, (pubMedPubDate \ "Day").text)
if (currentDate != null) article.setDate(currentDate)
val articleTitle = xml \ "MedlineCitation" \ "Article" \ "ArticleTitle"
article.setTitle(articleTitle.text)
val abstractText = xml \ "MedlineCitation" \ "Article" \ "Abstract" \ "AbstractText"
if (abstractText != null && abstractText.text != null && abstractText.text.nonEmpty)
article.setDescription(abstractText.text.split("\n").map(s => s.trim).mkString(" ").trim)
val language = xml \ "MedlineCitation" \ "Article" \ "Language"
article.setLanguage(language.text)
val subjects = xml \ "MedlineCitation" \ "MeshHeadingList" \ "MeshHeading"
article.setSubjects(
subjects
.take(20)
.map(subject => {
val descriptorName = (subject \ "DescriptorName").text
val ui = (subject \ "DescriptorName" \ "@UI").text
val s = new PMSubject
s.setValue(descriptorName)
s.setMeshId(ui)
s
})
.toList
.asJava
)
val publicationTypes = xml \ "MedlineCitation" \ "Article" \ "PublicationTypeList" \ "PublicationType"
article.setPublicationTypes(
publicationTypes
.map(pt => {
val s = new PMSubject
s.setValue(pt.text)
s
})
.toList
.asJava
)
article
}
def parse2(xml: XMLEventReader): PMArticle = {
var currentArticle: PMArticle = null
var currentSubject: PMSubject = null
var currentAuthor: PMAuthor = null
var currentJournal: PMJournal = null
var currentGrant: PMGrant = null
var currNode: String = null
var currentYear = "0"
var currentMonth = "01"
var currentDay = "01"
var currentArticleType: String = null
while (xml.hasNext) {
val ne = xml.next
ne match {
case EvElemStart(_, label, attrs, _) =>
currNode = label
label match {
case "PubmedArticle" => currentArticle = new PMArticle
case "Author" => currentAuthor = new PMAuthor
case "Journal" => currentJournal = new PMJournal
case "Grant" => currentGrant = new PMGrant
case "PublicationType" | "DescriptorName" =>
currentSubject = new PMSubject
currentSubject.setMeshId(extractAttributes(attrs, "UI"))
case "ArticleId" => currentArticleType = extractAttributes(attrs, "IdType")
case _ =>
}
case EvElemEnd(_, label) =>
label match {
case "PubmedArticle" => return currentArticle
case "Author" => currentArticle.getAuthors.add(currentAuthor)
case "Journal" => currentArticle.setJournal(currentJournal)
case "Grant" => currentArticle.getGrants.add(currentGrant)
case "PubMedPubDate" =>
if (currentArticle.getDate == null)
currentArticle.setDate(validate_Date(currentYear, currentMonth, currentDay))
case "PubDate" => currentJournal.setDate(s"$currentYear-$currentMonth-$currentDay")
case "DescriptorName" => currentArticle.getSubjects.add(currentSubject)
case "PublicationType" => currentArticle.getPublicationTypes.add(currentSubject)
case _ =>
}
case EvText(text) =>
if (currNode != null && text.trim.nonEmpty)
currNode match {
case "ArticleTitle" => {
if (currentArticle.getTitle == null)
currentArticle.setTitle(text.trim)
else
currentArticle.setTitle(currentArticle.getTitle + text.trim)
}
case "AbstractText" => {
if (currentArticle.getDescription == null)
currentArticle.setDescription(text.trim)
else
currentArticle.setDescription(currentArticle.getDescription + text.trim)
}
case "PMID" => currentArticle.setPmid(text.trim)
case "ArticleId" =>
if ("doi".equalsIgnoreCase(currentArticleType)) currentArticle.setDoi(text.trim)
if ("pmc".equalsIgnoreCase(currentArticleType)) currentArticle.setPmcId(text.trim)
case "Language" => currentArticle.setLanguage(text.trim)
case "ISSN" => currentJournal.setIssn(text.trim)
case "GrantID" => currentGrant.setGrantID(text.trim)
case "Agency" => currentGrant.setAgency(text.trim)
case "Country" => if (currentGrant != null) currentGrant.setCountry(text.trim)
case "Year" => currentYear = text.trim
case "Month" => currentMonth = text.trim
case "Day" => currentDay = text.trim
case "Volume" => currentJournal.setVolume(text.trim)
case "Issue" => currentJournal.setIssue(text.trim)
case "PublicationType" | "DescriptorName" => currentSubject.setValue(text.trim)
case "LastName" => {
if (currentAuthor != null)
currentAuthor.setLastName(text.trim)
}
case "ForeName" =>
if (currentAuthor != null)
currentAuthor.setForeName(text.trim)
case "Title" =>
if (currentJournal.getTitle == null)
currentJournal.setTitle(text.trim)
else
currentJournal.setTitle(currentJournal.getTitle + text.trim)
case _ =>
}
case _ =>
}
}
null
}
}

View File

@ -294,6 +294,24 @@ object PubMedToOaf {
author.setName(a.getForeName)
author.setSurname(a.getLastName)
author.setFullname(a.getFullName)
if (a.getIdentifier != null) {
author.setPid(
List(
OafMapperUtils.structuredProperty(
a.getIdentifier.getPid,
OafMapperUtils.qualifier(
a.getIdentifier.getType,
a.getIdentifier.getType,
ModelConstants.DNET_PID_TYPES,
ModelConstants.DNET_PID_TYPES
),
dataInfo
)
).asJava
)
}
if (a.getAffiliation != null)
author.setRawAffiliationString(List(a.getAffiliation.getName).asJava)
author.setRank(index + 1)
author
}(collection.breakOut)

View File

@ -98,9 +98,9 @@ public class PrepareAffiliationRelationsTest {
"-crossrefInputPath", crossrefAffiliationRelationPathNew,
"-pubmedInputPath", crossrefAffiliationRelationPath,
"-openapcInputPath", crossrefAffiliationRelationPathNew,
"-dataciteInputPath", crossrefAffiliationRelationPath,
"-webCrawlInputPath", crossrefAffiliationRelationPath,
"-publisherInputPath", publisherAffiliationRelationOldPath,
"-dataciteInputPath", crossrefAffiliationRelationPathNew,
"-webCrawlInputPath", crossrefAffiliationRelationPathNew,
"-publisherInputPath", publisherAffiliationRelationPath,
"-outputPath", outputPath
});
@ -112,7 +112,7 @@ public class PrepareAffiliationRelationsTest {
.map(aa -> ((Relation) aa.getPayload()));
// count the number of relations
assertEquals(150, tmp.count());// 18 + 24 *3 + 30 * 2 =
assertEquals(162, tmp.count());// 18 + 24 + 30 * 4 =
Dataset<Relation> dataset = spark.createDataset(tmp.rdd(), Encoders.bean(Relation.class));
dataset.createOrReplaceTempView("result");
@ -123,7 +123,7 @@ public class PrepareAffiliationRelationsTest {
// verify that we have equal number of bi-directional relations
Assertions
.assertEquals(
75, execVerification
81, execVerification
.filter(
"relClass='" + ModelConstants.HAS_AUTHOR_INSTITUTION + "'")
.collectAsList()
@ -131,7 +131,7 @@ public class PrepareAffiliationRelationsTest {
Assertions
.assertEquals(
75, execVerification
81, execVerification
.filter(
"relClass='" + ModelConstants.IS_AUTHOR_INSTITUTION_OF + "'")
.collectAsList()
@ -158,7 +158,7 @@ public class PrepareAffiliationRelationsTest {
Assertions
.assertEquals(
2, execVerification.filter("source = '" + publisherid + "' and target = '" + rorId + "'").count());
4, execVerification.filter("source = '" + publisherid + "' and target = '" + rorId + "'").count());
Assertions
.assertEquals(
@ -173,7 +173,7 @@ public class PrepareAffiliationRelationsTest {
Assertions
.assertEquals(
3, execVerification
1, execVerification
.filter(
"source = '" + ID_PREFIX
+ IdentifierFactory

View File

@ -0,0 +1,165 @@
package eu.dnetlib.dhp.actionmanager.raid;
import static java.nio.file.Files.createTempDirectory;
import static eu.dnetlib.dhp.actionmanager.Constants.OBJECT_MAPPER;
import static org.junit.jupiter.api.Assertions.assertEquals;
import java.io.File;
import java.nio.file.Paths;
import java.util.Arrays;
import java.util.List;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.rdd.RDD;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import eu.dnetlib.dhp.actionmanager.opencitations.CreateOpenCitationsASTest;
import eu.dnetlib.dhp.actionmanager.raid.model.RAiDEntity;
import eu.dnetlib.dhp.schema.action.AtomicAction;
import eu.dnetlib.dhp.schema.oaf.Oaf;
import eu.dnetlib.dhp.schema.oaf.OtherResearchProduct;
import eu.dnetlib.dhp.schema.oaf.Relation;
import scala.Tuple2;
public class GenerateRAiDActionSetJobTest {
private static String input_path;
private static String output_path;
static SparkSession spark;
@BeforeEach
void setUp() throws Exception {
input_path = Paths
.get(
GenerateRAiDActionSetJobTest.class
.getResource("/eu/dnetlib/dhp/actionmanager/raid/raid_example.json")
.toURI())
.toFile()
.getAbsolutePath();
output_path = createTempDirectory(GenerateRAiDActionSetJobTest.class.getSimpleName() + "-")
.toAbsolutePath()
.toString();
SparkConf conf = new SparkConf();
conf.setAppName(GenerateRAiDActionSetJobTest.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", output_path);
conf.set("hive.metastore.warehouse.dir", output_path);
spark = SparkSession
.builder()
.appName(GenerateRAiDActionSetJobTest.class.getSimpleName())
.config(conf)
.getOrCreate();
}
@AfterAll
static void cleanUp() throws Exception {
FileUtils.deleteDirectory(new File(output_path));
}
@Test
@Disabled
void testProcessRAiDEntities() {
GenerateRAiDActionSetJob.processRAiDEntities(spark, input_path, output_path + "/test_raid_action_set");
JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<? extends Oaf> result = sc
.sequenceFile(output_path + "/test_raid_action_set", Text.class, Text.class)
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(AtomicAction::getPayload);
assertEquals(80, result.count());
}
@Test
void testPrepareRAiD() {
List<AtomicAction<? extends Oaf>> atomicActions = GenerateRAiDActionSetJob
.prepareRAiD(
new RAiDEntity(
"-92190526",
Arrays
.asList(
"Berli, Justin", "Le Mao, Bérénice", "Guillaume Touya", "Wenclik, Laura",
"Courtial, Azelle", "Muehlenhaus, Ian", "Justin Berli", "Touya, Guillaume",
"Gruget, Maïeul", "Azelle Courtial", "Ian Muhlenhaus", "Maïeul Gruget", "Marion Dumont",
"Maïeul GRUGET", "Cécile Duchêne"),
"2021-09-10",
"2024-02-16",
Arrays
.asList(
"cartography, zoom, pan, desert fog", "Road network", "zooming", "Pan-scalar maps",
"pan-scalar map", "Python library", "QGIS", "map design", "landmarks",
"Cartes transscalaires", "anchor", "disorientation", "[INFO]Computer Science [cs]",
"[SHS.GEO]Humanities and Social Sciences/Geography", "cognitive cartography",
"eye-tracking", "Computers in Earth Sciences", "Topographic map", "National Mapping Agency",
"General Medicine", "Geography, Planning and Development", "multi-scales",
"pan-scalar maps", "Selection", "cartography", "General Earth and Planetary Sciences",
"progressiveness", "map generalisation", "Eye-tracker", "zoom", "algorithms", "Map Design",
"cartography, map generalisation, zoom, multi-scale map", "Interactive maps",
"Map generalisation", "Earth and Planetary Sciences (miscellaneous)",
"Cartographic generalization", "rivers", "Benchmark", "General Environmental Science",
"open source", "drawing", "Constraint", "Multi-scale maps"),
Arrays
.asList(
"Where do people look at during multi-scale map tasks?", "FogDetector survey raw data",
"Collection of cartographic disorientation stories", "Anchorwhat dataset",
"BasqueRoads: A Benchmark for Road Network Selection",
"Progressive river network selection for pan-scalar maps",
"BasqueRoads, a dataset to benchmark road selection algorithms",
"Missing the city for buildings? A critical review of pan-scalar map generalization and design in contemporary zoomable maps",
"Empirical approach to advance the generalisation of multi-scale maps",
"L'Alpe d'Huez: a dataset to benchmark topographic map generalisation",
"eye-tracking data from a survey on zooming in a pan-scalar map",
"Material of the experiment 'More is Less' from the MapMuxing project",
"Cartagen4py, an open source Python library for map generalisation",
"LAlpe dHuez: A Benchmark for Topographic Map Generalisation"),
Arrays
.asList(
"50|doi_dedup___::6915135e0aa39f913394513f809ae58a",
"50|doi_dedup___::754e3c283639bc6e104c925ff3e34007",
"50|doi_dedup___::13517477f3c1261d57a3364363ce6ce0",
"50|doi_dedup___::675b16c73accc4e7242bbb4ed9b3724a",
"50|doi_dedup___::94ce09906b2d7d37eb2206cea8a50153",
"50|dedup_wf_002::cc575d5ca5651ff8c3029a3a76e7e70a",
"50|doi_dedup___::c5e52baddda17c755d1bae012a97dc13",
"50|doi_dedup___::4f5f38c9e08fe995f7278963183f8ad4",
"50|doi_dedup___::a9bc4453273b2d02648a5cb453195042",
"50|doi_dedup___::5e893dc0cb7624a33f41c9b428bd59f7",
"50|doi_dedup___::c1ecdef48fd9be811a291deed950e1c5",
"50|doi_dedup___::9e93c8f2d97c35de8a6a57a5b53ef283",
"50|dedup_wf_002::d08be0ed27b13d8a880e891e08d093ea",
"50|doi_dedup___::f8d8b3b9eddeca2fc0e3bc9e63996555"),
"Exploring Multi-Scale Map Generalization and Design",
"This project aims to advance the generalization of multi-scale maps by investigating the impact of different design elements on user experience. The research involves collecting and analyzing data from various sources, including surveys, eye-tracking studies, and user experiments. The goal is to identify best practices for map generalization and design, with a focus on reducing disorientation and improving information retrieval during exploration. The project has led to the development of several datasets, including BasqueRoads, AnchorWhat, and L'Alpe d'Huez, which can be used to benchmark road selection algorithms and topographic map generalization techniques. The research has also resulted in the creation of a Python library, Cartagen4py, for map generalization. The findings of this project have the potential to improve the design and usability of multi-scale maps, making them more effective tools for navigation and information retrieval."));
OtherResearchProduct orp = (OtherResearchProduct) atomicActions.get(0).getPayload();
Relation rel = (Relation) atomicActions.get(1).getPayload();
assertEquals("Exploring Multi-Scale Map Generalization and Design", orp.getTitle().get(0).getValue());
assertEquals("50|raid________::759a564ce5cc7360cab030c517c7366b", rel.getSource());
assertEquals("50|doi_dedup___::6915135e0aa39f913394513f809ae58a", rel.getTarget());
}
}

View File

@ -0,0 +1,35 @@
package eu.dnetlib.dhp.collection.plugin.zenodo;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import java.util.zip.GZIPInputStream;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.collection.ApiDescriptor;
import eu.dnetlib.dhp.common.collection.CollectorException;
public class ZenodoPluginCollectionTest {
@Test
public void testZenodoIterator() throws Exception {
final GZIPInputStream gis = new GZIPInputStream(
getClass().getResourceAsStream("/eu/dnetlib/dhp/collection/zenodo/zenodo.tar.gz"));
try (ZenodoTarIterator it = new ZenodoTarIterator(gis)) {
Assertions.assertTrue(it.hasNext());
int i = 0;
while (it.hasNext()) {
Assertions.assertNotNull(it.next());
i++;
}
Assertions.assertEquals(10, i);
}
}
}

View File

@ -0,0 +1,6 @@
{"raid": "-9222092103004099540", "authors": ["Department of Archaeology & Museums", "Department of Archaeology and Museums", "Department Of Archaeology & Museums"], "subjects": ["Begamganj", "Raisen", "Bhopal", "Budhni", "Malwa site survey", "सीहोर", "Gauharganj", "बुधनी", "Budni", "Berasia"], "titles": ["Malwa site survey : Raisen District, Begamganj Tahsīl, photographic documentation", "Malwa site survey : Bhopal District, photographic documentation (version 1, TIFF files)", "Malwa site survey : Raisen District, Gauharganj Tahsīl, village finds", "Malwa site survey : Sehore सीहोर District, Budni Tahsīl, photographic documentation (part 1)", "Malwa site survey: Bhopal District, Berasia Tahsīl, photographic documentation (with villages named)", "Malwa site survey : Sehore सीहोर District, Budni Tahsīl, photographic documentation (part 2)", "Malwa site survey : Bhopal District, photographic documentation (version 2, JPEG files)"], "ids": ["50|doi_dedup___::7523d165970830dd857e6cbea4302adf", "50|doi_dedup___::02309ae8a9fae291df321e317f5c5330", "50|doi_dedup___::95347ba2c4264414fab39712ee7fe481", "50|doi_dedup___::970aa708fe667596754fd02a708780f5", "50|doi_dedup___::b7cd9128cc53b1257a4f000347f339b0", "50|doi_dedup___::c7d65da0ecedef4d2c702b9db197d90c", "50|doi_dedup___::addbb67cf5046e340f342ba091bcebfa"], "title": "Documentation of Malwa Region", "summary": "This project involves the documentation of the Malwa region through photographic surveys. The surveys were conducted by the Department of Archaeology and Museums, Madhya Pradesh, and cover various districts and tahsils. The documentation includes photographic records of sites, villages, and other relevant features. The project aims to provide a comprehensive understanding of the region's cultural and historical significance.", "startDate": "2019-03-06", "endDate": "2019-03-08"}
{"raid": "-9221424331076109424", "authors": ["Hutchings, Judy", "Ward, Catherine", "Baban, Adriana", "D<><44>nil<69><6C>, Ingrid", "Frantz, Inga", "Gardner, Frances", "Lachman, Jamie", "Lachman, Jamie M.", "Foran, Heather", "Heinrichs, Nina", "Murphy, Hugh", "B<><42>ban, Adriana", "Raleva, Marija", "Fang, Xiangming", "Jansen, Elena", "Taut, Diana", "Foran, Heather M.", "T<><54>ut, Diana", "Ward, Catherine L.", "Williams, Margiad", "Lesco, Galina", "Brühl, Antonia"], "subjects": ["3. Good health", "5. Gender equality", "Criminology not elsewhere classified", "1. No poverty", "2. Zero hunger"], "titles": ["sj-docx-1-vaw-10.1177_10778012231188090 - Supplemental material for Co-Occurrence of Intimate Partner Violence Against Mothers and Maltreatment of Their Children With Behavioral Problems in Eastern Europe", "Hunger in vulnerable families in Southeastern Europe: Associations with health and violence", "Prevention of child mental health problems through parenting interventions in Southeastern Europe (RISE): study protocol for a multi-site randomised controlled trial"], "ids": ["50|doi_dedup___::a70015063e5400dae2e097ee10b4a589", "50|doi_dedup___::6e1d12026fcde9087724622ccdeed430", "50|doi_dedup___::5b7bd5d46c5d95e2ef5b36663504a67e"], "title": "Exploring the Impact of Hunger and Violence on Child Health in Southeastern Europe", "summary": "This study aims to investigate the relationship between hunger, violence, and child health in vulnerable families in Southeastern Europe. The research will explore the experiences of families in FYR Macedonia, Republic of Moldova, and Romania, and examine the associations between hunger, maltreatment, and other health indicators. The study will also test the efficacy of a parenting intervention targeting child behavioral problems in alleviating these issues. The findings of this research will contribute to the development of effective interventions to address the complex needs of vulnerable families in the region.", "startDate": "2019-06-04", "endDate": "2023-01-01"}
{"raid": "-9219052635741785098", "authors": ["Berli, Justin", "Le Mao, Bérénice", "Guillaume Touya", "Wenclik, Laura", "Courtial, Azelle", "Muehlenhaus, Ian", "Justin Berli", "Touya, Guillaume", "Gruget, Maïeul", "Azelle Courtial", "Ian Muhlenhaus", "Maïeul Gruget", "Marion Dumont", "Maïeul GRUGET", "Cécile Duchêne"], "subjects": ["cartography, zoom, pan, desert fog", "Road network", "zooming", "Pan-scalar maps", "pan-scalar map", "Python library", "QGIS", "map design", "landmarks", "Cartes transscalaires", "anchor", "disorientation", "[INFO]Computer Science [cs]", "[SHS.GEO]Humanities and Social Sciences/Geography", "cognitive cartography", "eye-tracking", "Computers in Earth Sciences", "Topographic map", "National Mapping Agency", "General Medicine", "Geography, Planning and Development", "multi-scales", "pan-scalar maps", "Selection", "cartography", "General Earth and Planetary Sciences", "progressiveness", "map generalisation", "Eye-tracker", "zoom", "algorithms", "Map Design", "cartography, map generalisation, zoom, multi-scale map", "Interactive maps", "Map generalisation", "Earth and Planetary Sciences (miscellaneous)", "Cartographic generalization", "rivers", "Benchmark", "General Environmental Science", "open source", "drawing", "Constraint", "Multi-scale maps"], "titles": ["Where do people look at during multi-scale map tasks?", "FogDetector survey raw data", "Collection of cartographic disorientation stories", "Anchorwhat dataset", "BasqueRoads: A Benchmark for Road Network Selection", "Progressive river network selection for pan-scalar maps", "BasqueRoads, a dataset to benchmark road selection algorithms", "Missing the city for buildings? A critical review of pan-scalar map generalization and design in contemporary zoomable maps", "Empirical approach to advance the generalisation of multi-scale maps", "L'Alpe d'Huez: a dataset to benchmark topographic map generalisation", "eye-tracking data from a survey on zooming in a pan-scalar map", "Material of the experiment \"More is Less\" from the MapMuxing project", "Cartagen4py, an open source Python library for map generalisation", "LAlpe dHuez: A Benchmark for Topographic Map Generalisation"], "ids": ["50|doi_dedup___::6915135e0aa39f913394513f809ae58a", "50|doi_dedup___::754e3c283639bc6e104c925ff3e34007", "50|doi_dedup___::13517477f3c1261d57a3364363ce6ce0", "50|doi_dedup___::675b16c73accc4e7242bbb4ed9b3724a", "50|doi_dedup___::94ce09906b2d7d37eb2206cea8a50153", "50|dedup_wf_002::cc575d5ca5651ff8c3029a3a76e7e70a", "50|doi_dedup___::c5e52baddda17c755d1bae012a97dc13", "50|doi_dedup___::4f5f38c9e08fe995f7278963183f8ad4", "50|doi_dedup___::a9bc4453273b2d02648a5cb453195042", "50|doi_dedup___::5e893dc0cb7624a33f41c9b428bd59f7", "50|doi_dedup___::c1ecdef48fd9be811a291deed950e1c5", "50|doi_dedup___::9e93c8f2d97c35de8a6a57a5b53ef283", "50|dedup_wf_002::d08be0ed27b13d8a880e891e08d093ea", "50|doi_dedup___::f8d8b3b9eddeca2fc0e3bc9e63996555"], "title": "Exploring Multi-Scale Map Generalization and Design", "summary": "This project aims to advance the generalization of multi-scale maps by investigating the impact of different design elements on user experience. The research involves collecting and analyzing data from various sources, including surveys, eye-tracking studies, and user experiments. The goal is to identify best practices for map generalization and design, with a focus on reducing disorientation and improving information retrieval during exploration. The project has led to the development of several datasets, including BasqueRoads, AnchorWhat, and L'Alpe d'Huez, which can be used to benchmark road selection algorithms and topographic map generalization techniques. The research has also resulted in the creation of a Python library, Cartagen4py, for map generalization. The findings of this project have the potential to improve the design and usability of multi-scale maps, making them more effective tools for navigation and information retrieval.", "startDate": "2021-09-10", "endDate": "2024-02-16"}
{"raid": "-9216828847055450272", "authors": ["Grey, Alan", "Gorelov, Sergey", "Pall, Szilard", "Merz, Pascal", "Justin A., Lemkul", "Szilárd Páll", "Pasquadibisceglie, Andrea", "Kutzner, Carsten", "Schulz, Roland", "Nabet, Julien", "Abraham, Mark", "Jalalypour, Farzaneh", "Lundborg, Magnus", "Gray, Alan", "Villa, Alessandra", "Berk Hess", "Santuz, Hubert", "Irrgang, M. Eric", "Wingbermuehle, Sebastian", "Lemkul, Justin A.", "Jordan, Joe", "Pellegrino, Michele", "Doijade, Mahesh", "Shvetsov, Alexey", "Hess, Berk", "Behera, Sudarshan", "Andrey Alekseenko", "Shugaeva, Tatiana", "Fleischmann, Stefan", "Bergh, Cathrine", "Morozov, Dmitry", "Adam Hospital", "Briand, Eliane", "Lindahl, Erik", "Brown, Ania", "Marta Lloret Llinares", "Miletic, Vedran", "Alekseenko, Andrey", "Gouaillardet, Gilles", "Fiorin, Giacomo", "Basov, Vladimir"], "subjects": ["webinar"], "titles": ["Introduction to HPC: molecular dynamics simulations with GROMACS: log files", "BioExcel webinar #73: Competency frameworks to support training design and professional development", "Introduction to HPC: molecular dynamics simulations with GROMACS: output files - Devana", "GROMACS 2024.0 Manual", "BioExcel Webinar #71: GROMACS-PMX for accurate estimation of free energy differences", "Introduction to HPC: molecular dynamics simulations with GROMACS: input files", "BioExcel Webinar #68: What's new in GROMACS 2023", "BioExcel Webinar #69: BioBB-Wfs and BioBB-API, integrated web-based platform and programmatic interface for biomolecular simulations workflows using the BioExcel Building Blocks library", "GROMACS 2024-beta Source code"], "ids": ["50|doi_dedup___::8318fbc815ee1943c3269be7567f220b", "50|doi_dedup___::9530e03fb2aac63e82b18a40dc09e32c", "50|doi_dedup___::30174ab31075e76a428ca5b4f4d236b8", "50|doi_________::70b7c6dce09ae6f1361d22913fdf95eb", "50|doi_dedup___::337dd48600618f3c06257edd750d6201", "50|doi_dedup___::d622992ba9077617f37ebd268b3e806d", "50|doi_dedup___::0b0bcc6825d6c052c37882fd5cfc1e8c", "50|doi_dedup___::4b1541a7cee32527c65ace5d1ed57335", "50|doi_dedup___::1379861df59bd755e4fb39b9f95ffbd3"], "title": "Exploring High-Performance Computing and Biomolecular Simulations", "summary": "This project involves exploring high-performance computing (HPC) and biomolecular simulations using GROMACS. The objectives include understanding molecular dynamics simulations, log files, input files, and output files. Additionally, the project aims to explore competency frameworks for professional development, specifically in the field of computational biomolecular research. The tools and techniques used will include GROMACS, BioExcel Building Blocks, and competency frameworks. The expected outcomes include a deeper understanding of HPC and biomolecular simulations, as well as the development of skills in using GROMACS and BioExcel Building Blocks. The project will also contribute to the development of competency frameworks for professional development in the field of computational biomolecular research.", "startDate": "2023-04-25", "endDate": "2024-01-30"}
{"raid": "-9210544816395499758", "authors": ["Bateson, Melissa", "Andrews, Clare", "Verhulst, Simon", "Nettle, Daniel", "Zuidersma, Erica"], "subjects": ["2. Zero hunger"], "titles": ["Exposure to food insecurity increases energy storage and reduces somatic maintenance in European starlings", "Data and code archive for Andrews et al. 'Exposure to food insecurity increases energy storage and reduces somatic maintenance in European starlings'"], "ids": ["50|doi_dedup___::176117239be06189523c253e0ca9c5ec", "50|doi_dedup___::343e0b0ddf0d54763a89a62af1f7a379"], "title": "Investigating the Effects of Food Insecurity on Energy Storage and Somatic Maintenance in European Starlings", "summary": "This study examines the impact of food insecurity on energy storage and somatic maintenance in European starlings. The research involved exposing juvenile starlings to either uninterrupted food availability or a regime of unpredictable food unavailability. The results show that birds exposed to food insecurity stored more energy, but at the expense of somatic maintenance and repair. The study provides insights into the adaptive responses of birds to food scarcity and the trade-offs involved in energy storage and maintenance.", "startDate": "2021-06-28", "endDate": "2021-06-28"}
{"raid": "-9208499171224730388", "authors": ["Maniati, Eleni", "Bakker, Bjorn", "McClelland, Sarah E.", "Shaikh, Nadeem", "De Angelis, Simone", "Johnson, Sarah C.", "Wang, Jun", "Foijer, Floris", "Spierings, Diana C. J.", "Boemo, Michael A.", "Wardenaar, René", "Mazzagatti, Alice"], "subjects": [], "titles": ["Additional file 2 of Replication stress generates distinctive landscapes of DNA copy number alterations and chromosome scale losses", "Additional file 5 of Replication stress generates distinctive landscapes of DNA copy number alterations and chromosome scale losses"], "ids": ["50|doi_dedup___::a1bfeb173971f74a274fab8bdd78a4bc", "50|doi_dedup___::3d6e151aaeb2f7c40a320207fdd80ade"], "title": "Analysis of DNA Copy Number Alterations and Chromosome Scale Losses", "summary": "This study analyzed the effects of replication stress on DNA copy number alterations and chromosome scale losses. The results show distinctive landscapes of these alterations and losses, which were further investigated in additional files. The study provides valuable insights into the mechanisms of replication stress and its impact on genomic stability.", "startDate": "2022-01-01", "endDate": "2022-01-01"}

View File

@ -0,0 +1,232 @@
{
"indexed": {
"date-parts": [
[
2022,
4,
3
]
],
"date-time": "2022-04-03T01:45:59Z",
"timestamp": 1648950359167
},
"reference-count": 0,
"publisher": "American Society of Clinical Oncology (ASCO)",
"issue": "18_suppl",
"content-domain": {
"domain": [],
"crossmark-restriction": false
},
"short-container-title": [
"JCO"
],
"published-print": {
"date-parts": [
[
2007,
6,
20
]
]
},
"abstract": "<jats:p> 3507 </jats:p><jats:p> Purpose: To detect IGF-1R on circulating tumor cells (CTCs) as a biomarker in the clinical development of a monoclonal human antibody, CP-751,871, targeting IGF-1R. Experimental Design: An automated sample preparation and analysis system for enumerating CTCs (Celltracks) was adapted for detecting IGF-1R positive CTCs with a diagnostic antibody targeting a different IGF-1R epitope to CP-751,871. This assay was utilized in three phase I trials of CP-751,871 as a single agent or with chemotherapy and was validated using cell lines and blood samples from healthy volunteers and patients with metastatic carcinoma. Results: There was no interference between the analytical and therapeutic antibodies. CP-751,871 was well tolerated as a single agent, and in combination with docetaxel or carboplatin and paclitaxel, at doses ranging from 0.05 mg/kg to 20 mg/kg. Eighty patients were enrolled on phase 1 studies of CP-751,871, with 47 (59%) patients having CTCs detected during the study. Prior to treatment 26 patients (33%) had CTCs, with 23 having detectable IGF-1R positive CTCs. CP-751,871 alone, and CP-751,871 with cytotoxic chemotherapy, decreased CTCs and IGF-1R positive CTCs; these increased towards the end of the 21-day cycle in some patients, falling again with retreatment. CTCs were commonest in advanced hormone refractory prostate cancer (11/20). Detectable IGF-1R expression on CTCs before treatment with CP-751,871 and docetaxel was associated with a higher frequency of PSA decline by more than 50% (6/10 vs 2/8 patients). A relationship was observed between sustained falls in CTCs counts and PSA declines by more than 50%. Conclusions: IGF-1R expression is detectable by immunofluorescence on CTCs. These data support the further evaluation of CTCs in pharmacodynamic studies and patient selection, particularly in advanced prostate cancer. </jats:p><jats:p> No significant financial relationships to disclose. </jats:p>",
"DOI": "10.1200/jco.2007.25.18_suppl.3507",
"type": "journal-article",
"created": {
"date-parts": [
[
2020,
3,
6
]
],
"date-time": "2020-03-06T20:50:42Z",
"timestamp": 1583527842000
},
"page": "3507-3507",
"source": "Crossref",
"is-referenced-by-count": 0,
"title": [
"Circulating tumor cells expressing the insulin growth factor-1 receptor (IGF-1R): Method of detection, incidence and potential applications"
],
"prefix": "10.1200",
"volume": "25",
"author": [
{
"given": "J. S.",
"family": "de Bono",
"sequence": "first",
"affiliation": [
{
"name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA"
}
]
},
{
"given": "A.",
"family": "Adjei",
"sequence": "additional",
"affiliation": [
{
"name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA"
}
]
},
{
"given": "G.",
"family": "Attard",
"sequence": "additional",
"affiliation": [
{
"name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA"
}
]
},
{
"given": "M.",
"family": "Pollak",
"sequence": "additional",
"affiliation": [
{
"name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA"
}
]
},
{
"given": "P.",
"family": "Fong",
"sequence": "additional",
"affiliation": [
{
"name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA"
}
]
},
{
"given": "P.",
"family": "Haluska",
"sequence": "additional",
"affiliation": [
{
"name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA"
}
]
},
{
"given": "L.",
"family": "Roberts",
"sequence": "additional",
"affiliation": [
{
"name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA"
}
]
},
{
"given": "D.",
"family": "Chainese",
"sequence": "additional",
"affiliation": [
{
"name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA"
}
]
},
{
"given": "L.",
"family": "Terstappen",
"sequence": "additional",
"affiliation": [
{
"name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA"
}
]
},
{
"given": "A.",
"family": "Gualberto",
"sequence": "additional",
"affiliation": [
{
"name": "Royal Marsden Hospital, Surrey, United Kingdom; Mayo Clinic, Rochester, MN; McGill University & Lady Davis Research Institute, Montreal, PQ, Canada; Pfizer Global Research & Development, New London, CT; Immunicon Corporation, Huntingdon Valley, PA"
}
]
}
],
"member": "233",
"container-title": [
"Journal of Clinical Oncology"
],
"original-title": [],
"language": "en",
"deposited": {
"date-parts": [
[
2020,
3,
6
]
],
"date-time": "2020-03-06T20:51:03Z",
"timestamp": 1583527863000
},
"score": 1,
"resource": {
"primary": {
"URL": "http://ascopubs.org/doi/10.1200/jco.2007.25.18_suppl.3507"
}
},
"subtitle": [],
"short-title": [],
"issued": {
"date-parts": [
[
2007,
6,
20
]
]
},
"references-count": 0,
"journal-issue": {
"issue": "18_suppl",
"published-print": {
"date-parts": [
[
2007,
6,
20
]
]
}
},
"alternative-id": [
"10.1200/jco.2007.25.18_suppl.3507"
],
"URL": "http://dx.doi.org/10.1200/jco.2007.25.18_suppl.3507",
"relation": {},
"ISSN": [
"0732-183X",
"1527-7755"
],
"issn-type": [
{
"value": "0732-183X",
"type": "print"
},
{
"value": "1527-7755",
"type": "electronic"
}
],
"subject": [],
"published": {
"date-parts": [
[
2007,
6,
20
]
]
}
}

View File

@ -0,0 +1,157 @@
<PubmedArticle>
<MedlineCitation Status="MEDLINE" IndexingMethod="Curated" Owner="NLM">
<PMID Version="1">37318999</PMID>
<DateCompleted>
<Year>2024</Year>
<Month>02</Month>
<Day>09</Day>
</DateCompleted>
<DateRevised>
<Year>2024</Year>
<Month>02</Month>
<Day>09</Day>
</DateRevised>
<Article PubModel="Print-Electronic">
<Journal>
<ISSN IssnType="Electronic">1522-1229</ISSN>
<JournalIssue CitedMedium="Internet">
<Volume>47</Volume>
<Issue>3</Issue>
<PubDate>
<Year>2023</Year>
<Month>Sep</Month>
<Day>01</Day>
</PubDate>
</JournalIssue>
<Title>Advances in physiology education</Title>
<ISOAbbreviation>Adv Physiol Educ</ISOAbbreviation>
</Journal>
<ArticleTitle>Providing the choice of in-person or videoconference attendance in a clinical physiology course may harm learning outcomes for the entire cohort.</ArticleTitle>
<Pagination>
<MedlinePgn>548-556</MedlinePgn>
</Pagination>
<ELocationID EIdType="doi" ValidYN="Y">10.1152/advan.00160.2022</ELocationID>
<Abstract>
<AbstractText>Clinical Physiology 1 and 2 are flipped classes in which students watch prerecorded videos before class. During the 3-h class, students take practice assessments, work in groups on critical thinking exercises, work through case studies, and engage in drawing exercises. Due to the COVID pandemic, these courses were transitioned from in-person classes to online classes. Despite the university's return-to-class policy, some students were reluctant to return to in-person classes; therefore during the 2021-2022 academic year, Clinical Physiology 1 and 2 were offered as flipped, hybrid courses. In a hybrid format, students either attended the synchronous class in person or online. Here we evaluate the learning outcomes and the perceptions of the learning experience for students who attended Clinical Physiology 1 and 2 either online (2020-2021) or in a hybrid format (2021-2022). In addition to exam scores, in-class surveys and end of course evaluations were compiled to describe the student experience in the flipped hybrid setting. Retrospective linear mixed-model regression analysis of exam scores revealed that a hybrid modality (2021-2022) was associated with lower exam scores when controlling for sex, graduate/undergraduate status, delivery method, and the order in which the courses were taken (<i>F</i> test: <i>F</i> = 8.65, df1 = 2, df2 = 179.28, <i>P</i> = 0.0003). In addition, being a Black Indigenous Person of Color (BIPOC) student is associated with a lower exam score, controlling for the same previous factors (<i>F</i> test: <i>F</i> = 4.23, df1 = 1, df2 = 130.28, <i>P</i> = 0.04), albeit with lower confidence; the BIPOC representation in this sample is small (BIPOC: <i>n</i> = 144; total: <i>n</i> = 504). There is no significant interaction between the hybrid modality and race, meaning that BIPOC and White students are both negatively affected in a hybrid flipped course. Instructors should consider carefully about offering hybrid courses and build in extra student support.<b>NEW &amp; NOTEWORTHY</b> The transition from online to in-person teaching has been as challenging as the original transition to remote teaching with the onset of the pandemic. Since not all students were ready to return to the classroom, students could choose to take this course in person or online. This arrangement provided flexibility and opportunities for innovative class activities for students but introduced tradeoffs in lower test scores from the hybrid modality than fully online or fully in-person modalities.</AbstractText>
</Abstract>
<AuthorList CompleteYN="Y">
<Author ValidYN="Y">
<LastName>Anderson</LastName>
<ForeName>Lisa Carney</ForeName>
<Initials>LC</Initials>
<Identifier Source="ORCID">0000-0003-2261-1921</Identifier>
<AffiliationInfo>
<Affiliation>Department of Integrative Biology and Physiology, University of Minnesota, Minneapolis, Minnesota, United States.</Affiliation>
<Identifier Source="ROR">https://ror.org/017zqws13</Identifier>
</AffiliationInfo>
</Author>
<Author ValidYN="Y">
<LastName>Jacobson</LastName>
<ForeName>Tate</ForeName>
<Initials>T</Initials>
<AffiliationInfo>
<Affiliation>Department of Statistics, University of Minnesota, Minneapolis, Minnesota, United States.</Affiliation>
</AffiliationInfo>
</Author>
</AuthorList>
<Language>eng</Language>
<PublicationTypeList>
<PublicationType UI="D016428">Journal Article</PublicationType>
</PublicationTypeList>
<ArticleDate DateType="Electronic">
<Year>2023</Year>
<Month>06</Month>
<Day>15</Day>
</ArticleDate>
</Article>
<MedlineJournalInfo>
<Country>United States</Country>
<MedlineTA>Adv Physiol Educ</MedlineTA>
<NlmUniqueID>100913944</NlmUniqueID>
<ISSNLinking>1043-4046</ISSNLinking>
</MedlineJournalInfo>
<CitationSubset>IM</CitationSubset>
<MeshHeadingList>
<MeshHeading>
<DescriptorName UI="D010827" MajorTopicYN="Y">Physiology</DescriptorName>
<QualifierName UI="Q000193" MajorTopicYN="N">education</QualifierName>
</MeshHeading>
<MeshHeading>
<DescriptorName UI="D012189" MajorTopicYN="N">Retrospective Studies</DescriptorName>
</MeshHeading>
<MeshHeading>
<DescriptorName UI="D007858" MajorTopicYN="N">Learning</DescriptorName>
</MeshHeading>
<MeshHeading>
<DescriptorName UI="D058873" MajorTopicYN="N">Pandemics</DescriptorName>
</MeshHeading>
<MeshHeading>
<DescriptorName UI="D000086382" MajorTopicYN="N">COVID-19</DescriptorName>
</MeshHeading>
<MeshHeading>
<DescriptorName UI="D012044" MajorTopicYN="N">Regression Analysis</DescriptorName>
</MeshHeading>
<MeshHeading>
<DescriptorName UI="D013334" MajorTopicYN="N">Students</DescriptorName>
</MeshHeading>
<MeshHeading>
<DescriptorName UI="D006801" MajorTopicYN="N">Humans</DescriptorName>
</MeshHeading>
<MeshHeading>
<DescriptorName UI="D008297" MajorTopicYN="N">Male</DescriptorName>
</MeshHeading>
<MeshHeading>
<DescriptorName UI="D005260" MajorTopicYN="N">Female</DescriptorName>
</MeshHeading>
<MeshHeading>
<DescriptorName UI="D044465" MajorTopicYN="N">White People</DescriptorName>
</MeshHeading>
<MeshHeading>
<DescriptorName UI="D044383" MajorTopicYN="N">Black People</DescriptorName>
</MeshHeading>
<MeshHeading>
<DescriptorName UI="D020375" MajorTopicYN="N">Education, Distance</DescriptorName>
</MeshHeading>
<MeshHeading>
<DescriptorName UI="D003479" MajorTopicYN="N">Curriculum</DescriptorName>
</MeshHeading>
</MeshHeadingList>
<KeywordList Owner="NOTNLM">
<Keyword MajorTopicYN="N">flipped teaching</Keyword>
<Keyword MajorTopicYN="N">hybrid teaching</Keyword>
<Keyword MajorTopicYN="N">inequity</Keyword>
<Keyword MajorTopicYN="N">learning outcomes</Keyword>
<Keyword MajorTopicYN="N">responsive teaching</Keyword>
</KeywordList>
</MedlineCitation>
<PubmedData>
<History>
<PubMedPubDate PubStatus="medline">
<Year>2023</Year>
<Month>7</Month>
<Day>21</Day>
<Hour>6</Hour>
<Minute>44</Minute>
</PubMedPubDate>
<PubMedPubDate PubStatus="pubmed">
<Year>2023</Year>
<Month>6</Month>
<Day>15</Day>
<Hour>19</Hour>
<Minute>14</Minute>
</PubMedPubDate>
<PubMedPubDate PubStatus="entrez">
<Year>2023</Year>
<Month>6</Month>
<Day>15</Day>
<Hour>12</Hour>
<Minute>53</Minute>
</PubMedPubDate>
</History>
<PublicationStatus>ppublish</PublicationStatus>
<ArticleIdList>
<ArticleId IdType="pubmed">37318999</ArticleId>
<ArticleId IdType="doi">10.1152/advan.00160.2022</ArticleId>
</ArticleIdList>
</PubmedData>
</PubmedArticle>

View File

@ -3,12 +3,15 @@ package eu.dnetlib.dhp.collection.crossref
import com.fasterxml.jackson.databind.ObjectMapper
import eu.dnetlib.dhp.aggregation.AbstractVocabularyTest
import eu.dnetlib.dhp.collection.crossref.Crossref2Oaf.TransformationType
import eu.dnetlib.dhp.schema.oaf.Publication
import org.apache.commons.io.IOUtils
import org.junit.jupiter.api.{BeforeEach, Test}
import org.junit.jupiter.api.{Assertions, BeforeEach, Test}
import org.junit.jupiter.api.extension.ExtendWith
import org.mockito.junit.jupiter.MockitoExtension
import org.slf4j.{Logger, LoggerFactory}
import scala.collection.JavaConverters.asScalaBufferConverter
@ExtendWith(Array(classOf[MockitoExtension]))
class CrossrefMappingTest extends AbstractVocabularyTest {
@ -25,8 +28,32 @@ class CrossrefMappingTest extends AbstractVocabularyTest {
val input =
IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/collection/crossref/issn_pub.json"), "utf-8")
println(Crossref2Oaf.convert(input, vocabularies, TransformationType.All))
Crossref2Oaf
.convert(input, vocabularies, TransformationType.All)
.foreach(record => {
Assertions.assertNotNull(record)
})
}
@Test
def mappingAffiliation(): Unit = {
val input =
IOUtils.toString(
getClass.getResourceAsStream("/eu/dnetlib/dhp/collection/crossref/affiliationTest.json"),
"utf-8"
)
val data = Crossref2Oaf.convert(input, vocabularies, TransformationType.OnlyResult)
data.foreach(record => {
Assertions.assertNotNull(record)
Assertions.assertTrue(record.isInstanceOf[Publication])
val publication = record.asInstanceOf[Publication]
publication.getAuthor.asScala.foreach(author => {
Assertions.assertNotNull(author.getRawAffiliationString)
Assertions.assertTrue(author.getRawAffiliationString.size() > 0)
})
})
println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(data.head))
}
}

View File

@ -5,7 +5,10 @@ import eu.dnetlib.dhp.aggregation.AbstractVocabularyTest
import eu.dnetlib.dhp.schema.oaf.utils.PidType
import eu.dnetlib.dhp.schema.oaf.{Oaf, Publication, Relation, Result}
import eu.dnetlib.dhp.sx.bio.BioDBToOAF.ScholixResolved
import eu.dnetlib.dhp.sx.bio.pubmed.{PMArticle, PMParser, PMSubject, PubMedToOaf}
import eu.dnetlib.dhp.sx.bio.ebi.SparkCreatePubmedDump
import eu.dnetlib.dhp.sx.bio.pubmed._
import org.apache.commons.io.IOUtils
import org.apache.spark.sql.SparkSession
import org.json4s.DefaultFormats
import org.json4s.JsonAST.{JField, JObject, JString}
import org.json4s.jackson.JsonMethods.parse
@ -13,14 +16,16 @@ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.extension.ExtendWith
import org.junit.jupiter.api.{BeforeEach, Test}
import org.mockito.junit.jupiter.MockitoExtension
import org.slf4j.LoggerFactory
import java.io.{BufferedReader, InputStream, InputStreamReader}
import java.util.regex.Pattern
import java.util.zip.GZIPInputStream
import javax.xml.stream.XMLInputFactory
import scala.collection.JavaConverters._
import scala.collection.mutable
import scala.collection.mutable.ListBuffer
import scala.io.Source
import scala.xml.pull.XMLEventReader
@ExtendWith(Array(classOf[MockitoExtension]))
class BioScholixTest extends AbstractVocabularyTest {
@ -48,6 +53,76 @@ class BioScholixTest extends AbstractVocabularyTest {
}
}
@Test
def testPid(): Unit = {
val pids = List(
"0000000163025705",
"000000018494732X",
"0000000308873343",
"0000000335964515",
"0000000333457333",
"0000000335964515",
"0000000302921949",
"http://orcid.org/0000-0001-8567-3543",
"http://orcid.org/0000-0001-7868-8528",
"0000-0001-9189-1440",
"0000-0003-3727-9247",
"0000-0001-7246-1058",
"000000033962389X",
"0000000330371470",
"0000000171236123",
"0000000272569752",
"0000000293231371",
"http://orcid.org/0000-0003-3345-7333",
"0000000340145688",
"http://orcid.org/0000-0003-4894-1689"
)
pids.foreach(pid => {
val pidCleaned = new PMIdentifier(pid, "ORCID").getPid
// assert pid is in the format of ORCID
println(pidCleaned)
assertTrue(pidCleaned.matches("[0-9]{4}-[0-9]{4}-[0-9]{4}-[0-9]{3}[0-9X]"))
})
}
def extractAffiliation(s: String): List[String] = {
val regex: String = "<Affiliation>(.*)<\\/Affiliation>"
val pattern = Pattern.compile(regex, Pattern.MULTILINE)
val matcher = pattern.matcher(s)
val l: mutable.ListBuffer[String] = mutable.ListBuffer()
while (matcher.find()) {
l += matcher.group(1)
}
l.toList
}
case class AuthorPID(pidType: String, pid: String) {}
def extractAuthorIdentifier(s: String): List[AuthorPID] = {
val regex: String = "<Identifier Source=\"(.*)\">(.*)<\\/Identifier>"
val pattern = Pattern.compile(regex, Pattern.MULTILINE)
val matcher = pattern.matcher(s)
val l: mutable.ListBuffer[AuthorPID] = mutable.ListBuffer()
while (matcher.find()) {
l += AuthorPID(pidType = matcher.group(1), pid = matcher.group(2))
}
l.toList
}
@Test
def testParsingPubmed2(): Unit = {
val mapper = new ObjectMapper()
val xml = IOUtils.toString(getClass.getResourceAsStream("/eu/dnetlib/dhp/sx/graph/bio/single_pubmed.xml"))
val parser = new PMParser2()
val article = parser.parse(xml)
// println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(article))
println(mapper.writerWithDefaultPrettyPrinter().writeValueAsString(PubMedToOaf.convert(article, vocabularies)))
}
@Test
def testEBIData() = {
val inputFactory = XMLInputFactory.newInstance
@ -124,6 +199,14 @@ class BioScholixTest extends AbstractVocabularyTest {
}
}
def testPubmedSplitting(): Unit = {
val spark: SparkSession = SparkSession.builder().appName("test").master("local").getOrCreate()
new SparkCreatePubmedDump("", Array.empty, LoggerFactory.getLogger(getClass))
.createPubmedDump(spark, "/home/sandro/Downloads/pubmed", "/home/sandro/Downloads/pubmed_mapped", vocabularies)
}
@Test
def testPubmedOriginalID(): Unit = {
val article: PMArticle = new PMArticle

View File

@ -63,6 +63,7 @@
<path start="copy_software"/>
<path start="copy_datasource"/>
<path start="copy_project"/>
<path start="copy_person"/>
<path start="copy_organization"/>
</fork>
@ -120,6 +121,15 @@
<error to="Kill"/>
</action>
<action name="copy_person">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<arg>${nameNode}/${sourcePath}/person</arg>
<arg>${nameNode}/${outputPath}/person</arg>
</distcp>
<ok to="wait"/>
<error to="Kill"/>
</action>
<action name="copy_datasource">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<arg>${nameNode}/${sourcePath}/datasource</arg>

View File

@ -2,14 +2,13 @@
package eu.dnetlib.dhp.oa.dedup;
import java.util.*;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.commons.beanutils.BeanUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.FlatMapGroupsFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.ReduceFunction;
import org.apache.spark.sql.*;
import eu.dnetlib.dhp.oa.dedup.model.Identifier;
@ -107,6 +106,8 @@ public class DedupRecordFactory {
final HashSet<String> acceptanceDate = new HashSet<>();
boolean isVisible = false;
while (it.hasNext()) {
Tuple3<String, String, OafEntity> t = it.next();
OafEntity entity = t._3();
@ -114,6 +115,7 @@ public class DedupRecordFactory {
if (entity == null) {
aliases.add(t._2());
} else {
isVisible = isVisible || !entity.getDataInfo().getInvisible();
cliques.add(entity);
if (acceptanceDate.size() < MAX_ACCEPTANCE_DATE) {
@ -129,13 +131,20 @@ public class DedupRecordFactory {
}
if (acceptanceDate.size() >= MAX_ACCEPTANCE_DATE || cliques.isEmpty()) {
if (!isVisible || acceptanceDate.size() >= MAX_ACCEPTANCE_DATE || cliques.isEmpty()) {
return Collections.emptyIterator();
}
OafEntity mergedEntity = MergeUtils.mergeGroup(dedupId, cliques.iterator());
OafEntity mergedEntity = MergeUtils.mergeGroup(cliques.iterator());
// dedup records do not have date of transformation attribute
mergedEntity.setDateoftransformation(null);
mergedEntity
.setMergedIds(
Stream
.concat(cliques.stream().map(OafEntity::getId), aliases.stream())
.distinct()
.sorted()
.collect(Collectors.toList()));
return Stream
.concat(

View File

@ -91,7 +91,6 @@ public class SparkBlockStats extends AbstractSparkAction {
.read()
.textFile(DedupUtility.createEntityPath(graphBasePath, subEntity))
.transform(deduper.model().parseJsonDataset())
.transform(deduper.filterAndCleanup())
.transform(deduper.generateClustersWithCollect())
.filter(functions.size(new Column("block")).geq(1));

View File

@ -5,11 +5,11 @@ import static eu.dnetlib.dhp.schema.common.ModelConstants.DNET_PROVENANCE_ACTION
import static eu.dnetlib.dhp.schema.common.ModelConstants.PROVENANCE_DEDUP;
import java.io.IOException;
import java.util.Arrays;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.*;
import org.dom4j.DocumentException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -17,6 +17,7 @@ import org.xml.sax.SAXException;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelConstants;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.OafEntity;
@ -25,6 +26,8 @@ import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import eu.dnetlib.pace.config.DedupConfig;
import scala.collection.JavaConversions;
import scala.collection.JavaConverters;
public class SparkCreateDedupRecord extends AbstractSparkAction {
@ -85,6 +88,36 @@ public class SparkCreateDedupRecord extends AbstractSparkAction {
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath);
log.info("Updating mergerels for: '{}'", subEntity);
final Dataset<Row> dedupIds = spark
.read()
.schema("`id` STRING, `mergedIds` ARRAY<STRING>")
.json(outputPath)
.selectExpr("id as source", "explode(mergedIds) as target");
spark
.read()
.load(mergeRelPath)
.where("relClass == 'merges'")
.join(dedupIds, JavaConversions.asScalaBuffer(Arrays.asList("source", "target")), "left_semi")
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.save(workingPath + "/mergerel_filtered");
final Dataset<Row> validRels = spark.read().load(workingPath + "/mergerel_filtered");
final Dataset<Row> filteredMergeRels = validRels
.union(
validRels
.withColumnRenamed("source", "source_tmp")
.withColumnRenamed("target", "target_tmp")
.withColumn("relClass", functions.lit(ModelConstants.IS_MERGED_IN))
.withColumnRenamed("target_tmp", "source")
.withColumnRenamed("source_tmp", "target"));
saveParquet(filteredMergeRels, mergeRelPath, SaveMode.Overwrite);
removeOutputDir(spark, workingPath + "/mergerel_filtered");
}
}

View File

@ -69,6 +69,7 @@ public class SparkPropagateRelation extends AbstractSparkAction {
Dataset<Relation> mergeRels = spark
.read()
.schema(REL_BEAN_ENC.schema())
.load(DedupUtility.createMergeRelPath(workingPath, "*", "*"))
.as(REL_BEAN_ENC);

View File

@ -46,8 +46,8 @@ class DatasetMergerTest implements Serializable {
}
@Test
void datasetMergerTest() throws InstantiationException, IllegalAccessException, InvocationTargetException {
Dataset pub_merged = MergeUtils.mergeGroup(dedupId, datasets.stream().map(Tuple2::_2).iterator());
void datasetMergerTest() {
Dataset pub_merged = MergeUtils.mergeGroup(datasets.stream().map(Tuple2::_2).iterator());
// verify id
assertEquals(dedupId, pub_merged.getId());

View File

@ -96,7 +96,7 @@
"aggregation": "MAX",
"positive": "layer4",
"negative": "NO_MATCH",
"undefined": "MATCH",
"undefined": "layer4",
"ignoreUndefined": "true"
},
"layer4": {

View File

@ -560,9 +560,32 @@ case object Crossref2Oaf {
"10.13039/501100000266" | "10.13039/501100006041" | "10.13039/501100000265" | "10.13039/501100000270" |
"10.13039/501100013589" | "10.13039/501100000271" =>
generateSimpleRelationFromAward(funder, "ukri________", a => a)
//DFG
case "10.13039/501100001659" =>
val targetId = getProjectId("dfgf________", "1e5e62235d094afd01cd56e65112fc63")
queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY)
queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES)
case _ => logger.debug("no match for " + funder.DOI.get)
//Add for Danish funders
//Independent Research Fund Denmark (IRFD)
case "10.13039/501100004836" =>
generateSimpleRelationFromAward(funder, "irfd________", a => a)
val targetId = getProjectId("irfd________", "1e5e62235d094afd01cd56e65112fc63")
queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY)
queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES)
//Carlsberg Foundation (CF)
case "10.13039/501100002808" =>
generateSimpleRelationFromAward(funder, "cf__________", a => a)
val targetId = getProjectId("cf__________", "1e5e62235d094afd01cd56e65112fc63")
queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY)
queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES)
//Novo Nordisk Foundation (NNF)
case "10.13039/501100009708" =>
generateSimpleRelationFromAward(funder, "nnf___________", a => a)
val targetId = getProjectId("nnf_________", "1e5e62235d094afd01cd56e65112fc63")
queue += generateRelation(sourceId, targetId, ModelConstants.IS_PRODUCED_BY)
queue += generateRelation(targetId, sourceId, ModelConstants.PRODUCES)
case _ => logger.debug("no match for " + funder.DOI.get)
}
} else {

View File

@ -48,12 +48,7 @@
<groupId>io.github.classgraph</groupId>
<artifactId>classgraph</artifactId>
</dependency>
<dependency>
<groupId>eu.dnetlib.dhp</groupId>
<artifactId>dhp-aggregation</artifactId>
<version>1.2.5-SNAPSHOT</version>
<scope>compile</scope>
</dependency>
</dependencies>

View File

@ -171,7 +171,7 @@ public class Utils implements Serializable {
public static List<String> getCommunityIdList(String baseURL) throws IOException {
return getValidCommunities(baseURL)
.stream()
.map(community -> community.getId())
.map(CommunityModel::getId)
.collect(Collectors.toList());
}

View File

@ -13,13 +13,13 @@ public class CommunityContentprovider {
private String openaireId;
private SelectionConstraints selectioncriteria;
private String enabled;
private Boolean enabled;
public String getEnabled() {
public Boolean getEnabled() {
return enabled;
}
public void setEnabled(String enabled) {
public void setEnabled(Boolean enabled) {
this.enabled = enabled;
}

View File

@ -130,6 +130,7 @@ public class ResultTagger implements Serializable {
// log.info("Remove constraints for " + communityId);
if (conf.getRemoveConstraintsMap().keySet().contains(communityId) &&
conf.getRemoveConstraintsMap().get(communityId).getCriteria() != null &&
!conf.getRemoveConstraintsMap().get(communityId).getCriteria().isEmpty() &&
conf
.getRemoveConstraintsMap()
.get(communityId)
@ -161,29 +162,30 @@ public class ResultTagger implements Serializable {
// Tagging for datasource
final Set<String> datasources = new HashSet<>();
final Set<String> collfrom = new HashSet<>();
final Set<String> cfhb = new HashSet<>();
final Set<String> hostdby = new HashSet<>();
if (Objects.nonNull(result.getInstance())) {
for (Instance i : result.getInstance()) {
if (Objects.nonNull(i.getCollectedfrom()) && Objects.nonNull(i.getCollectedfrom().getKey())) {
collfrom.add(i.getCollectedfrom().getKey());
cfhb.add(i.getCollectedfrom().getKey());
}
if (Objects.nonNull(i.getHostedby()) && Objects.nonNull(i.getHostedby().getKey())) {
cfhb.add(i.getHostedby().getKey());
hostdby.add(i.getHostedby().getKey());
}
}
collfrom
cfhb
.forEach(
dsId -> datasources
.addAll(
conf.getCommunityForDatasource(dsId, param)));
hostdby.forEach(dsId -> {
datasources
.addAll(
conf.getCommunityForDatasource(dsId, param));
// datasources
// .addAll(
// conf.getCommunityForDatasource(dsId, param));
if (conf.isEoscDatasource(dsId)) {
datasources.add("eosc");
}
@ -226,6 +228,7 @@ public class ResultTagger implements Serializable {
.forEach(communityId -> {
if (!removeCommunities.contains(communityId) &&
conf.getSelectionConstraintsMap().get(communityId).getCriteria() != null &&
!conf.getSelectionConstraintsMap().get(communityId).getCriteria().isEmpty() &&
conf
.getSelectionConstraintsMap()
.get(communityId)

View File

@ -33,6 +33,8 @@ public class SelectionConstraints implements Serializable {
// Constraints in or
public boolean verifyCriteria(final Map<String, List<String>> param) {
if (criteria.isEmpty())
return true;
for (Constraints selc : criteria) {
if (selc.verifyCriteria(param)) {
return true;

View File

@ -0,0 +1,302 @@
package eu.dnetlib.dhp.person;
import static com.ibm.icu.text.PluralRules.Operand.w;
import static eu.dnetlib.dhp.PropagationConstant.*;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.Serializable;
import java.util.*;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.MapGroupsFunction;
import org.apache.spark.sql.*;
import org.apache.spark.sql.Dataset;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.common.person.CoAuthorshipIterator;
import eu.dnetlib.dhp.common.person.Coauthors;
import eu.dnetlib.dhp.countrypropagation.SparkCountryPropagationJob;
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.IdentifierFactory;
import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils;
import scala.Tuple2;
public class SparkExtractPersonRelations {
private static final Logger log = LoggerFactory.getLogger(SparkCountryPropagationJob.class);
private static final String PERSON_PREFIX = ModelSupport.getIdPrefix(Person.class) + "|orcid_______";
public static final DataInfo DATAINFO = OafMapperUtils
.dataInfo(
false,
"openaire",
true,
false,
OafMapperUtils
.qualifier(
ModelConstants.SYSIMPORT_CROSSWALK_REPOSITORY,
ModelConstants.SYSIMPORT_CROSSWALK_REPOSITORY,
ModelConstants.DNET_PROVENANCE_ACTIONS,
ModelConstants.DNET_PROVENANCE_ACTIONS),
"0.85");
public static void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils
.toString(
SparkCountryPropagationJob.class
.getResourceAsStream(
"/eu/dnetlib/dhp/wf/subworkflows/person/input_personpropagation_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
Boolean isSparkSessionManaged = isSparkSessionManaged(parser);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
String sourcePath = parser.get("sourcePath");
log.info("sourcePath: {}", sourcePath);
final String workingPath = parser.get("outputPath");
log.info("workingPath: {}", workingPath);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
extractRelations(
spark,
sourcePath,
workingPath);
removeIsolatedPerson(spark, sourcePath, workingPath);
});
}
private static void removeIsolatedPerson(SparkSession spark, String sourcePath, String workingPath) {
Dataset<Person> personDataset = spark
.read()
.schema(Encoders.bean(Person.class).schema())
.json(sourcePath + "person")
.as(Encoders.bean(Person.class));
Dataset<Relation> relationDataset = spark
.read()
.schema(Encoders.bean(Relation.class).schema())
.json(sourcePath + "relation")
.as(Encoders.bean(Relation.class));
personDataset
.join(relationDataset, personDataset.col("id").equalTo(relationDataset.col("source")), "left_semi")
.write()
.option("compression", "gzip")
.mode(SaveMode.Overwrite)
.json(workingPath + "person");
spark
.read()
.schema(Encoders.bean(Person.class).schema())
.json(workingPath + "person")
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(sourcePath + "person");
}
private static void extractRelations(SparkSession spark, String sourcePath, String workingPath) {
Dataset<Tuple2<String, Relation>> relationDataset = spark
.read()
.schema(Encoders.bean(Relation.class).schema())
.json(sourcePath + "relation")
.as(Encoders.bean(Relation.class))
.map(
(MapFunction<Relation, Tuple2<String, Relation>>) r -> new Tuple2<>(
r.getSource() + r.getRelClass() + r.getTarget(), r),
Encoders.tuple(Encoders.STRING(), Encoders.bean(Relation.class)));
ModelSupport.entityTypes
.keySet()
.stream()
.filter(ModelSupport::isResult)
.forEach(
e -> {
// 1. search for results having orcid_pending and orcid in the set of pids for the authors
Dataset<Result> resultWithOrcids = spark
.read()
.schema(Encoders.bean(Result.class).schema())
.json(sourcePath + e.name())
.as(Encoders.bean(Result.class))
.filter(
(FilterFunction<Result>) r -> !r.getDataInfo().getDeletedbyinference() &&
!r.getDataInfo().getInvisible() &&
Optional
.ofNullable(r.getAuthor())
.isPresent())
.filter(
(FilterFunction<Result>) r -> r
.getAuthor()
.stream()
.anyMatch(
a -> Optional
.ofNullable(
a
.getPid())
.isPresent() &&
a
.getPid()
.stream()
.anyMatch(
p -> Arrays
.asList("orcid", "orcid_pending")
.contains(p.getQualifier().getClassid().toLowerCase()))));
// 2. create authorship relations between the result identifier and the person entity with
// orcid_pending.
Dataset<Tuple2<String, Relation>> newRelations = resultWithOrcids
.flatMap(
(FlatMapFunction<Result, Relation>) r -> getAuthorshipRelations(r),
Encoders.bean(Relation.class))
// .groupByKey((MapFunction<Relation, String>) r-> r.getSource()+r.getTarget(), Encoders.STRING() )
// .mapGroups((MapGroupsFunction<String, Relation, Relation>) (k,it) -> it.next(), Encoders.bean(Relation.class) )
.map(
(MapFunction<Relation, Tuple2<String, Relation>>) r -> new Tuple2<>(
r.getSource() + r.getRelClass() + r.getTarget(), r),
Encoders.tuple(Encoders.STRING(), Encoders.bean(Relation.class)));
newRelations
.joinWith(relationDataset, newRelations.col("_1").equalTo(relationDataset.col("_1")), "left")
.map((MapFunction<Tuple2<Tuple2<String, Relation>, Tuple2<String, Relation>>, Relation>) t2 -> {
if (t2._2() == null)
return t2._1()._2();
return null;
}, Encoders.bean(Relation.class))
.filter((FilterFunction<Relation>) r -> r != null)
.write()
.mode(SaveMode.Append)
.option("compression", "gzip")
.json(workingPath);
// 2.1 store in a separate location the relation between the person and the pids for the result?
// 3. create co_authorship relations between the pairs of authors with orcid/orcid_pending pids
newRelations = resultWithOrcids
.map((MapFunction<Result, Coauthors>) r -> getAuthorsPidList(r), Encoders.bean(Coauthors.class))
.flatMap(
(FlatMapFunction<Coauthors, Relation>) c -> new CoAuthorshipIterator(c.getCoauthors()),
Encoders.bean(Relation.class))
.groupByKey(
(MapFunction<Relation, String>) r -> r.getSource() + r.getTarget(), Encoders.STRING())
.mapGroups(
(MapGroupsFunction<String, Relation, Relation>) (k, it) -> it.next(),
Encoders.bean(Relation.class))
.map(
(MapFunction<Relation, Tuple2<String, Relation>>) r -> new Tuple2<>(
r.getSource() + r.getRelClass() + r.getTarget(), r),
Encoders.tuple(Encoders.STRING(), Encoders.bean(Relation.class)));
newRelations
.joinWith(relationDataset, newRelations.col("_1").equalTo(relationDataset.col("_1")), "left")
.map((MapFunction<Tuple2<Tuple2<String, Relation>, Tuple2<String, Relation>>, Relation>) t2 -> {
if (t2._2() == null)
return t2._1()._2();
return null;
}, Encoders.bean(Relation.class))
.filter((FilterFunction<Relation>) r -> r != null)
.write()
.mode(SaveMode.Append)
.option("compression", "gzip")
.json(workingPath);
});
spark
.read()
.schema(Encoders.bean(Relation.class).schema())
.json(workingPath)
.write()
.mode(SaveMode.Append)
.option("compression", "gzip")
.json(sourcePath + "relation");
}
private static Coauthors getAuthorsPidList(Result r) {
Coauthors coauth = new Coauthors();
coauth
.setCoauthors(
r
.getAuthor()
.stream()
.filter(
a -> a
.getPid()
.stream()
.anyMatch(
p -> Arrays.asList("orcid", "orcid_pending").contains(p.getQualifier().getClassid())))
.map(a -> {
Optional<StructuredProperty> tmp = a
.getPid()
.stream()
.filter(p -> p.getQualifier().getClassid().equalsIgnoreCase("orcid"))
.findFirst();
if (tmp.isPresent())
return tmp.get().getValue();
tmp = a
.getPid()
.stream()
.filter(p -> p.getQualifier().getClassid().equalsIgnoreCase("orcid_pending"))
.findFirst();
if (tmp.isPresent())
return tmp.get().getValue();
return null;
})
.filter(Objects::nonNull)
.collect(Collectors.toList()));
return coauth;
}
private static Iterator<Relation> getAuthorshipRelations(Result r) {
List<Relation> relationList = new ArrayList<>();
for (Author a : r.getAuthor())
relationList.addAll(a.getPid().stream().map(p -> {
if (p.getQualifier().getClassid().equalsIgnoreCase("orcid_pending"))
return getRelation(p.getValue(), r.getId());
return null;
})
.filter(Objects::nonNull)
.collect(Collectors.toList()));
return relationList.iterator();
}
private static Relation getRelation(String orcid, String resultId) {
String source = PERSON_PREFIX + "::" + IdentifierFactory.md5(orcid);
Relation relation = OafMapperUtils
.getRelation(
source, resultId, ModelConstants.RESULT_PERSON_RELTYPE,
ModelConstants.RESULT_PERSON_SUBRELTYPE,
ModelConstants.RESULT_PERSON_HASAUTHORED,
null, // collectedfrom = null
DATAINFO,
null);
return relation;
}
}

View File

@ -1,11 +1,14 @@
package eu.dnetlib.dhp.resulttocommunityfromsemrel;
import static java.lang.String.join;
import static eu.dnetlib.dhp.PropagationConstant.*;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkHiveSession;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import org.apache.commons.io.IOUtils;
@ -19,6 +22,7 @@ import com.google.gson.Gson;
import eu.dnetlib.dhp.api.Utils;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList;
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.utils.ISLookupClientFactory;
@ -45,7 +49,7 @@ public class PrepareResultCommunitySetStep1 {
/**
* a dataset for example could be linked to more than one publication. For each publication linked to that dataset
* the previous query will produce a row: targetId set of community context the target could possibly inherit with
* the previous query will produce a row: targetId, set of community context the target could possibly inherit. With
* the following query there will be a single row for each result linked to more than one result of the result type
* currently being used
*/
@ -56,6 +60,27 @@ public class PrepareResultCommunitySetStep1 {
+ "where length(co) > 0 "
+ "group by resultId";
private static final String RESULT_CONTEXT_QUERY_TEMPLATE_IS_RELATED_TO = "select target as resultId, community_context "
+
"from resultWithContext rwc " +
"join relatedToRelations r " +
"join patents p " +
"on rwc.id = r.source and r.target = p.id";
private static final String RESULT_WITH_CONTEXT = "select id, collect_set(co.id) community_context \n" +
" from result " +
" lateral view explode (context) c as co " +
" where lower(co.id) IN %s" +
" group by id";
private static final String RESULT_PATENT = "select id " +
" from result " +
" where array_contains(instance.instancetype.classname, 'Patent')";
private static final String IS_RELATED_TO_RELATIONS = "select source, target " +
" from relation " +
" where lower(relClass) = 'isrelatedto' and datainfo.deletedbyinference = false";
public static void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils
.toString(
@ -82,14 +107,25 @@ public class PrepareResultCommunitySetStep1 {
SparkConf conf = new SparkConf();
conf.set("hive.metastore.uris", parser.get("hive_metastore_uris"));
final List<String> allowedsemrel = Arrays.asList(parser.get("allowedsemrels").split(";"));
log.info("allowedSemRel: {}", new Gson().toJson(allowedsemrel));
final String allowedsemrel = "(" + join(
",",
Arrays
.asList(parser.get("allowedsemrels").split(";"))
.stream()
.map(value -> "'" + value.toLowerCase() + "'")
.toArray(String[]::new))
+ ")";
log.info("allowedSemRel: {}", allowedsemrel);
final String baseURL = parser.get("baseURL");
log.info("baseURL: {}", baseURL);
final List<String> communityIdList = getCommunityList(baseURL);
log.info("communityIdList: {}", new Gson().toJson(communityIdList));
final String communityIdList = "(" + join(
",", getCommunityList(baseURL)
.stream()
.map(value -> "'" + value.toLowerCase() + "'")
.toArray(String[]::new))
+ ")";
final String resultType = resultClassName.substring(resultClassName.lastIndexOf(".") + 1).toLowerCase();
log.info("resultType: {}", resultType);
@ -118,10 +154,10 @@ public class PrepareResultCommunitySetStep1 {
SparkSession spark,
String inputPath,
String outputPath,
List<String> allowedsemrel,
String allowedsemrel,
Class<R> resultClazz,
String resultType,
List<String> communityIdList) {
String communityIdList) {
final String inputResultPath = inputPath + "/" + resultType;
log.info("Reading Graph table from: {}", inputResultPath);
@ -132,7 +168,8 @@ public class PrepareResultCommunitySetStep1 {
Dataset<Relation> relation = readPath(spark, inputRelationPath, Relation.class);
relation.createOrReplaceTempView("relation");
Dataset<R> result = readPath(spark, inputResultPath, resultClazz);
Dataset<R> result = readPath(spark, inputResultPath, resultClazz)
.where("datainfo.deletedbyinference != true AND datainfo.invisible != true");
result.createOrReplaceTempView("result");
final String outputResultPath = outputPath + "/" + resultType;
@ -141,10 +178,20 @@ public class PrepareResultCommunitySetStep1 {
String resultContextQuery = String
.format(
RESULT_CONTEXT_QUERY_TEMPLATE,
getConstraintList(" lower(co.id) = '", communityIdList),
getConstraintList(" lower(relClass) = '", allowedsemrel));
"AND lower(co.id) IN " + communityIdList,
"AND lower(relClass) IN " + allowedsemrel);
Dataset<Row> result_context = spark.sql(resultContextQuery);
Dataset<Row> rwc = spark.sql(String.format(RESULT_WITH_CONTEXT, communityIdList));
Dataset<Row> patents = spark.sql(RESULT_PATENT);
Dataset<Row> relatedToRelations = spark.sql(IS_RELATED_TO_RELATIONS);
rwc.createOrReplaceTempView("resultWithContext");
patents.createOrReplaceTempView("patents");
relatedToRelations.createOrReplaceTempView("relatedTorelations");
result_context = result_context.unionAll(spark.sql(RESULT_CONTEXT_QUERY_TEMPLATE_IS_RELATED_TO));
result_context.createOrReplaceTempView("result_context");
spark
@ -152,8 +199,9 @@ public class PrepareResultCommunitySetStep1 {
.as(Encoders.bean(ResultCommunityList.class))
.write()
.option("compression", "gzip")
.mode(SaveMode.Overwrite)
.mode(SaveMode.Append)
.json(outputResultPath);
}
public static List<String> getCommunityList(final String baseURL) throws IOException {

View File

@ -4,6 +4,7 @@ package eu.dnetlib.dhp.resulttocommunityfromsemrel;
import static eu.dnetlib.dhp.PropagationConstant.*;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.Set;
@ -76,22 +77,13 @@ public class PrepareResultCommunitySetStep2 {
if (b == null) {
return a;
}
Set<String> community_set = new HashSet<>();
a.getCommunityList().stream().forEach(aa -> community_set.add(aa));
b
.getCommunityList()
.stream()
.forEach(
aa -> {
if (!community_set.contains(aa)) {
a.getCommunityList().add(aa);
community_set.add(aa);
}
});
Set<String> community_set = new HashSet<>(a.getCommunityList());
community_set.addAll(b.getCommunityList());
a.setCommunityList(new ArrayList<>(community_set));
return a;
})
.map(Tuple2::_2)
.map(r -> OBJECT_MAPPER.writeValueAsString(r))
.map(OBJECT_MAPPER::writeValueAsString)
.saveAsTextFile(outputPath, GzipCodec.class);
}

View File

@ -8,3 +8,4 @@ result_project classpath eu/dnetlib/dhp/wf/subworkflows/projecttoresult/oozie_ap
community_project classpath eu/dnetlib/dhp/wf/subworkflows/resulttocommunityfromproject/oozie_app
community_sem_rel classpath eu/dnetlib/dhp/wf/subworkflows/resulttocommunityfromsemrel/oozie_app
country_propagation classpath eu/dnetlib/dhp/wf/subworkflows/countrypropagation/oozie_app
person_propagation classpath eu/dnetlib/dhp/wf/subworkflows/person/oozie_app

View File

@ -122,6 +122,7 @@
<case to="community_project">${wf:conf('resumeFrom') eq 'CommunityProject'}</case>
<case to="community_sem_rel">${wf:conf('resumeFrom') eq 'CommunitySemanticRelation'}</case>
<case to="country_propagation">${wf:conf('resumeFrom') eq 'CountryPropagation'}</case>
<case to="person_propagation">${wf:conf('resumeFrom') eq 'PersonPropagation'}</case>
<default to="orcid_propagation"/>
</switch>
</decision>
@ -291,10 +292,24 @@
</property>
</configuration>
</sub-workflow>
<ok to="person_propagation" />
<error to="Kill" />
</action>
<action name="person_propagation">
<sub-workflow>
<app-path>${wf:appPath()}/person_propagation
</app-path>
<propagate-configuration/>
<configuration>
<property>
<name>sourcePath</name>
<value>${outputPath}</value>
</property>
</configuration>
</sub-workflow>
<ok to="country_propagation" />
<error to="Kill" />
</action>
<action name="country_propagation">
<sub-workflow>
<app-path>${wf:appPath()}/country_propagation
@ -319,6 +334,8 @@
<error to="Kill" />
</action>
<end name="End"/>
</workflow-app>

View File

@ -34,6 +34,7 @@
<path start="copy_organization"/>
<path start="copy_projects"/>
<path start="copy_datasources"/>
<path start="copy_persons"/>
</fork>
<action name="copy_relation">
@ -80,6 +81,17 @@
<error to="Kill"/>
</action>
<action name="copy_persons">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<arg>${nameNode}/${sourcePath}/person</arg>
<arg>${nameNode}/${outputPath}/person</arg>
</distcp>
<ok to="copy_wait"/>
<error to="Kill"/>
</action>
<join name="copy_wait" to="fork_prepare_assoc_step1"/>
<fork name="fork_prepare_assoc_step1">

View File

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

View File

@ -0,0 +1 @@
sourcePath=/tmp/miriam/13_graph_copy

View File

@ -0,0 +1,58 @@
<configuration>
<property>
<name>jobTracker</name>
<value>yarnRM</value>
</property>
<property>
<name>nameNode</name>
<value>hdfs://nameservice1</value>
</property>
<property>
<name>oozie.use.system.libpath</name>
<value>true</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>spark2</value>
</property>
<property>
<name>hive_metastore_uris</name>
<value>thrift://iis-cdh5-test-m3.ocean.icm.edu.pl:9083</value>
</property>
<property>
<name>spark2YarnHistoryServerAddress</name>
<value>http://iis-cdh5-test-gw.ocean.icm.edu.pl:18089</value>
</property>
<property>
<name>spark2EventLogDir</name>
<value>/user/spark/spark2ApplicationHistory</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>sparkExecutorNumber</name>
<value>4</value>
</property>
<property>
<name>sparkDriverMemory</name>
<value>15G</value>
</property>
<property>
<name>sparkExecutorMemory</name>
<value>5G</value>
</property>
<property>
<name>sparkExecutorCores</name>
<value>4</value>
</property>
<property>
<name>spark2MaxExecutors</name>
<value>50</value>
</property>
</configuration>

View File

@ -0,0 +1,68 @@
<workflow-app name="person_propagation" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>sourcePath</name>
<description>the source path</description>
</property>
</parameters>
<global>
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<configuration>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>${oozieActionShareLibForSpark2}</value>
</property>
</configuration>
</global>
<start to="reset_outputpath"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="reset_outputpath">
<fs>
<delete path="${workingDir}"/>
<mkdir path="${workingDir}"/>
</fs>
<ok to="extract_person_relation_from_graph"/>
<error to="Kill"/>
</action>
<action name="extract_person_relation_from_graph">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>personPropagation</name>
<class>eu.dnetlib.dhp.person.SparkExtractPersonRelations</class>
<jar>dhp-enrichment-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
--executor-memory=${sparkExecutorMemory}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.speculation=false
--conf spark.hadoop.mapreduce.map.speculative=false
--conf spark.hadoop.mapreduce.reduce.speculative=false
--conf spark.sql.shuffle.partitions=7680
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}/</arg>
<arg>--outputPath</arg><arg>${workingDir}/relation</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,93 @@
package eu.dnetlib.dhp.person;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.io.FileUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
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.countrypropagation.SparkCountryPropagationJob;
import eu.dnetlib.dhp.schema.oaf.*;
import scala.Tuple2;
public class PersonPropagationJobTest {
private static final Logger log = LoggerFactory.getLogger(PersonPropagationJobTest.class);
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static SparkSession spark;
private static Path workingDir;
@BeforeAll
public static void beforeAll() throws IOException {
workingDir = Files.createTempDirectory(PersonPropagationJobTest.class.getSimpleName());
log.info("using work dir {}", workingDir);
SparkConf conf = new SparkConf();
conf.setAppName(PersonPropagationJobTest.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(PersonPropagationJobTest.class.getSimpleName())
.config(conf)
.getOrCreate();
}
@AfterAll
public static void afterAll() throws IOException {
FileUtils.deleteDirectory(workingDir.toFile());
spark.stop();
}
@Test
void testPersonPropagation() throws Exception {
final String sourcePath = getClass()
.getResource("/eu/dnetlib/dhp/personpropagation/graph")
.getPath();
SparkExtractPersonRelations
.main(
new String[] {
"--isSparkSessionManaged", Boolean.FALSE.toString(),
"--sourcePath", sourcePath,
"--outputPath", workingDir.toString()
});
final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext());
JavaRDD<Relation> tmp = sc
.textFile(workingDir.toString() + "/relation")
.map(item -> OBJECT_MAPPER.readValue(item, Relation.class));
// TODO write assertions and find relevant information for hte resource files
}
}

View File

@ -6,7 +6,9 @@ import static org.apache.spark.sql.functions.desc;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.commons.io.FileUtils;
import org.apache.spark.SparkConf;
@ -24,7 +26,9 @@ import org.slf4j.LoggerFactory;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.resulttocommunityfromorganization.ResultCommunityList;
import eu.dnetlib.dhp.schema.oaf.Dataset;
import scala.collection.Seq;
public class ResultToCommunityJobTest {
@ -271,4 +275,59 @@ public class ResultToCommunityJobTest {
.get(0)
.getString(0));
}
@Test
public void prepareStep1Test() throws Exception {
/*
* final String allowedsemrel = join(",", Arrays.stream(parser.get("allowedsemrels").split(";")) .map(value ->
* "'" + value.toLowerCase() + "'") .toArray(String[]::new)); log.info("allowedSemRel: {}", new
* Gson().toJson(allowedsemrel)); final String baseURL = parser.get("baseURL"); log.info("baseURL: {}",
* baseURL);
*/
PrepareResultCommunitySetStep1
.main(
new String[] {
"-isSparkSessionManaged", Boolean.FALSE.toString(),
"-sourcePath", getClass()
.getResource("/eu/dnetlib/dhp/resulttocommunityfromsemrel/graph")
.getPath(),
"-hive_metastore_uris", "",
"-resultTableName", "eu.dnetlib.dhp.schema.oaf.Publication",
"-outputPath", workingDir.toString() + "/preparedInfo",
"-allowedsemrels", "issupplementto;issupplementedby",
"-baseURL", "https://dev-openaire.d4science.org/openaire/community/"
});
org.apache.spark.sql.Dataset<ResultCommunityList> resultCommunityList = spark
.read()
.schema(Encoders.bean(ResultCommunityList.class).schema())
.json(workingDir.toString() + "/preparedInfo/publication")
.as(Encoders.bean(ResultCommunityList.class));
Assertions.assertEquals(2, resultCommunityList.count());
Assertions
.assertEquals(
1,
resultCommunityList.filter("resultId = '50|dedup_wf_001::06e51d2bf295531b2d2e7a1b55500783'").count());
Assertions
.assertEquals(
1,
resultCommunityList.filter("resultId = '50|pending_org_::82f63b2d21ae88596b9d8991780e9888'").count());
ArrayList<String> communities = resultCommunityList
.filter("resultId = '50|dedup_wf_001::06e51d2bf295531b2d2e7a1b55500783'")
.first()
.getCommunityList();
Assertions.assertEquals(2, communities.size());
Assertions.assertTrue(communities.stream().anyMatch(cid -> "beopen".equals(cid)));
Assertions.assertTrue(communities.stream().anyMatch(cid -> "dh-ch".equals(cid)));
communities = resultCommunityList
.filter("resultId = '50|pending_org_::82f63b2d21ae88596b9d8991780e9888'")
.first()
.getCommunityList();
Assertions.assertEquals(1, communities.size());
Assertions.assertEquals("dh-ch", communities.get(0));
}
}

File diff suppressed because one or more lines are too long

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