New propagation of ORCID to result exploiting the semantic relation connecting them. R has author with orcid o, R is bounf by strong semantic relationship with R1 that has the same author withouth orcid, then o is also associated to the author in R1

This commit is contained in:
Miriam Baglioni 2020-02-28 18:22:41 +01:00
parent 833c83c694
commit 2b7b05fb29
1 changed files with 187 additions and 60 deletions

View File

@ -3,19 +3,26 @@ package eu.dnetlib.dhp.orcidtoresultfromsemrel;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.TypedRow;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.schema.oaf.*;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.io.Text;
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.api.java.function.Function;
import org.apache.spark.api.java.function.PairFunction;
import org.apache.spark.sql.SparkSession;
import scala.Tuple2;
import java.io.File;
import java.util.Arrays;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static eu.dnetlib.dhp.PropagationConstant.getResultResultSemRel;
import static eu.dnetlib.dhp.PropagationConstant.*;
public class SparkOrcidToResultFromSemRelJob {
public static void main(String[] args) throws Exception {
@ -46,78 +53,198 @@ public class SparkOrcidToResultFromSemRelJob {
.map(item -> new ObjectMapper().readValue(item._2().toString(), Relation.class)).cache();
JavaPairRDD<String, TypedRow> result_result = getResultResultSemRel(allowedsemrel, relations);
}
}
/*
public class PropagationOrcidToResultMapper extends TableMapper<ImmutableBytesWritable, Text> {
private static final Log log = LogFactory.getLog(PropagationOrcidToResultMapper.class); // NOPMD by marko on 11/24/08 5:02 PM
private Text valueOut;
private ImmutableBytesWritable keyOut;
private String[] sem_rels;
private String trust;
@Override
protected void setup(final Context context) throws IOException, InterruptedException {
super.setup(context);
valueOut = new Text();
keyOut = new ImmutableBytesWritable();
JavaRDD<Publication> publications = sc.sequenceFile(inputPath + "/publication", Text.class, Text.class)
.map(item -> new ObjectMapper().readValue(item._2().toString(), Publication.class));
JavaRDD<Dataset> datasets = sc.sequenceFile(inputPath + "/dataset", Text.class, Text.class)
.map(item -> new ObjectMapper().readValue(item._2().toString(), Dataset.class));
JavaRDD<Software> software = sc.sequenceFile(inputPath + "/software", Text.class, Text.class)
.map(item -> new ObjectMapper().readValue(item._2().toString(), Software.class));
JavaRDD<OtherResearchProduct> other = sc.sequenceFile(inputPath + "/otherresearchproduct", Text.class, Text.class)
.map(item -> new ObjectMapper().readValue(item._2().toString(), OtherResearchProduct.class));
sem_rels = context.getConfiguration().getStrings("propagatetoorcid.semanticrelations", DEFAULT_RESULT_RELATION_SET);
trust = context.getConfiguration().get("propagatetoorcid.trust","0.85");
//get the results having at least one author pid we are interested in
JavaPairRDD<String, TypedRow> resultswithorcid = publications.map(p -> getTypedRow(p))
.filter(p -> !(p == null))
.mapToPair(toPair())
.union(datasets.map(p -> getTypedRow(p))
.filter(p -> !(p == null))
.mapToPair(toPair()))
.union(software.map(p -> getTypedRow(p))
.filter(p -> !(p == null))
.mapToPair(toPair()))
.union(other.map(p -> getTypedRow(p))
.filter(p -> !(p == null))
.mapToPair(toPair()));
JavaPairRDD<String, TypedRow> to_add_orcid_to_result = resultswithorcid.join(result_result)
.map(p -> p._2()._1().setSourceId(p._2()._2().getTargetId())) //associate the pid of the result (target) which should get the orcid to the typed row containing the authors with the orcid from the result(source)
.mapToPair(toPair());
JavaPairRDD<String, Result> pubs = publications.mapToPair(p -> new Tuple2<>(p.getId(),p));
JavaPairRDD<String, Result> dss = datasets.mapToPair(p -> new Tuple2<>(p.getId(),p));
JavaPairRDD<String, Result> sfw = software.mapToPair(p -> new Tuple2<>(p.getId(),p));
JavaPairRDD<String, Result> orp = other.mapToPair(p -> new Tuple2<>(p.getId(),p));
updateResult(pubs, to_add_orcid_to_result, outputPath, "publication");
updateResult(dss, to_add_orcid_to_result, outputPath, "dataset");
updateResult(sfw, to_add_orcid_to_result, outputPath, "software");
updateResult(orp, to_add_orcid_to_result, outputPath, "otherresearchproduct");
}
@Override
protected void map(final ImmutableBytesWritable keyIn, final Result value, final Context context) throws IOException, InterruptedException {
final TypeProtos.Type type = OafRowKeyDecoder.decode(keyIn.copyBytes()).getType();
final OafProtos.OafEntity entity = getEntity(value, type);//getEntity already verified that it is not delByInference
private static Author enrichAutor(Author autoritative_author, Author author) {
boolean toaddpid = false;
if (entity != null) {
if (StringUtils.isNoneEmpty(autoritative_author.getSurname())) {
if (StringUtils.isNoneEmpty(author.getSurname())) {
if (autoritative_author.getSurname().trim().equalsIgnoreCase(author.getSurname().trim())) {
if (type == TypeProtos.Type.result){
Set<String> result_result = new HashSet<>();
//verifico se il risultato ha una relazione semantica verso uno o piu' risultati.
//per ogni risultato linkato con issupplementto o issupplementedby emetto:
// id risultato linkato come chiave,
// id risultato oggetto del mapping e lista degli autori del risultato oggetto del mapper come value
for(String sem : sem_rels){
result_result.addAll(getRelationTarget(value, sem, context, COUNTER_PROPAGATION));
}
if(!result_result.isEmpty()){
List<String> authorlist = getAuthorList(entity.getResult().getMetadata().getAuthorList());
Emit e = new Emit();
e.setId(Bytes.toString(keyIn.get()));
e.setAuthor_list(authorlist);
valueOut.set(Value.newInstance(new Gson().toJson(e, Emit.class),
trust,
Type.fromsemrel).toJson());
for (String result: result_result){
keyOut.set(Bytes.toBytes(result));
context.write(keyOut,valueOut);
context.getCounter(COUNTER_PROPAGATION,"emit for sem_rel").increment(1);
//have the same surname. Check the name
if (StringUtils.isNoneEmpty(autoritative_author.getName())) {
if (StringUtils.isNoneEmpty(author.getName())) {
if (autoritative_author.getName().trim().equalsIgnoreCase(author.getName().trim())) {
toaddpid = true;
}
//they could be differently written (i.e. only the initials of the name in one of the two
if (autoritative_author.getName().trim().substring(0, 0).equalsIgnoreCase(author.getName().trim().substring(0, 0))) {
toaddpid = true;
}
}
}
//emetto anche id dell'oggetto del mapper come chiave e lista degli autori come valore
e.setId(keyIn.toString());
e.setAuthor_list(authorlist);
valueOut.set(Value.newInstance(new Gson().toJson(e, Emit.class), trust, Type.fromresult).toJson());
context.write(keyIn, valueOut);
context.getCounter(COUNTER_PROPAGATION,"emit for result with orcid").increment(1);
}
}
}
}
private List<String> getAuthorList(List<FieldTypeProtos.Author> author_list){
return author_list.stream().map(a -> new JsonFormat().printToString(a)).collect(Collectors.toList());
if (toaddpid){
StructuredProperty pid = new StructuredProperty();
for(StructuredProperty sp : autoritative_author.getPid()){
if (PROPAGATION_AUTHOR_PID.equals(sp.getQualifier().getClassid())){
pid.setValue(sp.getValue());
pid.setQualifier(getQualifier(sp.getQualifier().getClassid(),sp.getQualifier().getClassname() ));
pid.setDataInfo(getDataInfo(PROPAGATION_DATA_INFO_TYPE, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_ID, PROPAGATION_ORCID_TO_RESULT_FROM_SEM_REL_CLASS_NAME));
if(author.getPid() == null){
author.setPid(Arrays.asList(pid));
}else{
author.getPid().add(pid);
}
}
}
return author;
}
return null;
}
private static void updateResult(JavaPairRDD<String, Result> results, JavaPairRDD<String, TypedRow> toupdateresult, String outputPath, String type) {
results.leftOuterJoin(toupdateresult)
.map(p -> {
Result r = p._2()._1();
if (p._2()._2().isPresent()){
List<Author> autoritative_authors = p._2()._2().get().getAuthors();
List<Author> to_enrich_authors = r.getAuthor();
//.stream().filter(a -> !containsAllowedPid(a))
//.collect(Collectors.toList());
r.setAuthor(to_enrich_authors
.stream()
.map(a -> {
if (containsAllowedPid(a)) {
return a;
}
List<Author> lst = autoritative_authors.stream()
.map(aa -> enrichAutor(aa, a)).filter(au -> !(au == null)).collect(Collectors.toList());
if(lst.size() == 0){
return a;
}
return lst.get(0);//Each author can be enriched at most once. It cannot be the same as many different people
}).collect(Collectors.toList()));
}
return r;
})
.map(p -> new ObjectMapper().writeValueAsString(p))
.saveAsTextFile(outputPath+"/"+type);
}
private static TypedRow getTypedRow(Result p) {
TypedRow tp = new TypedRow();
tp.setSourceId(p.getId());
List<Author> authorList = p.getAuthor()
.stream()
.map(a -> {
if (a.getPid().stream().map(pid -> {
if (PROPAGATION_AUTHOR_PID.equals(pid.getQualifier().getClassid())) {
return a;
}
return null;
}).filter(aut -> !(aut == null)).collect(Collectors.toList()).size() > 0){
return a;
}
return null;
}).filter(a -> !(a == null)).collect(Collectors.toList());
tp.setAuthors(authorList);
if(authorList.size() > 0){
return tp;
}
return null;
}
private static boolean containsAllowedPid(Author a){
return (a.getPid().stream().map(pid -> {
if (PROPAGATION_AUTHOR_PID.equals(pid.getQualifier().getClassid())) {
return true;
}
return false;
}).filter(aut -> (aut == true)).collect(Collectors.toList()).size()) > 0;
}
}
*/
/*private ResultProtos.Result.Metadata.Builder searchMatch(List<FieldTypeProtos.Author> author_list){
ResultProtos.Result.Metadata.Builder metadataBuilder = ResultProtos.Result.Metadata.newBuilder();
boolean updated = false;
for (FieldTypeProtos.Author a: author_list){
FieldTypeProtos.Author.Builder author = searchAuthor(a, autoritative_authors);
if(author != null){
updated = true;
metadataBuilder.addAuthor(author);
}else{
metadataBuilder.addAuthor(FieldTypeProtos.Author.newBuilder(a));
}
}
if(updated)
return metadataBuilder;
return null;
}
private FieldTypeProtos.Author.Builder searchAuthor(FieldTypeProtos.Author a, List<FieldTypeProtos.Author> author_list){
if(containsOrcid(a.getPidList()))
return null;
for(FieldTypeProtos.Author autoritative_author : author_list) {
if (equals(autoritative_author, a)) {
if(!containsOrcid(a.getPidList()))
return update(a, autoritative_author);
}
}
return null;
}
private boolean containsOrcid(List<FieldTypeProtos.KeyValue> pidList){
if(pidList == null)
return false;
return pidList
.stream()
.filter(kv -> kv.getKey().equals(PropagationConstants.AUTHOR_PID))
.collect(Collectors.toList()).size() > 0;
}
*/