forked from D-Net/dnet-hadoop
implemented first version of exportation of Scholexplorer into ActionSet
This commit is contained in:
parent
ce12f236bb
commit
1d4275acc4
|
@ -1,11 +1,25 @@
|
|||
|
||||
package eu.dnetlib.dhp.schema.scholexplorer;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.KeyValue;
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation;
|
||||
|
||||
public class DLIRelation extends Relation {
|
||||
|
||||
private String dateOfCollection;
|
||||
|
||||
private List<KeyValue> collectedFrom;
|
||||
|
||||
public List<KeyValue> getCollectedFrom() {
|
||||
return collectedFrom;
|
||||
}
|
||||
|
||||
public void setCollectedFrom(List<KeyValue> collectedFrom) {
|
||||
this.collectedFrom = collectedFrom;
|
||||
}
|
||||
|
||||
public String getDateOfCollection() {
|
||||
return dateOfCollection;
|
||||
}
|
||||
|
|
|
@ -91,35 +91,29 @@ public class GenerateEventsApplication {
|
|||
private static final UpdateMatcher<Pair<Result, List<Software>>, ?> enrichMoreSoftware = new EnrichMoreSoftware();
|
||||
|
||||
private static final UpdateMatcher<Pair<Result, List<Publication>>, ?> enrichMisissingPublicationIsRelatedTo = new EnrichMissingPublicationIsRelatedTo();
|
||||
private static final UpdateMatcher<Pair<Result, List<Publication>>, ?> enrichMissingPublicationIsReferencedBy =
|
||||
new EnrichMissingPublicationIsReferencedBy();
|
||||
private static final UpdateMatcher<Pair<Result, List<Publication>>, ?> enrichMissingPublicationIsReferencedBy = new EnrichMissingPublicationIsReferencedBy();
|
||||
private static final UpdateMatcher<Pair<Result, List<Publication>>, ?> enrichMissingPublicationReferences = new EnrichMissingPublicationReferences();
|
||||
private static final UpdateMatcher<Pair<Result, List<Publication>>, ?> enrichMissingPublicationIsSupplementedTo =
|
||||
new EnrichMissingPublicationIsSupplementedTo();
|
||||
private static final UpdateMatcher<Pair<Result, List<Publication>>, ?> enrichMissingPublicationIsSupplementedBy =
|
||||
new EnrichMissingPublicationIsSupplementedBy();
|
||||
private static final UpdateMatcher<Pair<Result, List<Publication>>, ?> enrichMissingPublicationIsSupplementedTo = new EnrichMissingPublicationIsSupplementedTo();
|
||||
private static final UpdateMatcher<Pair<Result, List<Publication>>, ?> enrichMissingPublicationIsSupplementedBy = new EnrichMissingPublicationIsSupplementedBy();
|
||||
|
||||
private static final UpdateMatcher<Pair<Result, List<eu.dnetlib.dhp.schema.oaf.Dataset>>, ?> enrichMisissingDatasetIsRelatedTo =
|
||||
new EnrichMissingDatasetIsRelatedTo();
|
||||
private static final UpdateMatcher<Pair<Result, List<eu.dnetlib.dhp.schema.oaf.Dataset>>, ?> enrichMissingDatasetIsReferencedBy =
|
||||
new EnrichMissingDatasetIsReferencedBy();
|
||||
private static final UpdateMatcher<Pair<Result, List<eu.dnetlib.dhp.schema.oaf.Dataset>>, ?> enrichMissingDatasetReferences =
|
||||
new EnrichMissingDatasetReferences();
|
||||
private static final UpdateMatcher<Pair<Result, List<eu.dnetlib.dhp.schema.oaf.Dataset>>, ?> enrichMissingDatasetIsSupplementedTo =
|
||||
new EnrichMissingDatasetIsSupplementedTo();
|
||||
private static final UpdateMatcher<Pair<Result, List<eu.dnetlib.dhp.schema.oaf.Dataset>>, ?> enrichMissingDatasetIsSupplementedBy =
|
||||
new EnrichMissingDatasetIsSupplementedBy();
|
||||
private static final UpdateMatcher<Pair<Result, List<eu.dnetlib.dhp.schema.oaf.Dataset>>, ?> enrichMisissingDatasetIsRelatedTo = new EnrichMissingDatasetIsRelatedTo();
|
||||
private static final UpdateMatcher<Pair<Result, List<eu.dnetlib.dhp.schema.oaf.Dataset>>, ?> enrichMissingDatasetIsReferencedBy = new EnrichMissingDatasetIsReferencedBy();
|
||||
private static final UpdateMatcher<Pair<Result, List<eu.dnetlib.dhp.schema.oaf.Dataset>>, ?> enrichMissingDatasetReferences = new EnrichMissingDatasetReferences();
|
||||
private static final UpdateMatcher<Pair<Result, List<eu.dnetlib.dhp.schema.oaf.Dataset>>, ?> enrichMissingDatasetIsSupplementedTo = new EnrichMissingDatasetIsSupplementedTo();
|
||||
private static final UpdateMatcher<Pair<Result, List<eu.dnetlib.dhp.schema.oaf.Dataset>>, ?> enrichMissingDatasetIsSupplementedBy = new EnrichMissingDatasetIsSupplementedBy();
|
||||
|
||||
// Aggregators
|
||||
private static final TypedColumn<Tuple2<Result, Relation>, ResultGroup> resultAggrTypedColumn = new ResultAggregator().toColumn();
|
||||
private static final TypedColumn<Tuple2<Result, Relation>, ResultGroup> resultAggrTypedColumn = new ResultAggregator()
|
||||
.toColumn();
|
||||
|
||||
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
|
||||
|
||||
public static void main(final String[] args) throws Exception {
|
||||
final ArgumentApplicationParser parser = new ArgumentApplicationParser(
|
||||
IOUtils
|
||||
.toString(GenerateEventsApplication.class
|
||||
.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/merge_claims_parameters.json")));
|
||||
.toString(
|
||||
GenerateEventsApplication.class
|
||||
.getResourceAsStream("/eu/dnetlib/dhp/oa/graph/merge_claims_parameters.json")));
|
||||
parser.parseArgument(args);
|
||||
|
||||
final Boolean isSparkSessionManaged = Optional
|
||||
|
@ -172,18 +166,23 @@ public class GenerateEventsApplication {
|
|||
final Class<R> resultClazz,
|
||||
final DedupConfig dedupConfig) {
|
||||
|
||||
final Dataset<Result> results = readPath(spark, graphPath + "/" + resultClazz.getSimpleName().toLowerCase(), Result.class)
|
||||
.filter(r -> r.getDataInfo().getDeletedbyinference());
|
||||
final Dataset<Result> results = readPath(
|
||||
spark, graphPath + "/" + resultClazz.getSimpleName().toLowerCase(), Result.class)
|
||||
.filter(r -> r.getDataInfo().getDeletedbyinference());
|
||||
|
||||
final Dataset<Relation> mergedRels = readPath(spark, graphPath + "/relation", Relation.class)
|
||||
.filter(r -> r.getRelClass().equals(BrokerConstants.IS_MERGED_IN_CLASS));
|
||||
|
||||
return results.joinWith(mergedRels, results.col("id").equalTo(mergedRels.col("source")), "inner")
|
||||
return results
|
||||
.joinWith(mergedRels, results.col("id").equalTo(mergedRels.col("source")), "inner")
|
||||
.groupByKey((MapFunction<Tuple2<Result, Relation>, String>) t -> t._2.getTarget(), Encoders.STRING())
|
||||
.agg(resultAggrTypedColumn)
|
||||
.map((MapFunction<Tuple2<String, ResultGroup>, ResultGroup>) t -> t._2, Encoders.kryo(ResultGroup.class))
|
||||
.filter(ResultGroup::isValid)
|
||||
.map((MapFunction<ResultGroup, EventGroup>) g -> GenerateEventsApplication.generateSimpleEvents(g, dedupConfig), Encoders.kryo(EventGroup.class))
|
||||
.map(
|
||||
(MapFunction<ResultGroup, EventGroup>) g -> GenerateEventsApplication
|
||||
.generateSimpleEvents(g, dedupConfig),
|
||||
Encoders.kryo(EventGroup.class))
|
||||
.flatMap(group -> group.getData().iterator(), Encoders.kryo(Event.class));
|
||||
}
|
||||
|
||||
|
@ -207,16 +206,19 @@ public class GenerateEventsApplication {
|
|||
return events;
|
||||
}
|
||||
|
||||
private static <SRC extends Result, TRG extends OafEntity> Dataset<Event> generateRelationEvents(final SparkSession spark,
|
||||
private static <SRC extends Result, TRG extends OafEntity> Dataset<Event> generateRelationEvents(
|
||||
final SparkSession spark,
|
||||
final String graphPath,
|
||||
final Class<SRC> sourceClass,
|
||||
final Class<TRG> targetClass,
|
||||
final DedupConfig dedupConfig) {
|
||||
|
||||
final Dataset<Result> sources = readPath(spark, graphPath + "/" + sourceClass.getSimpleName().toLowerCase(), Result.class)
|
||||
.filter(r -> r.getDataInfo().getDeletedbyinference());
|
||||
final Dataset<Result> sources = readPath(
|
||||
spark, graphPath + "/" + sourceClass.getSimpleName().toLowerCase(), Result.class)
|
||||
.filter(r -> r.getDataInfo().getDeletedbyinference());
|
||||
|
||||
final Dataset<TRG> targets = readPath(spark, graphPath + "/" + sourceClass.getSimpleName().toLowerCase(), targetClass);
|
||||
final Dataset<TRG> targets = readPath(
|
||||
spark, graphPath + "/" + sourceClass.getSimpleName().toLowerCase(), targetClass);
|
||||
|
||||
final Dataset<Relation> mergedRels = readPath(spark, graphPath + "/relation", Relation.class)
|
||||
.filter(r -> r.getRelClass().equals(BrokerConstants.IS_MERGED_IN_CLASS));
|
||||
|
@ -224,7 +226,8 @@ public class GenerateEventsApplication {
|
|||
final Dataset<Relation> rels = readPath(spark, graphPath + "/relation", Relation.class)
|
||||
.filter(r -> !r.getRelClass().equals(BrokerConstants.IS_MERGED_IN_CLASS));
|
||||
|
||||
final Dataset<ResultGroup> duplicates = sources.joinWith(mergedRels, sources.col("id").equalTo(rels.col("source")), "inner")
|
||||
final Dataset<ResultGroup> duplicates = sources
|
||||
.joinWith(mergedRels, sources.col("id").equalTo(rels.col("source")), "inner")
|
||||
.groupByKey((MapFunction<Tuple2<Result, Relation>, String>) t -> t._2.getTarget(), Encoders.STRING())
|
||||
.agg(resultAggrTypedColumn)
|
||||
.map((MapFunction<Tuple2<String, ResultGroup>, ResultGroup>) t -> t._2, Encoders.kryo(ResultGroup.class))
|
||||
|
@ -243,7 +246,8 @@ public class GenerateEventsApplication {
|
|||
return null;
|
||||
}
|
||||
|
||||
private List<Event> generateProjectsEvents(final Collection<Pair<Result, List<Project>>> childrenWithProjects, final DedupConfig dedupConfig) {
|
||||
private List<Event> generateProjectsEvents(final Collection<Pair<Result, List<Project>>> childrenWithProjects,
|
||||
final DedupConfig dedupConfig) {
|
||||
final List<UpdateInfo<?>> list = new ArrayList<>();
|
||||
|
||||
for (final Pair<Result, List<Project>> target : childrenWithProjects) {
|
||||
|
@ -254,7 +258,8 @@ public class GenerateEventsApplication {
|
|||
return list.stream().map(EventFactory::newBrokerEvent).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private List<Event> generateSoftwareEvents(final Collection<Pair<Result, List<Software>>> childrenWithSoftwares, final DedupConfig dedupConfig) {
|
||||
private List<Event> generateSoftwareEvents(final Collection<Pair<Result, List<Software>>> childrenWithSoftwares,
|
||||
final DedupConfig dedupConfig) {
|
||||
final List<UpdateInfo<?>> list = new ArrayList<>();
|
||||
|
||||
for (final Pair<Result, List<Software>> target : childrenWithSoftwares) {
|
||||
|
@ -279,15 +284,30 @@ public class GenerateEventsApplication {
|
|||
|
||||
for (final Pair<Result, List<Publication>> target : cleanedChildrens) {
|
||||
if (relType.equals("isRelatedTo")) {
|
||||
list.addAll(enrichMisissingPublicationIsRelatedTo.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
list
|
||||
.addAll(
|
||||
enrichMisissingPublicationIsRelatedTo
|
||||
.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
} else if (relType.equals("references")) {
|
||||
list.addAll(enrichMissingPublicationReferences.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
list
|
||||
.addAll(
|
||||
enrichMissingPublicationReferences
|
||||
.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
} else if (relType.equals("isReferencedBy")) {
|
||||
list.addAll(enrichMissingPublicationIsReferencedBy.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
list
|
||||
.addAll(
|
||||
enrichMissingPublicationIsReferencedBy
|
||||
.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
} else if (relType.equals("isSupplementedTo")) {
|
||||
list.addAll(enrichMissingPublicationIsSupplementedTo.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
list
|
||||
.addAll(
|
||||
enrichMissingPublicationIsSupplementedTo
|
||||
.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
} else if (relType.equals("isSupplementedBy")) {
|
||||
list.addAll(enrichMissingPublicationIsSupplementedBy.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
list
|
||||
.addAll(
|
||||
enrichMissingPublicationIsSupplementedBy
|
||||
.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -310,15 +330,29 @@ public class GenerateEventsApplication {
|
|||
|
||||
for (final Pair<Result, List<eu.dnetlib.dhp.schema.oaf.Dataset>> target : cleanedChildrens) {
|
||||
if (relType.equals("isRelatedTo")) {
|
||||
list.addAll(enrichMisissingDatasetIsRelatedTo.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
list
|
||||
.addAll(
|
||||
enrichMisissingDatasetIsRelatedTo
|
||||
.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
} else if (relType.equals("references")) {
|
||||
list.addAll(enrichMissingDatasetReferences.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
list
|
||||
.addAll(
|
||||
enrichMissingDatasetReferences.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
} else if (relType.equals("isReferencedBy")) {
|
||||
list.addAll(enrichMissingDatasetIsReferencedBy.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
list
|
||||
.addAll(
|
||||
enrichMissingDatasetIsReferencedBy
|
||||
.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
} else if (relType.equals("isSupplementedTo")) {
|
||||
list.addAll(enrichMissingDatasetIsSupplementedTo.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
list
|
||||
.addAll(
|
||||
enrichMissingDatasetIsSupplementedTo
|
||||
.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
} else if (relType.equals("isSupplementedBy")) {
|
||||
list.addAll(enrichMissingDatasetIsSupplementedBy.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
list
|
||||
.addAll(
|
||||
enrichMissingDatasetIsSupplementedBy
|
||||
.searchUpdatesForRecord(target, cleanedChildrens, dedupConfig));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -339,8 +373,12 @@ public class GenerateEventsApplication {
|
|||
private static DedupConfig loadDedupConfig(final String isLookupUrl, final String profId) throws Exception {
|
||||
final ISLookUpService isLookUpService = ISLookupClientFactory.getLookUpService(isLookupUrl);
|
||||
|
||||
final String conf = isLookUpService.getResourceProfileByQuery(String
|
||||
.format("for $x in /RESOURCE_PROFILE[.//RESOURCE_IDENTIFIER/@value = '%s'] return $x//DEDUPLICATION/text()", profId));
|
||||
final String conf = isLookUpService
|
||||
.getResourceProfileByQuery(
|
||||
String
|
||||
.format(
|
||||
"for $x in /RESOURCE_PROFILE[.//RESOURCE_IDENTIFIER/@value = '%s'] return $x//DEDUPLICATION/text()",
|
||||
profId));
|
||||
|
||||
final DedupConfig dedupConfig = new ObjectMapper().readValue(conf, DedupConfig.class);
|
||||
dedupConfig.getPace().initModel();
|
||||
|
|
|
@ -22,7 +22,8 @@ public abstract class UpdateMatcher<K, T> {
|
|||
this.multipleUpdate = multipleUpdate;
|
||||
}
|
||||
|
||||
public Collection<UpdateInfo<T>> searchUpdatesForRecord(final K res, final Collection<K> others, final DedupConfig dedupConfig) {
|
||||
public Collection<UpdateInfo<T>> searchUpdatesForRecord(final K res, final Collection<K> others,
|
||||
final DedupConfig dedupConfig) {
|
||||
|
||||
final Map<String, UpdateInfo<T>> infoMap = new HashMap<>();
|
||||
|
||||
|
@ -30,7 +31,8 @@ public abstract class UpdateMatcher<K, T> {
|
|||
if (source != res) {
|
||||
for (final UpdateInfo<T> info : findUpdates(source, res, dedupConfig)) {
|
||||
final String s = DigestUtils.md5Hex(info.getHighlightValueAsString());
|
||||
if (!infoMap.containsKey(s) || infoMap.get(s).getTrust() < info.getTrust()) {} else {
|
||||
if (!infoMap.containsKey(s) || infoMap.get(s).getTrust() < info.getTrust()) {
|
||||
} else {
|
||||
infoMap.put(s, info);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,9 +18,11 @@ public class EnrichMissingAbstract extends UpdateMatcher<Result, String> {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected List<UpdateInfo<String>> findUpdates(final Result source, final Result target, final DedupConfig dedupConfig) {
|
||||
protected List<UpdateInfo<String>> findUpdates(final Result source, final Result target,
|
||||
final DedupConfig dedupConfig) {
|
||||
if (isMissing(target.getDescription()) && !isMissing(source.getDescription())) {
|
||||
return Arrays.asList(generateUpdateInfo(source.getDescription().get(0).getValue(), source, target, dedupConfig));
|
||||
return Arrays
|
||||
.asList(generateUpdateInfo(source.getDescription().get(0).getValue(), source, target, dedupConfig));
|
||||
}
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
|
|
@ -19,7 +19,8 @@ public class EnrichMissingAuthorOrcid extends UpdateMatcher<Result, Pair<String,
|
|||
}
|
||||
|
||||
@Override
|
||||
protected List<UpdateInfo<Pair<String, String>>> findUpdates(final Result source, final Result target, final DedupConfig dedupConfig) {
|
||||
protected List<UpdateInfo<Pair<String, String>>> findUpdates(final Result source, final Result target,
|
||||
final DedupConfig dedupConfig) {
|
||||
// TODO
|
||||
// return Arrays.asList(new EnrichMissingAbstract("xxxxxxx", 0.9f));
|
||||
return Arrays.asList();
|
||||
|
|
|
@ -21,7 +21,8 @@ public class EnrichMissingOpenAccess extends UpdateMatcher<Result, Instance> {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected List<UpdateInfo<Instance>> findUpdates(final Result source, final Result target, final DedupConfig dedupConfig) {
|
||||
protected List<UpdateInfo<Instance>> findUpdates(final Result source, final Result target,
|
||||
final DedupConfig dedupConfig) {
|
||||
final long count = target
|
||||
.getInstance()
|
||||
.stream()
|
||||
|
@ -29,7 +30,9 @@ public class EnrichMissingOpenAccess extends UpdateMatcher<Result, Instance> {
|
|||
.filter(right -> right.equals(BrokerConstants.OPEN_ACCESS))
|
||||
.count();
|
||||
|
||||
if (count > 0) { return Arrays.asList(); }
|
||||
if (count > 0) {
|
||||
return Arrays.asList();
|
||||
}
|
||||
|
||||
return source
|
||||
.getInstance()
|
||||
|
|
|
@ -20,10 +20,13 @@ public class EnrichMissingPid extends UpdateMatcher<Result, Pid> {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected List<UpdateInfo<Pid>> findUpdates(final Result source, final Result target, final DedupConfig dedupConfig) {
|
||||
protected List<UpdateInfo<Pid>> findUpdates(final Result source, final Result target,
|
||||
final DedupConfig dedupConfig) {
|
||||
final long count = target.getPid().size();
|
||||
|
||||
if (count > 0) { return Arrays.asList(); }
|
||||
if (count > 0) {
|
||||
return Arrays.asList();
|
||||
}
|
||||
|
||||
return source
|
||||
.getPid()
|
||||
|
@ -33,7 +36,8 @@ public class EnrichMissingPid extends UpdateMatcher<Result, Pid> {
|
|||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public UpdateInfo<Pid> generateUpdateInfo(final Pid highlightValue, final Result source, final Result target, final DedupConfig dedupConfig) {
|
||||
public UpdateInfo<Pid> generateUpdateInfo(final Pid highlightValue, final Result source, final Result target,
|
||||
final DedupConfig dedupConfig) {
|
||||
return new UpdateInfo<>(
|
||||
Topic.ENRICH_MISSING_PID,
|
||||
highlightValue, source, target,
|
||||
|
|
|
@ -18,9 +18,11 @@ public class EnrichMissingPublicationDate extends UpdateMatcher<Result, String>
|
|||
}
|
||||
|
||||
@Override
|
||||
protected List<UpdateInfo<String>> findUpdates(final Result source, final Result target, final DedupConfig dedupConfig) {
|
||||
protected List<UpdateInfo<String>> findUpdates(final Result source, final Result target,
|
||||
final DedupConfig dedupConfig) {
|
||||
if (isMissing(target.getDateofacceptance()) && !isMissing(source.getDateofacceptance())) {
|
||||
return Arrays.asList(generateUpdateInfo(source.getDateofacceptance().getValue(), source, target, dedupConfig));
|
||||
return Arrays
|
||||
.asList(generateUpdateInfo(source.getDateofacceptance().getValue(), source, target, dedupConfig));
|
||||
}
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
|
|
@ -23,7 +23,8 @@ public class EnrichMissingSubject extends UpdateMatcher<Result, Pair<String, Str
|
|||
}
|
||||
|
||||
@Override
|
||||
protected List<UpdateInfo<Pair<String, String>>> findUpdates(final Result source, final Result target, final DedupConfig dedupConfig) {
|
||||
protected List<UpdateInfo<Pair<String, String>>> findUpdates(final Result source, final Result target,
|
||||
final DedupConfig dedupConfig) {
|
||||
final Set<String> existingTypes = target
|
||||
.getSubject()
|
||||
.stream()
|
||||
|
|
|
@ -21,7 +21,8 @@ public class EnrichMoreOpenAccess extends UpdateMatcher<Result, Instance> {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected List<UpdateInfo<Instance>> findUpdates(final Result source, final Result target, final DedupConfig dedupConfig) {
|
||||
protected List<UpdateInfo<Instance>> findUpdates(final Result source, final Result target,
|
||||
final DedupConfig dedupConfig) {
|
||||
final Set<String> urls = target
|
||||
.getInstance()
|
||||
.stream()
|
||||
|
|
|
@ -20,7 +20,8 @@ public class EnrichMorePid extends UpdateMatcher<Result, Pid> {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected List<UpdateInfo<Pid>> findUpdates(final Result source, final Result target, final DedupConfig dedupConfig) {
|
||||
protected List<UpdateInfo<Pid>> findUpdates(final Result source, final Result target,
|
||||
final DedupConfig dedupConfig) {
|
||||
final Set<String> existingPids = target
|
||||
.getPid()
|
||||
.stream()
|
||||
|
@ -36,7 +37,8 @@ public class EnrichMorePid extends UpdateMatcher<Result, Pid> {
|
|||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public UpdateInfo<Pid> generateUpdateInfo(final Pid highlightValue, final Result source, final Result target, final DedupConfig dedupConfig) {
|
||||
public UpdateInfo<Pid> generateUpdateInfo(final Pid highlightValue, final Result source, final Result target,
|
||||
final DedupConfig dedupConfig) {
|
||||
return new UpdateInfo<>(
|
||||
Topic.ENRICH_MORE_PID,
|
||||
highlightValue, source, target,
|
||||
|
|
|
@ -21,7 +21,8 @@ public class EnrichMoreSubject extends UpdateMatcher<Result, Pair<String, String
|
|||
}
|
||||
|
||||
@Override
|
||||
protected List<UpdateInfo<Pair<String, String>>> findUpdates(final Result source, final Result target, final DedupConfig dedupConfig) {
|
||||
protected List<UpdateInfo<Pair<String, String>>> findUpdates(final Result source, final Result target,
|
||||
final DedupConfig dedupConfig) {
|
||||
final Set<String> existingSubjects = target
|
||||
.getSubject()
|
||||
.stream()
|
||||
|
|
|
@ -1,3 +1,4 @@
|
|||
|
||||
package eu.dnetlib.dhp.broker.oa.util;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
|
|
@ -1,3 +1,4 @@
|
|||
|
||||
package eu.dnetlib.dhp.broker.oa.util;
|
||||
|
||||
import org.apache.spark.sql.Encoder;
|
||||
|
|
|
@ -1,3 +1,4 @@
|
|||
|
||||
package eu.dnetlib.dhp.broker.oa.util;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
|
|
@ -1,14 +1,22 @@
|
|||
|
||||
package eu.dnetlib.dhp.broker.oa.util;
|
||||
|
||||
public class TrustUtils {
|
||||
|
||||
public static float rescale(final double score, final double threshold) {
|
||||
if (score >= BrokerConstants.MAX_TRUST) { return BrokerConstants.MAX_TRUST; }
|
||||
if (score >= BrokerConstants.MAX_TRUST) {
|
||||
return BrokerConstants.MAX_TRUST;
|
||||
}
|
||||
|
||||
final double val = (score - threshold) * (BrokerConstants.MAX_TRUST - BrokerConstants.MIN_TRUST) / (BrokerConstants.MAX_TRUST - threshold);
|
||||
final double val = (score - threshold) * (BrokerConstants.MAX_TRUST - BrokerConstants.MIN_TRUST)
|
||||
/ (BrokerConstants.MAX_TRUST - threshold);
|
||||
|
||||
if (val < BrokerConstants.MIN_TRUST) { return BrokerConstants.MIN_TRUST; }
|
||||
if (val > BrokerConstants.MAX_TRUST) { return BrokerConstants.MAX_TRUST; }
|
||||
if (val < BrokerConstants.MIN_TRUST) {
|
||||
return BrokerConstants.MIN_TRUST;
|
||||
}
|
||||
if (val > BrokerConstants.MAX_TRUST) {
|
||||
return BrokerConstants.MAX_TRUST;
|
||||
}
|
||||
|
||||
return (float) val;
|
||||
}
|
||||
|
|
|
@ -68,8 +68,10 @@ public final class UpdateInfo<T> {
|
|||
private float calculateTrust(final DedupConfig dedupConfig, final Result r1, final Result r2) {
|
||||
try {
|
||||
final ObjectMapper objectMapper = new ObjectMapper();
|
||||
final MapDocument doc1 = MapDocumentUtil.asMapDocumentWithJPath(dedupConfig, objectMapper.writeValueAsString(r1));
|
||||
final MapDocument doc2 = MapDocumentUtil.asMapDocumentWithJPath(dedupConfig, objectMapper.writeValueAsString(r2));
|
||||
final MapDocument doc1 = MapDocumentUtil
|
||||
.asMapDocumentWithJPath(dedupConfig, objectMapper.writeValueAsString(r1));
|
||||
final MapDocument doc2 = MapDocumentUtil
|
||||
.asMapDocumentWithJPath(dedupConfig, objectMapper.writeValueAsString(r2));
|
||||
|
||||
final double score = new TreeProcessor(dedupConfig).computeScore(doc1, doc2);
|
||||
final double threshold = dedupConfig.getWf().getThreshold();
|
||||
|
@ -118,7 +120,8 @@ public final class UpdateInfo<T> {
|
|||
.map(Instance::getUrl)
|
||||
.flatMap(List::stream)
|
||||
.findFirst()
|
||||
.orElse(null);;
|
||||
.orElse(null);
|
||||
;
|
||||
|
||||
final Provenance provenance = new Provenance().setId(provId).setRepositoryName(provRepo).setUrl(provUrl);
|
||||
|
||||
|
|
|
@ -1,3 +1,4 @@
|
|||
|
||||
package eu.dnetlib.dhp.broker.oa.util;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
|
|
@ -1,14 +1,18 @@
|
|||
<configuration>
|
||||
<property>
|
||||
<name>jobTracker</name>
|
||||
<value>yarnRM</value>
|
||||
<value>hadoop-rm3.garr-pa1.d4science.org:8032</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>nameNode</name>
|
||||
<value>hdfs://nameservice1</value>
|
||||
<value>hdfs://hadoop-rm1.garr-pa1.d4science.org:8020</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozie.action.sharelib.for.java</name>
|
||||
<name>oozie.use.system.libpath</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozie.action.sharelib.for.spark</name>
|
||||
<value>spark2</value>
|
||||
</property>
|
||||
<property>
|
||||
|
@ -16,7 +20,23 @@
|
|||
<value>true</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>oozie.launcher.mapreduce.map.java.opts</name>
|
||||
<value>-Xmx4g</value>
|
||||
<name>hive_metastore_uris</name>
|
||||
<value>thrift://hadoop-edge2.garr-pa1.d4science.org:9083</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>spark2YarnHistoryServerAddress</name>
|
||||
<value>http://hadoop-edge1.garr-pa1.d4science.org: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>
|
||||
</configuration>
|
|
@ -159,7 +159,7 @@ public class DatasetScholexplorerParser extends AbstractScholexplorerParser {
|
|||
.setDescription(
|
||||
descs
|
||||
.stream()
|
||||
.map(it -> it.length() < 10000 ? it : it.substring(0, 10000))
|
||||
// .map(it -> it.length() < 10000 ? it : it.substring(0, 10000))
|
||||
.map(
|
||||
it -> {
|
||||
final Field<String> d = new Field<>();
|
||||
|
|
|
@ -213,10 +213,10 @@ public class PublicationScholexplorerParser extends AbstractScholexplorerParser
|
|||
.setValue(
|
||||
VtdUtilityParser.getSingleValue(ap, vn, "//*[local-name()='description']"));
|
||||
|
||||
if (StringUtils.isNotBlank(description.getValue())
|
||||
&& description.getValue().length() > 10000) {
|
||||
description.setValue(description.getValue().substring(0, 10000));
|
||||
}
|
||||
// if (StringUtils.isNotBlank(description.getValue())
|
||||
// && description.getValue().length() > 10000) {
|
||||
// description.setValue(description.getValue().substring(0, 10000));
|
||||
// }
|
||||
|
||||
parsedObject.setDescription(Collections.singletonList(description));
|
||||
|
||||
|
|
|
@ -47,8 +47,8 @@
|
|||
|
||||
<decision name="ReuseContent">
|
||||
<switch>
|
||||
<case to="ResetWorkingPath">${wf:conf('reuseContent') eq false}</case>
|
||||
<case to="ResetTargetPath">${wf:conf('reuseContent') eq true}</case>
|
||||
<case to="ImportEntitiesFromMongo">${wf:conf('reuseContent') eq false}</case>
|
||||
<case to="ConvertXML2Entity">${wf:conf('reuseContent') eq true}</case>
|
||||
<default to="ResetWorkingPath"/>
|
||||
</switch>
|
||||
</decision>
|
||||
|
|
|
@ -0,0 +1,376 @@
|
|||
package eu.dnetlib.dhp.export
|
||||
|
||||
import java.time.LocalDateTime
|
||||
import java.time.format.DateTimeFormatter
|
||||
|
||||
import eu.dnetlib.dhp.common.PacePerson
|
||||
import eu.dnetlib.dhp.schema.oaf.{Author, DataInfo, Dataset, Field, Instance, KeyValue, Publication, Qualifier, Relation, StructuredProperty}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation}
|
||||
import org.apache.commons.lang3.StringUtils
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
|
||||
case class DLIExternalReference(id: String, url: String, sitename: String, label: String, pid: String, classId: String) {}
|
||||
|
||||
object DLIToOAF {
|
||||
|
||||
|
||||
val collectedFromMap: Map[String, KeyValue] = Map(
|
||||
"dli_________::r3d100010527" -> generateKeyValue("10|re3data_____::c2a591f440598b63d854556beaf01591", "European Nucleotide Archive"),
|
||||
"dli_________::r3d100010255" -> generateKeyValue("10|re3data_____::480d275ed6f9666ee76d6a1215eabf26", "Inter-university Consortium for Political and Social Research"),
|
||||
"dli_________::r3d100011868" -> generateKeyValue("10|re3data_____::db814dc656a911b556dba42a331cebe9", "Mendeley Data"),
|
||||
"dli_________::elsevier" -> generateKeyValue("10|openaire____::8f87e10869299a5fe80b315695296b88", "Elsevier"),
|
||||
"dli_________::openaire" -> generateKeyValue("10|infrastruct_::f66f1bd369679b5b077dcdf006089556", "OpenAIRE"),
|
||||
"dli_________::thomsonreuters" -> generateKeyValue("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2", "Crossref"),
|
||||
"dli_________::r3d100010216" -> generateKeyValue("10|re3data_____::0fd79429de04343dbbec705d9b5f429f", "4TU.Centre for Research Data"),
|
||||
"dli_________::r3d100010134" -> generateKeyValue("10|re3data_____::9633d1e8c4309c833c2c442abeb0cfeb", "PANGAEA"),
|
||||
"dli_________::ieee" -> generateKeyValue("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2", "Crossref"),
|
||||
"dli_________::r3d100010197" -> generateKeyValue("10|re3data_____::9fd1d79973f7fda60cbe1d82e3819a68", "The Cambridge Structural Database"),
|
||||
"dli_________::nature" -> generateKeyValue("10|openaire____::6e380d9cf51138baec8480f5a0ce3a2e", "Springer Nature"),
|
||||
"dli_________::datacite" -> generateKeyValue("10|openaire____::9e3be59865b2c1c335d32dae2fe7b254", "Datacite"),
|
||||
"dli_________::r3d100010578" -> generateKeyValue("10|re3data_____::c4d751f29a7568011a4c80136b30b444", "IEDA"),
|
||||
"dli_________::r3d100010464" -> generateKeyValue("10|re3data_____::23e2a81591099828f6b83a1c83150666", "Research Data Australia"),
|
||||
"dli_________::r3d100010327" -> generateKeyValue("10|re3data_____::a644620b81135243dc9acc15d2362246", "Worldwide Protein Data Bank"),
|
||||
"dli_________::pubmed" -> generateKeyValue("10|opendoar____::eda80a3d5b344bc40f3bc04f65b7a357", "PubMed Central"),
|
||||
"dli_________::europe_pmc__" -> generateKeyValue("10|opendoar____::8b6dd7db9af49e67306feb59a8bdc52c", "Europe PubMed Central"),
|
||||
"dli_________::crossref" -> generateKeyValue("10|openaire____::081b82f96300b6a6e3d282bad31cb6e2", "Crossref")
|
||||
)
|
||||
|
||||
|
||||
val relationTypeMapping: Map[String, (String, String)] = Map(
|
||||
"IsReferencedBy" -> ("isRelatedTo", "relationship"),
|
||||
"References" -> ("isRelatedTo", "relationship"),
|
||||
"IsRelatedTo" -> ("isRelatedTo", "relationship"),
|
||||
"IsSupplementedBy" -> ("IsSupplementedBy", "supplement"),
|
||||
"Cites" -> ("cites", "citation"),
|
||||
"Unknown" -> ("isRelatedTo", "relationship"),
|
||||
"IsSourceOf" -> ("isRelatedTo", "relationship"),
|
||||
"IsCitedBy" -> ("IsCitedBy", "citation"),
|
||||
"Reviews" -> ("reviews", "review"),
|
||||
"Describes" -> ("isRelatedTo", "relationship"),
|
||||
"HasAssociationWith" -> ("isRelatedTo", "relationship")
|
||||
)
|
||||
|
||||
val expectecdPidType = List("uniprot", "ena", "chembl", "ncbi-n", "ncbi-p", "genbank", "pdb", "url")
|
||||
|
||||
|
||||
val filteredURL = List(
|
||||
"www.ebi.ac.uk",
|
||||
"www.uniprot.org",
|
||||
"f1000.com",
|
||||
"en.wikipedia.org",
|
||||
"flybase.org",
|
||||
"www.yeastgenome.org",
|
||||
"research.bioinformatics.udel.edu",
|
||||
"cancer.sanger.ac.uk",
|
||||
"www.iedb.org",
|
||||
"www.crd.york.ac.uk",
|
||||
"www.wormbase.org",
|
||||
"web.expasy.org",
|
||||
"www.hal.inserm.fr",
|
||||
"sabiork.h-its.org",
|
||||
"zfin.org",
|
||||
"www.pombase.org",
|
||||
"www.guidetopharmacology.org",
|
||||
"reactome.org"
|
||||
)
|
||||
|
||||
|
||||
def filterPid(p: StructuredProperty): Boolean = {
|
||||
if (expectecdPidType.contains(p.getQualifier.getClassname) && p.getQualifier.getClassname.equalsIgnoreCase("url"))
|
||||
if (filteredURL.exists(u => p.getValue.contains(u)))
|
||||
return true
|
||||
else
|
||||
return false
|
||||
expectecdPidType.contains(p.getQualifier.getClassname)
|
||||
}
|
||||
|
||||
|
||||
def extractTitle(titles: java.util.List[StructuredProperty]): String = {
|
||||
|
||||
if (titles == null)
|
||||
return null
|
||||
|
||||
val label = titles.asScala.map(p => p.getValue).find(p => p.nonEmpty)
|
||||
label.orNull
|
||||
}
|
||||
|
||||
def convertDLIDatasetToExternalReference(dataset: DLIDataset): DLIExternalReference = {
|
||||
val currentId = generateId(dataset.getId)
|
||||
val pids = dataset.getPid.asScala.filter(filterPid)
|
||||
|
||||
if (pids == null || pids.isEmpty)
|
||||
return null
|
||||
|
||||
val pid: StructuredProperty = pids.head
|
||||
|
||||
|
||||
pid.getQualifier.getClassname match {
|
||||
case "uniprot" => DLIExternalReference(generateId(dataset.getId), s"https://www.uniprot.org/uniprot/${pid.getValue}", "UniProt", extractTitle(dataset.getTitle), pid.getValue, "accessionNumber")
|
||||
case "ena" =>
|
||||
if(pid.getValue!= null && pid.getValue.nonEmpty && pid.getValue.length>7)
|
||||
DLIExternalReference(generateId(dataset.getId), s"https://www.ebi.ac.uk/ena/data/view/${pid.getValue.substring(0, 8)}", "European Nucleotide Archive", extractTitle(dataset.getTitle), pid.getValue, "accessionNumber")
|
||||
else
|
||||
null
|
||||
case "chembl" => DLIExternalReference(generateId(dataset.getId), s"https://www.ebi.ac.uk/chembl/compound_report_card/${pid.getValue}", "ChEMBL", extractTitle(dataset.getTitle), pid.getValue, "accessionNumber")
|
||||
case "ncbi-n" => DLIExternalReference(generateId(dataset.getId), s"https://www.ncbi.nlm.nih.gov/nuccore/${pid.getValue}", "Nucleotide Database", extractTitle(dataset.getTitle), pid.getValue, "accessionNumber")
|
||||
case "ncbi-p" => DLIExternalReference(generateId(dataset.getId), s"https://www.ncbi.nlm.nih.gov/nuccore/${pid.getValue}", "Nucleotide Database", extractTitle(dataset.getTitle), pid.getValue, "accessionNumber")
|
||||
case "genbank" => DLIExternalReference(generateId(dataset.getId), s"https://www.ncbi.nlm.nih.gov/nuccore/${pid.getValue}", "GenBank", extractTitle(dataset.getTitle), pid.getValue, "accessionNumber")
|
||||
case "pdb" => DLIExternalReference(generateId(dataset.getId), s"https://www.ncbi.nlm.nih.gov/nuccore/${pid.getValue}", "Protein Data Bank", extractTitle(dataset.getTitle), pid.getValue, "accessionNumber")
|
||||
case "url" => DLIExternalReference(generateId(dataset.getId), pid.getValue, "", extractTitle(dataset.getTitle), pid.getValue, "url")
|
||||
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
def convertDLIPublicationToOAF(p: DLIPublication): Publication = {
|
||||
|
||||
val result = new Publication
|
||||
result.setId(generateId(p.getId))
|
||||
result.setDataInfo(generateDataInfo(invisibile = true))
|
||||
if (p.getCollectedfrom == null || p.getCollectedfrom.size() == 0 || (p.getCollectedfrom.size() == 1 && p.getCollectedfrom.get(0) == null))
|
||||
return null
|
||||
|
||||
result.setCollectedfrom(p.getCollectedfrom.asScala.map(c => collectedFromMap.getOrElse(c.getKey, null)).asJava)
|
||||
result.setPid(p.getPid)
|
||||
result.setDateofcollection(p.getDateofcollection)
|
||||
result.setOriginalId(p.getPid.asScala.map(p => p.getValue).asJava)
|
||||
result.setDateoftransformation(LocalDateTime.now().format(DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss'Z'")))
|
||||
if (p.getAuthor == null || p.getAuthor.isEmpty)
|
||||
return null
|
||||
result.setAuthor(p.getAuthor.asScala.map(convertAuthor).asJava)
|
||||
result.setResulttype(createQualifier(p.getResulttype.getClassid, p.getResulttype.getClassname, "dnet:result_typologies", "dnet:result_typologies"))
|
||||
|
||||
if (p.getSubject != null)
|
||||
result.setSubject(p.getSubject.asScala.map(convertSubject).asJava)
|
||||
|
||||
if (p.getTitle == null || p.getTitle.isEmpty)
|
||||
return null
|
||||
|
||||
result.setTitle(List(patchTitle(p.getTitle.get(0))).asJava)
|
||||
|
||||
if (p.getRelevantdate == null || p.getRelevantdate.size() == 0)
|
||||
return null
|
||||
|
||||
result.setRelevantdate(p.getRelevantdate.asScala.map(patchRelevantDate).asJava)
|
||||
|
||||
|
||||
result.setDescription(p.getDescription)
|
||||
|
||||
result.setDateofacceptance(asField(p.getRelevantdate.get(0).getValue))
|
||||
result.setPublisher(p.getPublisher)
|
||||
result.setSource(p.getSource)
|
||||
result.setBestaccessright(createQualifier("UNKNOWN", "not available", "dnet:access_modes", "dnet:access_modes"))
|
||||
|
||||
val dois = result.getPid.asScala.filter(p => "doi".equalsIgnoreCase(p.getQualifier.getClassname)).map(p => p.getValue)
|
||||
if (dois.isEmpty)
|
||||
return null
|
||||
|
||||
|
||||
val i: Instance = createInstance(s"https://dx.doi.org/${dois.head}", firstInstanceOrNull(p.getInstance()), result.getDateofacceptance)
|
||||
|
||||
if (i != null)
|
||||
result.setInstance(List(i).asJava)
|
||||
|
||||
result
|
||||
}
|
||||
|
||||
|
||||
def convertDLIRelation(r: DLIRelation): Relation = {
|
||||
|
||||
val result = new Relation
|
||||
if (!relationTypeMapping.contains(r.getRelType))
|
||||
return null
|
||||
|
||||
if (r.getCollectedFrom == null || r.getCollectedFrom.size() == 0 || (r.getCollectedFrom.size() == 1 && r.getCollectedFrom.get(0) == null))
|
||||
return null
|
||||
val t = relationTypeMapping.get(r.getRelType)
|
||||
|
||||
result.setRelType("resultResult")
|
||||
result.setRelClass(t.get._1)
|
||||
result.setSubRelType(t.get._2)
|
||||
result.setCollectedfrom(r.getCollectedFrom.asScala.map(c => collectedFromMap.getOrElse(c.getKey, null)).filter(p => p != null).asJava)
|
||||
result.setSource(generateId(r.getSource))
|
||||
result.setTarget(generateId(r.getTarget))
|
||||
|
||||
if (result.getSource.equals(result.getTarget))
|
||||
return null
|
||||
result.setDataInfo(generateDataInfo())
|
||||
|
||||
result
|
||||
}
|
||||
|
||||
|
||||
def convertDLIDatasetTOOAF(d: DLIDataset): Dataset = {
|
||||
|
||||
if (d.getCollectedfrom == null || d.getCollectedfrom.size() == 0 || (d.getCollectedfrom.size() == 1 && d.getCollectedfrom.get(0) == null))
|
||||
return null
|
||||
val result: Dataset = new Dataset
|
||||
result.setId(generateId(d.getId))
|
||||
result.setDataInfo(generateDataInfo())
|
||||
result.setCollectedfrom(d.getCollectedfrom.asScala.map(c => collectedFromMap.getOrElse(c.getKey, null)).asJava)
|
||||
|
||||
|
||||
result.setPid(d.getPid)
|
||||
|
||||
val fpids = result.getPid.asScala.filter(p => "doi".equalsIgnoreCase(p.getQualifier.getClassname) ||
|
||||
"pdb".equalsIgnoreCase(p.getQualifier.getClassname)
|
||||
).map(p => p.getValue)
|
||||
|
||||
if (fpids == null || fpids.isEmpty)
|
||||
return null
|
||||
|
||||
|
||||
result.setDateofcollection(d.getDateofcollection)
|
||||
result.setOriginalId(d.getPid.asScala.map(d => d.getValue).asJava)
|
||||
result.setDateoftransformation(LocalDateTime.now().format(DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss'Z'")))
|
||||
if (d.getAuthor == null || d.getAuthor.isEmpty)
|
||||
return null
|
||||
result.setAuthor(d.getAuthor.asScala.map(convertAuthor).asJava)
|
||||
result.setResulttype(createQualifier(d.getResulttype.getClassid, d.getResulttype.getClassname, "dnet:result_typologies", "dnet:result_typologies"))
|
||||
|
||||
if (d.getSubject != null)
|
||||
result.setSubject(d.getSubject.asScala.map(convertSubject).asJava)
|
||||
|
||||
if (d.getTitle == null || d.getTitle.isEmpty)
|
||||
return null
|
||||
|
||||
result.setTitle(List(patchTitle(d.getTitle.get(0))).asJava)
|
||||
|
||||
if (d.getRelevantdate == null || d.getRelevantdate.size() == 0)
|
||||
return null
|
||||
|
||||
result.setRelevantdate(d.getRelevantdate.asScala.map(patchRelevantDate).asJava)
|
||||
|
||||
|
||||
result.setDescription(d.getDescription)
|
||||
|
||||
result.setDateofacceptance(asField(d.getRelevantdate.get(0).getValue))
|
||||
result.setPublisher(d.getPublisher)
|
||||
result.setSource(d.getSource)
|
||||
result.setBestaccessright(createQualifier("UNKNOWN", "not available", "dnet:access_modes", "dnet:access_modes"))
|
||||
|
||||
|
||||
val instance_urls = if (fpids.head.length < 5) s"https://www.rcsb.org/structure/${fpids.head}" else s"https://dx.doi.org/${fpids.head}"
|
||||
|
||||
val i: Instance = createInstance(instance_urls, firstInstanceOrNull(d.getInstance()), result.getDateofacceptance, true)
|
||||
if (i != null)
|
||||
result.setInstance(List(i).asJava)
|
||||
|
||||
result
|
||||
}
|
||||
|
||||
|
||||
def firstInstanceOrNull(instances: java.util.List[Instance]): Instance = {
|
||||
|
||||
if (instances == null || instances.size() == 0)
|
||||
return null
|
||||
instances.get(0)
|
||||
|
||||
}
|
||||
|
||||
|
||||
def createInstance(url: String, originalInstance: Instance, doa: Field[String], dataset: Boolean = false): Instance = {
|
||||
|
||||
val i = new Instance
|
||||
i.setUrl(List(url).asJava)
|
||||
if (dataset)
|
||||
i.setInstancetype(createQualifier("0021", "Dataset", "dnet:publication_resource", "dnet:publication_resource"))
|
||||
else
|
||||
i.setInstancetype(createQualifier("0000", "UNKNOWN", "dnet:publication_resource", "dnet:publication_resource"))
|
||||
if (originalInstance != null && originalInstance.getHostedby != null)
|
||||
i.setHostedby(originalInstance.getHostedby)
|
||||
|
||||
i.setAccessright(createQualifier("UNKNOWN", "not available", "dnet:access_modes", "dnet:access_modes"))
|
||||
i.setDateofacceptance(doa)
|
||||
|
||||
i
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
def patchRelevantDate(d: StructuredProperty): StructuredProperty = {
|
||||
d.setQualifier(createQualifier("UNKNOWN", "dnet:dataCite_date"))
|
||||
d
|
||||
|
||||
}
|
||||
|
||||
def patchTitle(t: StructuredProperty): StructuredProperty = {
|
||||
t.setQualifier(createQualifier("main title", "dnet:dataCite_title"))
|
||||
t
|
||||
}
|
||||
|
||||
|
||||
def convertSubject(s: StructuredProperty): StructuredProperty = {
|
||||
s.setQualifier(createQualifier("keyword", "dnet:subject_classification_typologies"))
|
||||
s
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
def convertAuthor(a: Author): Author = {
|
||||
if (a == null)
|
||||
return a
|
||||
val p = new PacePerson(a.getFullname, false)
|
||||
if (p.isAccurate) {
|
||||
a.setName(p.getNameString)
|
||||
a.setSurname(p.getSurnameString)
|
||||
}
|
||||
a
|
||||
}
|
||||
|
||||
|
||||
def generateId(id: String): String = {
|
||||
val md5 = if (id.contains("::")) StringUtils.substringAfter(id, "::") else StringUtils.substringAfter(id, "|")
|
||||
s"50|scholix_____::$md5"
|
||||
}
|
||||
|
||||
|
||||
def generateKeyValue(key: String, value: String): KeyValue = {
|
||||
val kv: KeyValue = new KeyValue()
|
||||
kv.setKey(key)
|
||||
kv.setValue(value)
|
||||
kv.setDataInfo(generateDataInfo("0.9"))
|
||||
kv
|
||||
}
|
||||
|
||||
|
||||
def generateDataInfo(trust: String = "0.9", invisibile: Boolean = false): DataInfo = {
|
||||
val di = new DataInfo
|
||||
di.setDeletedbyinference(false)
|
||||
di.setInferred(false)
|
||||
di.setInvisible(false)
|
||||
di.setTrust(trust)
|
||||
di.setProvenanceaction(createQualifier("sysimport:actionset", "dnet:provenanceActions"))
|
||||
di
|
||||
}
|
||||
|
||||
def createQualifier(cls: String, sch: String): Qualifier = {
|
||||
createQualifier(cls, cls, sch, sch)
|
||||
}
|
||||
|
||||
|
||||
def createQualifier(classId: String, className: String, schemeId: String, schemeName: String): Qualifier = {
|
||||
val q: Qualifier = new Qualifier
|
||||
q.setClassid(classId)
|
||||
q.setClassname(className)
|
||||
q.setSchemeid(schemeId)
|
||||
q.setSchemename(schemeName)
|
||||
q
|
||||
}
|
||||
|
||||
|
||||
def asField[T](value: T): Field[T] = {
|
||||
val tmp = new Field[T]
|
||||
tmp.setValue(value)
|
||||
tmp
|
||||
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,118 @@
|
|||
package eu.dnetlib.dhp.`export`
|
||||
|
||||
import eu.dnetlib.dhp.application.ArgumentApplicationParser
|
||||
import eu.dnetlib.dhp.schema.oaf.{Publication, Relation, Dataset => OafDataset}
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation}
|
||||
import org.apache.commons.io.IOUtils
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.{Dataset, Encoder, Encoders, SaveMode, SparkSession}
|
||||
import org.apache.spark.{SparkConf, SparkContext}
|
||||
import org.codehaus.jackson.map.ObjectMapper
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
|
||||
|
||||
object SparkExportContentForOpenAire {
|
||||
|
||||
|
||||
def main(args: Array[String]): Unit = {
|
||||
val conf: SparkConf = new SparkConf()
|
||||
val parser = new ArgumentApplicationParser(IOUtils.toString(SparkExportContentForOpenAire.getClass.getResourceAsStream("input_export_content_parameters.json")))
|
||||
parser.parseArgument(args)
|
||||
val spark: SparkSession =
|
||||
SparkSession
|
||||
.builder()
|
||||
.config(conf)
|
||||
.appName(SparkExportContentForOpenAire.getClass.getSimpleName)
|
||||
.master(parser.get("master")).getOrCreate()
|
||||
|
||||
|
||||
val sc:SparkContext = spark.sparkContext
|
||||
|
||||
val workingPath = parser.get("workingDirPath")
|
||||
|
||||
implicit val pubEncoder: Encoder[Publication] = Encoders.bean(classOf[Publication])
|
||||
implicit val datEncoder: Encoder[OafDataset] = Encoders.bean(classOf[OafDataset])
|
||||
implicit val relEncoder: Encoder[Relation] = Encoders.bean(classOf[Relation])
|
||||
implicit val dliRelEncoder: Encoder[DLIRelation] = Encoders.bean(classOf[DLIRelation])
|
||||
import spark.implicits._
|
||||
|
||||
//
|
||||
// val relRDD:RDD[Relation] = sc.textFile(s"$workingPath/relation_j")
|
||||
// .map(s => new ObjectMapper().readValue(s, classOf[DLIRelation]))
|
||||
// .filter(p => p.getDataInfo.getDeletedbyinference == false)
|
||||
// .map(DLIToOAF.convertDLIRelation).filter(p=>p!= null)
|
||||
// spark.createDataset(relRDD).write.mode(SaveMode.Overwrite).save(s"$workingPath/relationDS")
|
||||
//
|
||||
// val datRDD:RDD[OafDataset] = sc.textFile(s"$workingPath/dataset")
|
||||
// .map(s => new ObjectMapper().readValue(s, classOf[DLIDataset]))
|
||||
// .filter(p => p.getDataInfo.getDeletedbyinference == false)
|
||||
// .map(DLIToOAF.convertDLIDatasetTOOAF).filter(p=>p!= null)
|
||||
// spark.createDataset(datRDD).write.mode(SaveMode.Overwrite).save(s"$workingPath/datasetDS")
|
||||
//
|
||||
//
|
||||
// val pubRDD:RDD[Publication] = sc.textFile(s"$workingPath/publication")
|
||||
// .map(s => new ObjectMapper().readValue(s, classOf[DLIPublication]))
|
||||
// .filter(p => p.getDataInfo.getDeletedbyinference == false)
|
||||
// .map(DLIToOAF.convertDLIPublicationToOAF).filter(p=>p!= null)
|
||||
// spark.createDataset(pubRDD).write.mode(SaveMode.Overwrite).save(s"$workingPath/publicationDS")
|
||||
//
|
||||
//
|
||||
//
|
||||
// val pubs:Dataset[Publication] = spark.read.load(s"$workingPath/publicationDS").as[Publication]
|
||||
// val dats :Dataset[OafDataset] = spark.read.load(s"$workingPath/datasetDS").as[OafDataset]
|
||||
var relDS :Dataset[Relation] = spark.read.load(s"$workingPath/relationDS").as[Relation]
|
||||
//
|
||||
//
|
||||
// pubs.joinWith(relDS, pubs("id").equalTo(relDS("source"))).map(k => k._2).write.mode(SaveMode.Overwrite).save(s"$workingPath/relationDS_f1")
|
||||
//
|
||||
// relDS= spark.read.load(s"$workingPath/relationDS_f1").as[Relation]
|
||||
//
|
||||
// relDS.joinWith(dats, relDS("target").equalTo(dats("id"))).map(k => k._1).write.mode(SaveMode.Overwrite).save(s"$workingPath/relationDS_filtered")
|
||||
//
|
||||
//
|
||||
// val r_source = relDS.select(relDS("source")).distinct()
|
||||
// val r_target = relDS.select(relDS("source")).distinct()
|
||||
//
|
||||
//
|
||||
// pubs.joinWith(r_source, pubs("id").equalTo(r_source("source")), "inner").map(k => k._1).write.mode(SaveMode.Overwrite).save(s"$workingPath/publicationDS_filtered")
|
||||
//
|
||||
// dats.joinWith(r_target, dats("id").equalTo(r_target("target")), "inner").map(k => k._1).write.mode(SaveMode.Overwrite).save(s"$workingPath/datasetDS_filtered")
|
||||
//
|
||||
// spark.createDataset(sc.textFile(s"$workingPath/dataset")
|
||||
// .map(s => new ObjectMapper().readValue(s, classOf[DLIDataset]))
|
||||
// .map(DLIToOAF.convertDLIDatasetToExternalReference)
|
||||
// .filter(p => p != null)).as[DLIExternalReference].write.mode(SaveMode.Overwrite).save(s"$workingPath/externalReference")
|
||||
//
|
||||
|
||||
val pf = spark.read.load(s"$workingPath/publicationDS_filtered").select("id")
|
||||
relDS = spark.read.load(s"$workingPath/relationDS").as[Relation]
|
||||
val relationTo = pf.joinWith(relDS, pf("id").equalTo(relDS("source")),"inner").map(t =>t._2)
|
||||
|
||||
val extRef = spark.read.load(s"$workingPath/externalReference").as[DLIExternalReference]
|
||||
|
||||
spark.createDataset(relationTo.joinWith(extRef, relationTo("target").equalTo(extRef("id")), "inner").map(d => {
|
||||
val r = d._1
|
||||
val ext = d._2
|
||||
(r.getSource, ext)
|
||||
}).rdd.groupByKey.map(f => {
|
||||
var dli_ext = ArrayBuffer[DLIExternalReference]()
|
||||
f._2.foreach(d => if (dli_ext.size < 100) dli_ext += d )
|
||||
(f._1, dli_ext)
|
||||
})).write.mode(SaveMode.Overwrite).save(s"$workingPath/externalReference_grouped")
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,14 @@
|
|||
[
|
||||
{
|
||||
"paramName": "mt",
|
||||
"paramLongName": "master",
|
||||
"paramDescription": "should be local or yarn",
|
||||
"paramRequired": true
|
||||
},
|
||||
{
|
||||
"paramName": "w",
|
||||
"paramLongName": "workingDirPath",
|
||||
"paramDescription": "the working path where generated files",
|
||||
"paramRequired": true
|
||||
}
|
||||
]
|
|
@ -0,0 +1,42 @@
|
|||
<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>oozie.wf.rerun.failnodes</name>
|
||||
<value>false</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>
|
||||
</configuration>
|
|
@ -0,0 +1,49 @@
|
|||
<workflow-app name="Export Scholexplorer Graph to OpenAIRE" xmlns="uri:oozie:workflow:0.5">
|
||||
<parameters>
|
||||
<property>
|
||||
<name>workingDirPath</name>
|
||||
<description>the source path</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>memory for individual executor</description>
|
||||
</property>
|
||||
</parameters>
|
||||
|
||||
<start to="ExtractOAF"/>
|
||||
|
||||
<kill name="Kill">
|
||||
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
|
||||
</kill>
|
||||
|
||||
<action name="ExtractOAF">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn-cluster</master>
|
||||
<mode>cluster</mode>
|
||||
<name>ExtractOAF</name>
|
||||
<class>eu.dnetlib.dhp.export.SparkExportContentForOpenAire</class>
|
||||
<jar>dhp-graph-provision-scholexplorer-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
${sparkExtraOPT}
|
||||
</spark-opts>
|
||||
<arg>--workingDirPath</arg><arg>${workingDirPath}</arg>
|
||||
<arg>--master</arg><arg>yarn-cluster</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<end name="End"/>
|
||||
</workflow-app>
|
|
@ -0,0 +1,75 @@
|
|||
package eu.dnetlib.dhp.export
|
||||
|
||||
import java.time.LocalDateTime
|
||||
import java.time.format.DateTimeFormatter
|
||||
|
||||
import eu.dnetlib.dhp.schema.oaf.Relation
|
||||
import eu.dnetlib.dhp.schema.scholexplorer.{DLIDataset, DLIPublication, DLIRelation}
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.codehaus.jackson.map.{ObjectMapper, SerializationConfig}
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
import scala.io.Source
|
||||
|
||||
class ExportDLITOOAFTest {
|
||||
|
||||
val mapper = new ObjectMapper()
|
||||
|
||||
@Test
|
||||
def testDate():Unit = {
|
||||
println(LocalDateTime.now().format(DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss'Z'")))
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
def testPublicationMapping():Unit = {
|
||||
|
||||
mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT)
|
||||
val json = Source.fromInputStream(getClass.getResourceAsStream("publication.json")).mkString
|
||||
|
||||
|
||||
val oaf =DLIToOAF.convertDLIPublicationToOAF(mapper.readValue(json, classOf[DLIPublication]))
|
||||
|
||||
println(mapper.writeValueAsString(oaf))
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
def testExternalReferenceMapping():Unit = {
|
||||
|
||||
mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT)
|
||||
val json = Source.fromInputStream(getClass.getResourceAsStream("dataset.json")).mkString
|
||||
|
||||
|
||||
val oaf =DLIToOAF.convertDLIDatasetToExternalReference(mapper.readValue(json, classOf[DLIDataset]))
|
||||
|
||||
println(oaf)
|
||||
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
|
||||
@Test
|
||||
def testRelationMapping():Unit = {
|
||||
|
||||
mapper.getSerializationConfig.enable(SerializationConfig.Feature.INDENT_OUTPUT)
|
||||
val json = Source.fromInputStream(getClass.getResourceAsStream("relation.json")).mkString
|
||||
|
||||
|
||||
val oaf =DLIToOAF.convertDLIRelation(mapper.readValue(json, classOf[DLIRelation]))
|
||||
|
||||
println(mapper.writeValueAsString(oaf))
|
||||
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,101 @@
|
|||
{
|
||||
"dataInfo": {
|
||||
"invisible": false,
|
||||
"inferred": null,
|
||||
"deletedbyinference": false,
|
||||
"trust": "0.9",
|
||||
"inferenceprovenance": null,
|
||||
"provenanceaction": null
|
||||
},
|
||||
"lastupdatetimestamp": null,
|
||||
"id": "60|719f19e5a996de1b87cddf93871bf2d4",
|
||||
"originalId": [
|
||||
"a0a3p2gws9::uniprot"
|
||||
],
|
||||
"collectedfrom": [
|
||||
{
|
||||
"key": "dli_________::europe_pmc__",
|
||||
"value": "Europe PMC",
|
||||
"dataInfo": null
|
||||
}
|
||||
],
|
||||
"pid": [
|
||||
{
|
||||
"value": "acc63471",
|
||||
"qualifier": {
|
||||
"classid": "ena",
|
||||
"classname": "ena",
|
||||
"schemeid": "dnet:pid_types",
|
||||
"schemename": "dnet:pid_types"
|
||||
},
|
||||
"dataInfo": null
|
||||
}
|
||||
],
|
||||
"dateofcollection": "2019-07-05T12:47:11.545+02:00",
|
||||
"dateoftransformation": null,
|
||||
"extraInfo": null,
|
||||
"oaiprovenance": null,
|
||||
"author": null,
|
||||
"resulttype": {
|
||||
"classid": "dataset",
|
||||
"classname": "dataset",
|
||||
"schemeid": "dataset",
|
||||
"schemename": "dataset"
|
||||
},
|
||||
"language": null,
|
||||
"country": null,
|
||||
"subject": [],
|
||||
"title": [
|
||||
{
|
||||
"value": "CMD domain-containing protein",
|
||||
"qualifier": null,
|
||||
"dataInfo": null
|
||||
}
|
||||
],
|
||||
"relevantdate": [
|
||||
{
|
||||
"value": "2019-07-15T16:14:28.636",
|
||||
"qualifier": {
|
||||
"classid": "resolvedDate",
|
||||
"classname": "resolvedDate",
|
||||
"schemeid": "dnet::date",
|
||||
"schemename": "dnet::date"
|
||||
},
|
||||
"dataInfo": null
|
||||
}
|
||||
],
|
||||
"description": null,
|
||||
"dateofacceptance": null,
|
||||
"publisher": {
|
||||
"value": "UniProt",
|
||||
"dataInfo": null
|
||||
},
|
||||
"embargoenddate": null,
|
||||
"source": null,
|
||||
"fulltext": null,
|
||||
"format": null,
|
||||
"contributor": null,
|
||||
"resourcetype": null,
|
||||
"coverage": null,
|
||||
"bestaccessright": null,
|
||||
"context": null,
|
||||
"externalReference": null,
|
||||
"instance": [],
|
||||
"storagedate": null,
|
||||
"device": null,
|
||||
"size": null,
|
||||
"version": null,
|
||||
"lastmetadataupdate": null,
|
||||
"metadataversionnumber": null,
|
||||
"geolocation": null,
|
||||
"originalObjIdentifier": "europe_pmc__::719f19e5a996de1b87cddf93871bf2d4",
|
||||
"dlicollectedfrom": [
|
||||
{
|
||||
"id": "dli_________::europe_pmc__",
|
||||
"name": "Europe PMC",
|
||||
"completionStatus": "complete",
|
||||
"collectionMode": null
|
||||
}
|
||||
],
|
||||
"completionStatus": "complete"
|
||||
}
|
|
@ -0,0 +1,128 @@
|
|||
{
|
||||
"dataInfo": {
|
||||
"invisible": false,
|
||||
"inferred": null,
|
||||
"deletedbyinference": false,
|
||||
"trust": "0.9",
|
||||
"inferenceprovenance": null,
|
||||
"provenanceaction": null
|
||||
},
|
||||
"lastupdatetimestamp": null,
|
||||
"id": "50|9e117414be07bf03cbce8889d22d661a",
|
||||
"originalId": [
|
||||
"9e117414be07bf03cbce8889d22d661a"
|
||||
],
|
||||
"collectedfrom": [
|
||||
{
|
||||
"key": "dli_________::crossref",
|
||||
"value": "Crossref",
|
||||
"dataInfo": null
|
||||
}
|
||||
],
|
||||
"pid": [
|
||||
{
|
||||
"value": "10.1007/978-94-017-3490-5_15",
|
||||
"qualifier": {
|
||||
"classid": "doi",
|
||||
"classname": "doi",
|
||||
"schemeid": "dnet:pid_types",
|
||||
"schemename": "dnet:pid_types"
|
||||
},
|
||||
"dataInfo": null
|
||||
}
|
||||
],
|
||||
"dateofcollection": "2020-06-08T07:28:55.731Z",
|
||||
"dateoftransformation": null,
|
||||
"extraInfo": null,
|
||||
"oaiprovenance": null,
|
||||
"author": [
|
||||
{
|
||||
"fullname": "Calcaterra Domenico",
|
||||
"name": null,
|
||||
"surname": null,
|
||||
"rank": null,
|
||||
"pid": null,
|
||||
"affiliation": null
|
||||
},
|
||||
{
|
||||
"fullname": "Parise Mario",
|
||||
"name": null,
|
||||
"surname": null,
|
||||
"rank": null,
|
||||
"pid": null,
|
||||
"affiliation": null
|
||||
}
|
||||
],
|
||||
"resulttype": {
|
||||
"classid": "publication",
|
||||
"classname": "publication",
|
||||
"schemeid": "publication",
|
||||
"schemename": "publication"
|
||||
},
|
||||
"language": null,
|
||||
"country": null,
|
||||
"subject":[
|
||||
{
|
||||
"value":"Strain-linked information about bacterial and archaeal biodiversity",
|
||||
"qualifier":{
|
||||
"classid":"dnet:subject",
|
||||
"classname":"dnet:subject",
|
||||
"schemeid":"",
|
||||
"schemename":""
|
||||
},
|
||||
"dataInfo":null
|
||||
}
|
||||
],
|
||||
"title": [
|
||||
{
|
||||
"value": "The Contribution of Historical Information in the Assessment of Landslide Hazard",
|
||||
"qualifier": null,
|
||||
"dataInfo": null
|
||||
}
|
||||
],
|
||||
"relevantdate": [
|
||||
{
|
||||
"value": "2013-01-29T16:50:44Z",
|
||||
"qualifier": {
|
||||
"classid": "date",
|
||||
"classname": "date",
|
||||
"schemeid": "dnet::date",
|
||||
"schemename": "dnet::date"
|
||||
},
|
||||
"dataInfo": null
|
||||
}
|
||||
],
|
||||
"description": [
|
||||
{
|
||||
"value": null,
|
||||
"dataInfo": null
|
||||
}
|
||||
],
|
||||
"dateofacceptance": null,
|
||||
"publisher": {
|
||||
"value": "Springer Netherlands",
|
||||
"dataInfo": null
|
||||
},
|
||||
"embargoenddate": null,
|
||||
"source": null,
|
||||
"fulltext": null,
|
||||
"format": null,
|
||||
"contributor": null,
|
||||
"resourcetype": null,
|
||||
"coverage": null,
|
||||
"bestaccessright": null,
|
||||
"context": null,
|
||||
"externalReference": null,
|
||||
"instance": [],
|
||||
"journal": null,
|
||||
"originalObjIdentifier": "dli_resolver::9e117414be07bf03cbce8889d22d661a",
|
||||
"dlicollectedfrom": [
|
||||
{
|
||||
"id": "dli_________::crossref",
|
||||
"name": "Crossref",
|
||||
"completionStatus": "complete",
|
||||
"collectionMode": "resolved"
|
||||
}
|
||||
],
|
||||
"completionStatus": "complete"
|
||||
}
|
|
@ -0,0 +1,23 @@
|
|||
{
|
||||
"subRelType": null,
|
||||
"relClass": "datacite",
|
||||
"dataInfo": {
|
||||
"deletedbyinference": false,
|
||||
"provenanceaction": null,
|
||||
"inferred": null,
|
||||
"inferenceprovenance": null,
|
||||
"invisible": false,
|
||||
"trust": "0.9"
|
||||
},
|
||||
"target": "50|00062410e2a15322480277d063c181bb",
|
||||
"lastupdatetimestamp": null,
|
||||
"relType": "IsReferencedBy",
|
||||
"source": "60|4ee78ab329b49416b45c3774c132f244",
|
||||
"collectedFrom": [
|
||||
{
|
||||
"dataInfo": null,
|
||||
"value": "Europe PMC",
|
||||
"key": "dli_________::europe_pmc__"
|
||||
}
|
||||
]
|
||||
}
|
Loading…
Reference in New Issue