[Person] added test for the inclusion of relations from publisher web pages

This commit is contained in:
Miriam Baglioni 2024-10-31 14:43:01 +01:00
parent a6ecb0f825
commit 4152f00338
5 changed files with 249 additions and 188 deletions

View File

@ -1,60 +1,61 @@
package eu.dnetlib.dhp.schema.oaf;
import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.commons.lang3.builder.HashCodeBuilder;
public class HashableKeyValue extends KeyValue{
public class HashableKeyValue extends KeyValue {
public static HashableKeyValue newInstance(String key, String value) {
if (value == null) {
return null;
}
final HashableKeyValue kv = new HashableKeyValue();
kv.setValue(value);
kv.setKey(key);
public static HashableKeyValue newInstance(String key, String value) {
if (value == null) {
return null;
}
final HashableKeyValue kv = new HashableKeyValue();
kv.setValue(value);
kv.setKey(key);
return kv;
}
return kv;
}
public static HashableKeyValue newInstance(KeyValue kv) {
HashableKeyValue hkv = new HashableKeyValue();
hkv.setKey(kv.getKey());
hkv.setValue(kv.getValue());
public static HashableKeyValue newInstance(KeyValue kv) {
HashableKeyValue hkv = new HashableKeyValue();
hkv.setKey(kv.getKey());
hkv.setValue(kv.getValue());
return hkv;
}
return hkv;
}
public static KeyValue toKeyValue(HashableKeyValue hkv) {
KeyValue kv = new KeyValue();
kv.setKey(hkv.getKey());
kv.setValue(hkv.getValue());
public static KeyValue toKeyValue(HashableKeyValue hkv) {
KeyValue kv = new KeyValue();
kv.setKey(hkv.getKey());
kv.setValue(hkv.getValue());
return kv;
}
return kv;
}
@Override
public int hashCode() {
return new HashCodeBuilder(11, 91)
.append(getKey())
.append(getValue())
.hashCode();
}
@Override
public int hashCode() {
return new HashCodeBuilder(11, 91)
.append(getKey())
.append(getValue())
.hashCode();
}
@Override
public boolean equals(Object obj) {
if (obj == null) {
return false;
}
if (obj == this) {
return true;
}
if (obj.getClass() != getClass()) {
return false;
}
final HashableKeyValue rhs = (HashableKeyValue) obj;
return new EqualsBuilder()
.append(getKey(), rhs.getKey())
.append(getValue(), rhs.getValue())
.isEquals();
}
@Override
public boolean equals(Object obj) {
if (obj == null) {
return false;
}
if (obj == this) {
return true;
}
if (obj.getClass() != getClass()) {
return false;
}
final HashableKeyValue rhs = (HashableKeyValue) obj;
return new EqualsBuilder()
.append(getKey(), rhs.getKey())
.append(getValue(), rhs.getValue())
.isEquals();
}
}

View File

@ -296,9 +296,12 @@ public class MergeUtils {
right = s;
}
List<KeyValue> collect = unionDistinctLists(left.stream().map(HashableKeyValue::newInstance).collect(Collectors.toList()),
right.stream().map(HashableKeyValue::newInstance).collect(Collectors.toList()), trust)
.stream().map(HashableKeyValue::toKeyValue).collect(Collectors.toList());
List<KeyValue> collect = unionDistinctLists(
left.stream().map(HashableKeyValue::newInstance).collect(Collectors.toList()),
right.stream().map(HashableKeyValue::newInstance).collect(Collectors.toList()), trust)
.stream()
.map(HashableKeyValue::toKeyValue)
.collect(Collectors.toList());
return collect;
}

View File

@ -13,11 +13,6 @@ import java.sql.SQLException;
import java.util.*;
import java.util.stream.Collectors;
import eu.dnetlib.dhp.actionmanager.ror.GenerateRorActionSetJob;
import eu.dnetlib.dhp.common.person.CoAuthorshipIterator;
import eu.dnetlib.dhp.common.person.Coauthors;
import eu.dnetlib.dhp.schema.oaf.*;
import eu.dnetlib.dhp.schema.oaf.utils.*;
import org.apache.commons.cli.ParseException;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
@ -30,8 +25,8 @@ 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.functions.*;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.functions.*;
import org.jetbrains.annotations.NotNull;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -39,17 +34,23 @@ import org.spark_project.jetty.util.StringUtil;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.actionmanager.ror.GenerateRorActionSetJob;
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.*;
import eu.dnetlib.dhp.schema.oaf.utils.*;
import eu.dnetlib.dhp.utils.DHPUtils;
import scala.Tuple2;
import scala.collection.Seq;
public class ExtractPerson implements Serializable {
private static final Logger log = LoggerFactory.getLogger(ExtractPerson.class);
@ -67,6 +68,7 @@ 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 OPENORGS_PREFIX = "20|openorgs____::";
private static final String PERSON_PREFIX = ModelSupport.getIdPrefix(Person.class) + "|orcid_______";
public static final String ORCID_AUTHORS_CLASSID = "sysimport:crosswalk:orcid";
public static final String ORCID_AUTHORS_CLASSNAME = "Imported from ORCID";
@ -93,18 +95,18 @@ public class ExtractPerson implements Serializable {
"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");
.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 {
@ -152,8 +154,8 @@ public class ExtractPerson implements Serializable {
HdfsSupport.remove(outputPath, spark.sparkContext().hadoopConfiguration());
extractInfoForActionSetFromORCID(spark, inputPath, workingDir);
extractInfoForActionSetFromProjects(
dbUrl, dbUser, dbPassword, workingDir + "/project", hdfsNameNode, isSparkSessionManaged);
extractInfoForActionSetFromPublisher(spark,publisherInputPath, workingDir);
dbUrl, dbUser, dbPassword, workingDir + "/project", hdfsNameNode, isSparkSessionManaged);
extractInfoForActionSetFromPublisher(spark, publisherInputPath, workingDir);
createActionSet(spark, outputPath, workingDir);
});
@ -161,104 +163,143 @@ public class ExtractPerson implements Serializable {
private static void extractInfoForActionSetFromPublisher(SparkSession spark, String inputPath, String workingDir) {
//Read the publishers output
Dataset<Row> df = spark
.read()
.schema(
"`DOI` STRING, " +
"`Authors` ARRAY<STRUCT<`Corresponding` : STRING, " +
"`Contributor_roles` : ARRAY<STRUCT<`Scheme`:STRING, `Value`:STRING>> ," +
"`Name` : STRUCT<`Full`:STRING, `First` : STRING, `Last`: STRING>, " +
"`Matchings`: ARRAY<STRUCT<`PID`:STRING, `Value`:STRING,`Confidence`:DOUBLE, `Status`:STRING>>, " +
"`PIDs` : STRUCT<`Schema`:STRING , `Value`: STRING>>>")
.json(inputPath + "/*/")
.where("DOI is not null");
Dataset<Row> df = spark
.read()
.schema(
"`DOI` STRING, " +
"`Authors` ARRAY<STRUCT<`Corresponding` : STRING, " +
"`Contributor_roles` : ARRAY<STRUCT<`Schema`:STRING, `Value`:STRING>> ," +
"`Name` : STRUCT<`Full`:STRING, `First` : STRING, `Last`: STRING>, " +
"`Matchings`: ARRAY<STRUCT<`PID`:STRING, `Value`:STRING,`Confidence`:DOUBLE, `Status`:STRING>>, " +
"`PIDs` : STRUCT<`Schema`:STRING , `Value`: STRING>>>")
.json(inputPath + "/*/")
.where("DOI is not null");
//Select the relevant information
Dataset<Row> authors = df.selectExpr("DOI", "explode(Authors) as author")
.selectExpr("DOI", "author.Contributor_roles as roles",
"author.Corresponding as corresponding", "author.Matchings as affs",
"authors.PIDs as pid")
.where("pid.Schema=='ORCID'")
.selectExpr("explode affs as affiliation", "DOI", "corresponding", "roles", "pid.Value as orcid")
.where("aff.Status == 'active")
.selectExpr("affiliation", "DOI", "corresponding", "explode roles as role", "orcid");
Dataset<Row> authors = df
.selectExpr("DOI", "explode(Authors) as author")
.selectExpr(
"DOI", "author.Contributor_roles as roles",
"author.Corresponding as corresponding", "author.Matchings as affs",
"author.PIDs as pid")
.where("pid.Schema = 'ORCID'")
.selectExpr("explode (affs) as affiliation", "DOI", "corresponding", "roles", "pid.Value as orcid")
.where("affiliation.Status = 'active'")
.selectExpr(
"affiliation.Value as orgid", "affiliation.PID as orgpid", "affiliation.Confidence as trust", "DOI",
"corresponding", "roles", "orcid");
//create the relation dataset with possible redundant relations
Dataset<Relation> relations = authors.flatMap((FlatMapFunction<Row, Relation>) a ->
Arrays.asList(getAuthorshipRelation(a), getAffiliationRelation(a)).iterator(), Encoders.bean(Relation.class))
.unionAll(
authors = authors
.where("roles is null")
.selectExpr("*", " '' AS roleschema", " '' AS rolevalue")
.drop("roles")
.unionAll(
authors
.where("roles is not null")
.selectExpr("orgid", "orgpid", "trust", "DOI", "corresponding", "explode(roles) as role", "orcid")
.selectExpr("*", "role.Schema as roleschema", "role.Value as rolevalue")
.drop("role"));
df.selectExpr("DOI","explode Authors as author").where("author.PIDs.Schema == 'ORCID")
.selectExpr("DOI", "author.PIDs.value as orcid")
.groupByKey((MapFunction<Row, String>) r -> r.getAs("DOI"), Encoders.STRING() )
// create the relation dataset with possible redundant relations
Dataset<Relation> relations = authors
.flatMap(
(FlatMapFunction<Row, Relation>) a -> Arrays
.asList(getAuthorshipRelation(a), getAffiliationRelation(a))
.iterator(),
Encoders.bean(Relation.class))
.unionAll(
df
.selectExpr("DOI", "explode (Authors) as author")
.where("author.PIDs.Schema = 'ORCID'")
.selectExpr("DOI", "author.PIDs.Value as orcid")
.groupByKey((MapFunction<Row, String>) r -> r.getAs("DOI"), Encoders.STRING())
.mapGroups((MapGroupsFunction<String, Row, Coauthors>) (k,it) -> extractCoAuthorsRow(it),
.mapGroups(
(MapGroupsFunction<String, Row, Coauthors>) (k, it) -> extractCoAuthorsRow(it),
Encoders.bean(Coauthors.class))
.flatMap(
.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)));
.groupByKey((MapFunction<Relation, String>) r -> r.getSource() + r.getTarget(), Encoders.STRING())
.mapGroups(
(MapGroupsFunction<String, Relation, Relation>) (k, it) -> it.next(),
Encoders.bean(Relation.class)));
//produce one dataset with only one relation per source, target and semantics. Eventually extend the list of properties
// produce one dataset with only one relation per source, target and semantics. Eventually extend the list of
// properties
relations.groupByKey((MapFunction<Relation, String>) r -> r.getSource() + r.getRelClass() + r.getTarget(), Encoders.STRING() )
.mapGroups((MapGroupsFunction<String, Relation, Relation>) (k,it) -> mergeRelation(it), Encoders.bean(Relation.class) )
.write()
.mode(SaveMode.Overwrite)
.option("compression","gzip")
.json(workingDir + "/publishers");
relations
.groupByKey(
(MapFunction<Relation, String>) r -> r.getSource() + r.getRelClass() + r.getTarget(), Encoders.STRING())
.mapGroups(
(MapGroupsFunction<String, Relation, Relation>) (k, it) -> mergeRelation(it),
Encoders.bean(Relation.class))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(workingDir + "/publishers");
}
private static Relation mergeRelation(Iterator<Relation> it) {
Relation r = it.next();
while (it.hasNext()) {
Relation r1 = it.next();
r = MergeUtils.mergeRelation(r, r1);
}
private static Relation mergeRelation(Iterator<Relation> it){
Relation r = it.next();
while (it.hasNext()){
Relation r1 = it.next();
r = MergeUtils.mergeRelation(r, r1);
}
return r;
}
return r;
}
private static @NotNull Relation getAuthorshipRelation(Row a) {
String target = DOI_PREFIX
+ IdentifierFactory
+ IdentifierFactory
.md5(PidCleaner.normalizePidValue(PidType.doi.toString(), a.getAs("DOI")));
;
String source = PERSON_PREFIX + "::" + IdentifierFactory.md5(a.getAs("orcid"));
Relation relation = OafMapperUtils
.getRelation(
source, target, ModelConstants.RESULT_PERSON_RELTYPE,
ModelConstants.RESULT_PERSON_SUBRELTYPE,
ModelConstants.RESULT_PERSON_HASAUTHORED,
OafMapperUtils.listKeyValues(OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME),
null,
null);
.getRelation(
source, target, ModelConstants.RESULT_PERSON_RELTYPE,
ModelConstants.RESULT_PERSON_SUBRELTYPE,
ModelConstants.RESULT_PERSON_HASAUTHORED,
OafMapperUtils.listKeyValues(OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME),
null,
null);
if(StringUtil.isNotBlank(a.getAs("aff.Value"))){
if (StringUtil.isNotBlank(a.getAs("orgid"))) {
KeyValue kv = new KeyValue();
kv.setKey("declared_affiliation");
kv.setValue(ROR_PREFIX
+ IdentifierFactory.md5(PidCleaner.normalizePidValue("ROR", a.getAs("aff.Value"))));
relation.getProperties().add(new KeyValue());
if (((String) a.getAs("orgpid")).equalsIgnoreCase("ror"))
kv.setValue(a.getAs("orgid"));
else
kv
.setValue(
OPENORGS_PREFIX
+ IdentifierFactory.md5(PidCleaner.normalizePidValue("OPENORGS", a.getAs("orgid"))));
if (!Optional.ofNullable(relation.getProperties()).isPresent())
relation.setProperties(new ArrayList<>());
relation.getProperties().add(kv);
}
if(StringUtil.isNotBlank(a.getAs("corresponding")) && a.getAs("corresponding").equals("true")){
if (Optional.ofNullable(a.getAs("corresponding")).isPresent() &&
a.getAs("corresponding").equals("true")) {
KeyValue kv = new KeyValue();
kv.setKey("corresponding");
kv.setValue("true");
relation.getProperties().add(new KeyValue());
if (!Optional.ofNullable(relation.getProperties()).isPresent())
relation.setProperties(new ArrayList<>());
relation.getProperties().add(kv);
}
if(StringUtil.isNotBlank(a.getAs("role"))){
if (StringUtil.isNotBlank(a.getAs("roleschema"))) {
KeyValue kv = new KeyValue();
kv.setKey("role");
String role = (String) a.getAs("role.Schema")
+(String) a.getAs("role.value");
String role = (String) a.getAs("roleschema")
+ (String) a.getAs("rolevalue");
kv.setValue(role);
relation.getProperties().add(new KeyValue());
if (!Optional.ofNullable(relation.getProperties()).isPresent())
relation.setProperties(new ArrayList<>());
relation.getProperties().add(kv);
}
return relation;
}
@ -267,31 +308,31 @@ public class ExtractPerson implements Serializable {
String source = PERSON_PREFIX + "::" + IdentifierFactory.md5(a.getAs("orcid"));
String target = ROR_PREFIX
+ IdentifierFactory.md5(PidCleaner.normalizePidValue("ROR", a.getAs("aff.Value")));
+ IdentifierFactory.md5(PidCleaner.normalizePidValue("ROR", a.getAs("orgid")));
Relation relation = OafMapperUtils
.getRelation(
source, target, ModelConstants.ORG_PERSON_RELTYPE,
ModelConstants.ORG_PERSON_SUBRELTYPE,
ModelConstants.ORG_PERSON_PARTICIPATES,
OafMapperUtils.listKeyValues(OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME),
null,
null);
.getRelation(
source, target, ModelConstants.ORG_PERSON_RELTYPE,
ModelConstants.ORG_PERSON_SUBRELTYPE,
ModelConstants.ORG_PERSON_PARTICIPATES,
OafMapperUtils.listKeyValues(OPENAIRE_DATASOURCE_ID, OPENAIRE_DATASOURCE_NAME),
null,
null);
return relation;
}
private static void extractInfoForActionSetFromProjects(
String dbUrl, String dbUser, String dbPassword, String hdfsPath, String hdfsNameNode, Boolean exec) throws IOException {
String dbUrl, String dbUser, String dbPassword, String hdfsPath, String hdfsNameNode, Boolean exec)
throws IOException {
Configuration conf = new Configuration();
conf.set("fs.defaultFS", hdfsNameNode);
if (exec) {
Configuration conf = new Configuration();
conf.set("fs.defaultFS", hdfsNameNode);
FileSystem fileSystem = FileSystem.get(conf);
Path hdfsWritePath = new Path(hdfsPath);
FileSystem fileSystem = FileSystem.get(conf);
Path hdfsWritePath = new Path(hdfsPath);
if(exec){
FSDataOutputStream fos = fileSystem.create(hdfsWritePath);
try (DbClient dbClient = new DbClient(dbUrl, dbUser, dbPassword)) {
try (BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fos, StandardCharsets.UTF_8))) {
@ -303,7 +344,6 @@ public class ExtractPerson implements Serializable {
}
}
}
public static Relation getRelationWithProject(ResultSet rs) {
@ -328,7 +368,7 @@ public class ExtractPerson implements Serializable {
source, target, ModelConstants.PROJECT_PERSON_RELTYPE, ModelConstants.PROJECT_PERSON_SUBRELTYPE,
ModelConstants.PROJECT_PERSON_PARTICIPATES,
collectedfromOpenAIRE,
FUNDERDATAINFO,
FUNDERDATAINFO,
null);
relation.setValidated(true);
@ -382,10 +422,10 @@ public class ExtractPerson implements Serializable {
getRelations(spark, workingDir + "/project")
.toJavaRDD()
.map(r -> new AtomicAction(r.getClass(), r)))
.union(
getRelations(spark, workingDir + "/publishers")
.toJavaRDD()
.map(r -> new AtomicAction(r.getClass(), r)))
.union(
getRelations(spark, workingDir + "/publishers")
.toJavaRDD()
.map(r -> new AtomicAction(r.getClass(), r)))
.mapToPair(
aa -> new Tuple2<>(new Text(aa.getClazz().getCanonicalName()),
new Text(OBJECT_MAPPER.writeValueAsString(aa))))
@ -535,13 +575,16 @@ public class ExtractPerson implements Serializable {
}
private static Dataset<Relation> getRelations(SparkSession spark, String path) {
return spark
.read()
.textFile(path)
.map(
(MapFunction<String, Relation>) value -> OBJECT_MAPPER
.readValue(value, Relation.class),
Encoders.bean(Relation.class));// spark.read().json(path).as(Encoders.bean(Relation.class));
if (HdfsSupport.exists(path, spark.sparkContext().hadoopConfiguration()))
return spark
.read()
.textFile(path)
.map(
(MapFunction<String, Relation>) value -> OBJECT_MAPPER
.readValue(value, Relation.class),
Encoders.bean(Relation.class));// spark.read().json(path).as(Encoders.bean(Relation.class));
else
return spark.emptyDataset(Encoders.bean(Relation.class));
}
private static Coauthors extractCoAuthors(Iterator<Tuple2<String, String>> it) {
@ -575,7 +618,7 @@ 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)),
ORCIDDATAINFO,
ORCIDDATAINFO,
null);
relation.setValidated(true);
@ -645,8 +688,8 @@ public class ExtractPerson implements Serializable {
source, target, ModelConstants.RESULT_PERSON_RELTYPE,
ModelConstants.RESULT_PERSON_SUBRELTYPE,
ModelConstants.RESULT_PERSON_HASAUTHORED,
Collections.singletonList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)),
ORCIDDATAINFO,
Collections.singletonList(OafMapperUtils.keyValue(orcidKey, ModelConstants.ORCID_DS)),
ORCIDDATAINFO,
null);
relation.setValidated(true);
return relation;

View File

@ -75,6 +75,7 @@ public class CreatePersonAS {
"/eu/dnetlib/dhp/actionmanager/person/")
.getPath();
//
// spark
// .read()
// .parquet(inputPath + "Authors")
@ -96,10 +97,11 @@ public class CreatePersonAS {
workingDir.toString() + "/actionSet1",
"-workingDir",
workingDir.toString() + "/working",
"-postgresUrl", "noneed",
"-postgresUser","noneed",
"-postgresPassword", "noneed",
"-publisherInputPath", getClass().getResource("/eu/dnetlib/dhp/actionmanager/personpublisher/")
"-postgresUrl", "noneed",
"-postgresUser", "noneed",
"-postgresPassword", "noneed",
"-publisherInputPath", getClass()
.getResource("/eu/dnetlib/dhp/actionmanager/personpublisher/")
.getPath()
});
@ -118,62 +120,73 @@ public class CreatePersonAS {
.map(value -> OBJECT_MAPPER.readValue(value._2().toString(), AtomicAction.class))
.map(aa -> ((Person) aa.getPayload()));
//
Assertions.assertEquals(7, people.count());
Assertions.assertEquals(8, people.count());
Assertions
.assertEquals(
"Paulo",
"Manuel Edelberto",
people
.filter(
p -> p.getPid().stream().anyMatch(id -> id.getValue().equalsIgnoreCase("0000-0002-3210-3034")))
p -> p.getPid().stream().anyMatch(id -> id.getValue().equalsIgnoreCase("0000-0003-0046-4895")))
.first()
.getGivenName());
Assertions
.assertEquals(
"Tavares",
"Ortega Coello",
people
.filter(
p -> p.getPid().stream().anyMatch(id -> id.getValue().equalsIgnoreCase("0000-0002-3210-3034")))
p -> p.getPid().stream().anyMatch(id -> id.getValue().equalsIgnoreCase("0000-0003-0046-4895")))
.first()
.getFamilyName());
Assertions
.assertEquals(
4,
1,
people
.filter(
p -> p.getPid().stream().anyMatch(id -> id.getValue().equalsIgnoreCase("0000-0002-3210-3034")))
p -> p.getPid().stream().anyMatch(id -> id.getValue().equalsIgnoreCase("0000-0003-0046-4895")))
.first()
.getAlternativeNames()
.size());
Assertions
.assertEquals(
4,
2,
people
.filter(
p -> p.getPid().stream().anyMatch(id -> id.getValue().equalsIgnoreCase("0000-0002-3210-3034")))
p -> p.getPid().stream().anyMatch(id -> id.getValue().equalsIgnoreCase("0000-0003-0046-4895")))
.first()
.getPid()
.size());
System.out
.println(
new ObjectMapper()
.writeValueAsString(
people
.filter(
p -> p
.getPid()
.stream()
.anyMatch(id -> id.getValue().equalsIgnoreCase("0000-0003-0046-4895")))
.first()));
Assertions
.assertTrue(
people
.filter(
p -> p.getPid().stream().anyMatch(id -> id.getValue().equalsIgnoreCase("0000-0002-3210-3034")))
p -> p.getPid().stream().anyMatch(id -> id.getValue().equalsIgnoreCase("0000-0003-0046-4895")))
.first()
.getPid()
.stream()
.anyMatch(
p -> p.getQualifier().getSchemename().equalsIgnoreCase("Scopus Author ID")
&& p.getValue().equalsIgnoreCase("15119405200")));
p -> p.getQualifier().getClassname().equalsIgnoreCase("Scopus Author ID")
&& p.getValue().equalsIgnoreCase("6603539671")));
Assertions
.assertEquals(
16,
19,
relations
.filter(r -> r.getRelClass().equalsIgnoreCase(ModelConstants.RESULT_PERSON_HASAUTHORED))
.count());
Assertions
.assertEquals(
14,
16,
relations
.filter(r -> r.getRelClass().equalsIgnoreCase(ModelConstants.PERSON_PERSON_HASCOAUTHORED))
.count());
@ -212,7 +225,8 @@ public class CreatePersonAS {
r -> r.getSource().equalsIgnoreCase("30|orcid_______::" + DHPUtils.md5("0000-0001-6291-9619"))
&& r.getRelClass().equalsIgnoreCase(ModelConstants.PERSON_PERSON_HASCOAUTHORED))
.count());
Assertions.assertEquals(33, relations.count());
Assertions.assertEquals(38, relations.count());
relations.foreach(r -> System.out.println(new ObjectMapper().writeValueAsString(r)));
}

View File

@ -1,6 +1,6 @@
{"DOI":"10.1109\/ETFA46521.2020.9212101","Authors":[{"Name":{"Full":"Chris Paul Iatrou","First":"Chris Paul","Last":"Iatrou"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Chair of Process Control Systems, Technische Universit\u00e4t Dresden, Dresden, Germany"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/042aqky30","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"37085798458"}},{"Name":{"Full":"Lukas Ketzel","First":"Lukas","Last":"Ketzel"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Chair of Process Control Systems, Technische Universit\u00e4t Dresden, Dresden, Germany"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/042aqky30","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"37088521467"}},{"Name":{"Full":"Markus Graube","First":"Markus","Last":"Graube"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["TraceTronic GmbH, Dresden, Germany"],"Matchings":[],"PIDs":{"Schema":"IEEE","Value":"37889063900"}},{"Name":{"Full":"Martin H\u00e4fner","First":"Martin","Last":"H\u00e4fner"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["B\u00fcrkert Fluidic Control Systems, Ingelfingen, Germay"],"Matchings":[],"PIDs":{"Schema":"IEEE","Value":"37824535500"}},{"Name":{"Full":"Leon Urbas","First":"Leon","Last":"Urbas"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Chair of Process Control Systems, Technische Universit\u00e4t Dresden, Dresden, Germany"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/042aqky30","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"37265255100"}}],"Organizations":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/042aqky30","Confidence":1,"Status":"active"}]}
{"DOI":"10.1109\/4.735554","Authors":[{"Name":{"Full":"T. Cummins","First":"T.","Last":"Cummins"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Analog Devices, Inc., Limerick, Ireland"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/05582kr93","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"37372617500"}},{"Name":{"Full":"E. Byrne","First":"E.","Last":"Byrne"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Analog Devices, Inc., Limerick, Ireland"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/05582kr93","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"38226811000"}},{"Name":{"Full":"D. Brannick","First":"D.","Last":"Brannick"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Analog Devices, Inc., Limerick, Ireland"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/05582kr93","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"37375404700"}},{"Name":{"Full":"D.A. Dempsey","First":"D.A.","Last":"Dempsey"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Analog Devices, Inc., Limerick, Ireland"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/05582kr93","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"37341692700"}}],"Organizations":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/05582kr93","Confidence":1,"Status":"active"}]}
{"DOI":"10.1109\/AICS60730.2023.10470926","Authors":[{"Name":{"Full":"Gyanendar Manohar","First":"Gyanendar","Last":"Manohar"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Department of Computer Science, Munster Technological University, Cork, Ireland"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/013xpqh61","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"961565437242602"}},{"Name":{"Full":"Ruairi O'Reilly","First":"Ruairi","Last":"O'Reilly"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Department of Computer Science, Munster Technological University, Cork, Ireland"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/013xpqh61","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"37547617100"}}],"Organizations":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/013xpqh61","Confidence":1,"Status":"active"}]}
{"DOI":"10.1109\/4.735554","Authors":[{"Name":{"Full":"T. Cummins","First":"T.","Last":"Cummins"},"Corresponding":true,"Contributor_roles":[{"Schema": "credit", "Value": "writer"},{"Schema": "credit","Value": "developer"}],"Raw_affiliations":["Analog Devices, Inc., Limerick, Ireland"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/05582kr93","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"ORCID","Value":"37372617500"}},{"Name":{"Full":"E. Byrne","First":"E.","Last":"Byrne"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Analog Devices, Inc., Limerick, Ireland"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/05582kr93","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"ORCID","Value":"38226811000"}},{"Name":{"Full":"D. Brannick","First":"D.","Last":"Brannick"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Analog Devices, Inc., Limerick, Ireland"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/05582kr93","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"37375404700"}},{"Name":{"Full":"D.A. Dempsey","First":"D.A.","Last":"Dempsey"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Analog Devices, Inc., Limerick, Ireland"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/05582kr93","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"37341692700"}}],"Organizations":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/05582kr93","Confidence":1,"Status":"active"}]}
{"DOI":"10.1109\/AICS60730.2023.10470926","Authors":[{"Name":{"Full":"Gyanendar Manohar","First":"Gyanendar","Last":"Manohar"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Department of Computer Science, Munster Technological University, Cork, Ireland"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/013xpqh61","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"ORCID","Value":"961565437242602"}},{"Name":{"Full":"Ruairi O'Reilly","First":"Ruairi","Last":"O'Reilly"},"Corresponding":true,"Contributor_roles":[{"Schema": "credit", "Value": "writer"},{"Schema": "credit","Value": "developer"}],"Raw_affiliations":["Department of Computer Science, Munster Technological University, Cork, Ireland"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/013xpqh61","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"37547617100"}}],"Organizations":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/013xpqh61","Confidence":1,"Status":"active"}]}
{"DOI":"10.1109\/20.706738","Authors":[{"Name":{"Full":"J.H. Steele","First":"J.H.","Last":"Steele"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Department of Electrical and Computer Engineering, Carnegie Mellon University, Pittsburgh, PA, USA"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/05x2bcf33","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"37361970600"}},{"Name":{"Full":"W.C. Messner","First":"W.C.","Last":"Messner"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Department of Electrical and Computer Engineering, Carnegie Mellon University, Pittsburgh, PA, USA"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/05x2bcf33","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"37315619100"}},{"Name":{"Full":"J.A. Bain","First":"J.A.","Last":"Bain"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Department of Electrical and Computer Engineering, Carnegie Mellon University, Pittsburgh, PA, USA"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/05x2bcf33","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"37266530200"}},{"Name":{"Full":"T.A. Schwarz","First":"T.A.","Last":"Schwarz"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Peregrine Recording Technology, Woodbury, MN, USA"],"Matchings":[],"PIDs":{"Schema":"IEEE","Value":"37334354900"}},{"Name":{"Full":"W.J. O'Kane","First":"W.J.","Last":"O'Kane"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":[],"Matchings":[],"PIDs":{"Schema":"IEEE","Value":"37369941900"}},{"Name":{"Full":"M.P. Connolly","First":"M.P.","Last":"Connolly"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Springtown Industrial Estate, Seagate Technology Ireland, Londonderry, UK"],"Matchings":[],"PIDs":{"Schema":"IEEE","Value":"37373077300"}}],"Organizations":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/05x2bcf33","Confidence":1,"Status":"active"}]}
{"DOI":"10.1109\/GLOBECOM48099.2022.10001219","Authors":[{"Name":{"Full":"Anestis Dalgkitsis","First":"Anestis","Last":"Dalgkitsis"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Iquadrat Informatica S.L., Barcelona, Spain"],"Matchings":[],"PIDs":{"Schema":"IEEE","Value":"37088665104"}},{"Name":{"Full":"Ashima Chawla","First":"Ashima","Last":"Chawla"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Network Management Research Lab, LM Ericsson, Athlone, Ireland"],"Matchings":[],"PIDs":{"Schema":"IEEE","Value":"37088569286"}},{"Name":{"Full":"Anne-Marie Bosneag","First":"Anne-Marie","Last":"Bosneag"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Network Management Research Lab, LM Ericsson, Athlone, Ireland"],"Matchings":[],"PIDs":{"Schema":"IEEE","Value":"37283409900"}},{"Name":{"Full":"Christos Verikoukis","First":"Christos","Last":"Verikoukis"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Computer Engineering & Informatics Dept., University of Patras, Greece"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/017wvtq80","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"37272215400"}}],"Organizations":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/017wvtq80","Confidence":1,"Status":"active"}]}
{"DOI":"10.1109\/59.207350","Authors":[{"Name":{"Full":"A. Scully","First":"A.","Last":"Scully"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Electricity Supply Board, Dublin, Ireland"],"Matchings":[],"PIDs":{"Schema":"IEEE","Value":"37389412900"}},{"Name":{"Full":"A. Harpur","First":"A.","Last":"Harpur"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Electricity Supply Board, Dublin, Ireland"],"Matchings":[],"PIDs":{"Schema":"IEEE","Value":"37389412100"}},{"Name":{"Full":"K.D. Le","First":"K.D.","Last":"Le"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Advanced Systems Technology, ABB Power Systems, Inc., Pittsburgh, PA, USA"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/036hc2148","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"37302366200"}},{"Name":{"Full":"J.T. Day","First":"J.T.","Last":"Day"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Advanced Systems Technology, ABB Power Systems, Inc., Pittsburgh, PA, USA"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/036hc2148","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"37298894800"}},{"Name":{"Full":"M.J. Malone","First":"M.J.","Last":"Malone"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Advanced Systems Technology, ABB Power Systems, Inc., Pittsburgh, PA, USA"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/036hc2148","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"37298895200"}},{"Name":{"Full":"T.E. Mousseau","First":"T.E.","Last":"Mousseau"},"Corresponding":null,"Contributor_roles":null,"Raw_affiliations":["Advanced Systems Technology, ABB Power Systems, Inc., Pittsburgh, PA, USA"],"Matchings":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/036hc2148","Confidence":1,"Status":"active"}],"PIDs":{"Schema":"IEEE","Value":"37389412300"}}],"Organizations":[{"Provenance":"AffRo","PID":"ROR","Value":"https:\/\/ror.org\/036hc2148","Confidence":1,"Status":"active"}]}