Merge pull request 'Dedup ID creation policy' (#48) from deduptesting into stable_ids

This commit is contained in:
Claudio Atzori 2020-10-30 15:15:32 +01:00
commit 4ca75d6951
42 changed files with 2251 additions and 112 deletions

View File

@ -90,6 +90,10 @@
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-core</artifactId>
</dependency>
<dependency>
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>
</dependency>
</dependencies>

View File

@ -29,6 +29,7 @@ import eu.dnetlib.pace.config.DedupConfig;
abstract class AbstractSparkAction implements Serializable {
protected static final int NUM_PARTITIONS = 1000;
protected static final int NUM_CONNECTIONS = 20;
protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper()
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);

View File

@ -18,7 +18,7 @@ import eu.dnetlib.dhp.schema.oaf.Field;
public class DatePicker {
private static final String DATE_PATTERN = "\\d{4}-\\d{2}-\\d{2}";
public static final String DATE_PATTERN = "\\d{4}-\\d{2}-\\d{2}";
private static final String DATE_DEFAULT_SUFFIX = "01-01";
private static final int YEAR_LB = 1300;
private static final int YEAR_UB = Year.now().getValue() + 5;
@ -114,7 +114,7 @@ public class DatePicker {
}
}
private static boolean inRange(final String date) {
public static boolean inRange(final String date) {
final int year = Integer.parseInt(substringBefore(date, "-"));
return year >= YEAR_LB && year <= YEAR_UB;
}

View File

@ -17,6 +17,7 @@ import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists;
import eu.dnetlib.dhp.oa.dedup.model.Identifier;
import eu.dnetlib.dhp.oa.merge.AuthorMerger;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
@ -81,11 +82,16 @@ public class DedupRecordFactory {
final Collection<String> dates = Lists.newArrayList();
final List<List<Author>> authors = Lists.newArrayList();
final List<Identifier> bestPids = Lists.newArrayList(); // best pids list
entities
.forEachRemaining(
t -> {
T duplicate = t._2();
// prepare the list of pids to use for the id generation
bestPids.addAll(IdGenerator.bestPidToIdentifier(duplicate));
entity.mergeFrom(duplicate);
if (ModelSupport.isSubClass(duplicate, Result.class)) {
Result r1 = (Result) duplicate;
@ -94,6 +100,7 @@ public class DedupRecordFactory {
if (r1.getDateofacceptance() != null)
dates.add(r1.getDateofacceptance().getValue());
}
});
// set authors and date
@ -102,10 +109,12 @@ public class DedupRecordFactory {
((Result) entity).setAuthor(AuthorMerger.merge(authors));
}
entity.setId(id);
entity.setId(IdGenerator.generate(bestPids, id));
entity.setLastupdatetimestamp(ts);
entity.setDataInfo(dataInfo);
return entity;
}
}

View File

@ -70,17 +70,6 @@ public class DedupUtility {
return Sets.newHashSet(BlacklistAwareClusteringCombiner.filterAndCombine(doc, conf));
}
public static String md5(final String s) {
try {
final MessageDigest md = MessageDigest.getInstance("MD5");
md.update(s.getBytes(StandardCharsets.UTF_8));
return new String(Hex.encodeHex(md.digest()));
} catch (final Exception e) {
System.err.println("Error creating id");
return null;
}
}
public static String createDedupRecordPath(
final String basePath, final String actionSetId, final String entityType) {
return String.format("%s/%s/%s_deduprecord", basePath, actionSetId, entityType);

View File

@ -0,0 +1,124 @@
package eu.dnetlib.dhp.oa.dedup;
import java.io.Serializable;
import java.text.ParseException;
import java.text.SimpleDateFormat;
import java.util.*;
import org.apache.commons.lang.StringUtils;
import com.google.common.collect.Lists;
import eu.dnetlib.dhp.oa.dedup.model.Identifier;
import eu.dnetlib.dhp.oa.dedup.model.PidType;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.Field;
import eu.dnetlib.dhp.schema.oaf.OafEntity;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
import eu.dnetlib.dhp.utils.DHPUtils;
public class IdGenerator implements Serializable {
public static String CROSSREF_ID = "10|openaire____::081b82f96300b6a6e3d282bad31cb6e2";
public static String DATACITE_ID = "10|openaire____::9e3be59865b2c1c335d32dae2fe7b254";
public static String BASE_DATE = "2000-01-01";
// pick the best pid from the list (consider date and pidtype)
public static String generate(List<Identifier> pids, String defaultID) {
if (pids == null || pids.size() == 0)
return defaultID;
Optional<Identifier> bp = pids
.stream()
.max(Identifier::compareTo);
if (bp.get().isUseOriginal() || bp.get().getPid().getValue() == null) {
return bp.get().getOriginalID().split("\\|")[0] + "|dedup_wf_001::"
+ DHPUtils.md5(bp.get().getOriginalID());
} else {
return bp.get().getOriginalID().split("\\|")[0] + "|"
+ createPrefix(bp.get().getPid().getQualifier().getClassid()) + "::"
+ DHPUtils.md5(bp.get().getPid().getValue());
}
}
public static <T extends OafEntity> ArrayList<Identifier> createBasePid(T entity, SimpleDateFormat sdf) {
Date date;
try {
date = sdf.parse(BASE_DATE);
} catch (ParseException e) {
date = new Date();
}
return Lists
.newArrayList(
new Identifier(new StructuredProperty(), date, PidType.original, entity.getCollectedfrom(),
EntityType.fromClass(entity.getClass()), entity.getId()));
}
// pick the best pid from the entity. Returns a list (length 1) to save time in the call
public static <T extends OafEntity> List<Identifier> bestPidToIdentifier(T entity) {
SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd");
if (entity.getPid() == null || entity.getPid().size() == 0)
return createBasePid(entity, sdf);
Optional<StructuredProperty> bp = entity
.getPid()
.stream()
.filter(pid -> PidType.classidValueOf(pid.getQualifier().getClassid()) != PidType.undefined)
.max(Comparator.comparing(pid -> PidType.classidValueOf(pid.getQualifier().getClassid())));
return bp
.map(
structuredProperty -> Lists
.newArrayList(
new Identifier(structuredProperty, extractDate(entity, sdf),
PidType.classidValueOf(structuredProperty.getQualifier().getClassid()),
entity.getCollectedfrom(), EntityType.fromClass(entity.getClass()), entity.getId())))
.orElseGet(() -> createBasePid(entity, sdf));
}
// create the prefix (length = 12): dedup_+ pidType
public static String createPrefix(String pidType) {
StringBuilder prefix = new StringBuilder("dedup_" + pidType);
while (prefix.length() < 12) {
prefix.append("_");
}
return prefix.toString().substring(0, 12);
}
// extracts the date from the record. If the date is not available or is not wellformed, it returns a base date:
// 00-01-01
public static <T extends OafEntity> Date extractDate(T duplicate, SimpleDateFormat sdf) {
String date = BASE_DATE;
if (ModelSupport.isSubClass(duplicate, Result.class)) {
Result result = (Result) duplicate;
if (isWellformed(result.getDateofacceptance())) {
date = result.getDateofacceptance().getValue();
}
}
try {
return sdf.parse(date);
} catch (ParseException e) {
return new Date();
}
}
public static boolean isWellformed(Field<String> date) {
return date != null && StringUtils.isNotBlank(date.getValue())
&& date.getValue().matches(DatePicker.DATE_PATTERN) && DatePicker.inRange(date.getValue());
}
}

View File

@ -0,0 +1,184 @@
package eu.dnetlib.dhp.oa.dedup;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.sql.*;
import org.dom4j.DocumentException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.oaf.DataInfo;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import eu.dnetlib.pace.config.DedupConfig;
import scala.Tuple2;
public class SparkCollectSimRels extends AbstractSparkAction {
private static final Logger log = LoggerFactory.getLogger(SparkCollectSimRels.class);
Dataset<Row> simGroupsDS;
Dataset<Row> groupsDS;
public SparkCollectSimRels(ArgumentApplicationParser parser, SparkSession spark, Dataset<Row> simGroupsDS,
Dataset<Row> groupsDS) {
super(parser, spark);
this.simGroupsDS = simGroupsDS;
this.groupsDS = groupsDS;
}
public static void main(String[] args) throws Exception {
ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
SparkBlockStats.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/dedup/collectSimRels_parameters.json")));
parser.parseArgument(args);
SparkConf conf = new SparkConf();
final String dbUrl = parser.get("postgresUrl");
final String dbUser = parser.get("postgresUser");
final String dbPassword = parser.get("postgresPassword");
SparkSession spark = getSparkSession(conf);
DataFrameReader readOptions = spark
.read()
.format("jdbc")
.option("url", dbUrl)
.option("user", dbUser)
.option("password", dbPassword);
new SparkCollectSimRels(
parser,
spark,
readOptions.option("dbtable", "similarity_groups").load(),
readOptions.option("dbtable", "groups").load())
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
}
@Override
void run(ISLookUpService isLookUpService) throws DocumentException, ISLookUpException, IOException {
// read oozie parameters
final String isLookUpUrl = parser.get("isLookUpUrl");
final String actionSetId = parser.get("actionSetId");
final String workingPath = parser.get("workingPath");
final int numPartitions = Optional
.ofNullable(parser.get("numPartitions"))
.map(Integer::valueOf)
.orElse(NUM_PARTITIONS);
final String dbUrl = parser.get("postgresUrl");
final String dbUser = parser.get("postgresUser");
log.info("numPartitions: '{}'", numPartitions);
log.info("isLookUpUrl: '{}'", isLookUpUrl);
log.info("actionSetId: '{}'", actionSetId);
log.info("workingPath: '{}'", workingPath);
log.info("postgresUser: {}", dbUser);
log.info("postgresUrl: {}", dbUrl);
log.info("postgresPassword: xxx");
JavaPairRDD<String, List<String>> similarityGroup = simGroupsDS
.toJavaRDD()
.mapToPair(r -> new Tuple2<>(r.getString(0), r.getString(1)))
.groupByKey()
.mapToPair(
i -> new Tuple2<>(i._1(), StreamSupport
.stream(i._2().spliterator(), false)
.collect(Collectors.toList())));
JavaPairRDD<String, String> groupIds = groupsDS
.toJavaRDD()
.mapToPair(r -> new Tuple2<>(r.getString(0), r.getString(1)));
JavaRDD<Tuple2<Tuple2<String, String>, List<String>>> groups = similarityGroup
.leftOuterJoin(groupIds)
.filter(g -> g._2()._2().isPresent())
.map(g -> new Tuple2<>(new Tuple2<>(g._1(), g._2()._2().get()), g._2()._1()));
JavaRDD<Relation> relations = groups.flatMap(g -> {
String firstId = g._2().get(0);
List<Relation> rels = new ArrayList<>();
for (String id : g._2()) {
if (!firstId.equals(id))
rels.add(createSimRel(firstId, id, g._1()._2()));
}
return rels.iterator();
});
Dataset<Relation> resultRelations = spark
.createDataset(
relations.filter(r -> r.getRelType().equals("resultResult")).rdd(),
Encoders.bean(Relation.class))
.repartition(numPartitions);
Dataset<Relation> organizationRelations = spark
.createDataset(
relations.filter(r -> r.getRelType().equals("organizationOrganization")).rdd(),
Encoders.bean(Relation.class))
.repartition(numPartitions);
for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) {
switch (dedupConf.getWf().getSubEntityValue()) {
case "organization":
savePostgresRelation(organizationRelations, workingPath, actionSetId, "organization");
break;
default:
savePostgresRelation(
resultRelations, workingPath, actionSetId, dedupConf.getWf().getSubEntityValue());
break;
}
}
}
private Relation createSimRel(String source, String target, String entity) {
final Relation r = new Relation();
r.setSubRelType("dedupSimilarity");
r.setRelClass("isSimilarTo");
r.setDataInfo(new DataInfo());
switch (entity) {
case "result":
r.setSource("50|" + source);
r.setTarget("50|" + target);
r.setRelType("resultResult");
break;
case "organization":
r.setSource("20|" + source);
r.setTarget("20|" + target);
r.setRelType("organizationOrganization");
break;
default:
throw new IllegalArgumentException("unmanaged entity type: " + entity);
}
return r;
}
private void savePostgresRelation(Dataset<Relation> newRelations, String workingPath, String actionSetId,
String entityType) {
newRelations
.write()
.mode(SaveMode.Append)
.parquet(DedupUtility.createSimRelPath(workingPath, actionSetId, entityType));
}
}

View File

@ -106,10 +106,8 @@ public class SparkCreateMergeRels extends AbstractSparkAction {
final RDD<Edge<String>> edgeRdd = spark
.read()
.textFile(DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity))
.map(
(MapFunction<String, Relation>) r -> OBJECT_MAPPER.readValue(r, Relation.class),
Encoders.bean(Relation.class))
.load(DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity))
.as(Encoders.bean(Relation.class))
.javaRDD()
.map(it -> new Edge<>(hash(it.getSource()), hash(it.getTarget()), it.getRelClass()))
.rdd();

View File

@ -100,12 +100,17 @@ public class SparkCreateSimRels extends AbstractSparkAction {
.repartition(numPartitions);
// create relations by comparing only elements in the same group
Deduper
.computeRelations(sc, blocks, dedupConf)
.map(t -> createSimRel(t._1(), t._2(), entity))
.repartition(numPartitions)
.map(r -> OBJECT_MAPPER.writeValueAsString(r))
.saveAsTextFile(outputPath);
spark
.createDataset(
Deduper
.computeRelations(sc, blocks, dedupConf)
.map(t -> createSimRel(t._1(), t._2(), entity))
.repartition(numPartitions)
.rdd(),
Encoders.bean(Relation.class))
.write()
.mode(SaveMode.Append)
.parquet(outputPath);
}
}

View File

@ -0,0 +1,170 @@
package eu.dnetlib.dhp.oa.dedup;
import java.io.IOException;
import java.util.Optional;
import java.util.Properties;
import org.apache.commons.io.IOUtils;
import org.apache.http.client.methods.CloseableHttpResponse;
import org.apache.http.client.methods.HttpGet;
import org.apache.http.impl.client.CloseableHttpClient;
import org.apache.http.impl.client.HttpClients;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.oa.dedup.model.OrgSimRel;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.Organization;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import scala.Tuple2;
public class SparkPrepareNewOrgs extends AbstractSparkAction {
private static final Logger log = LoggerFactory.getLogger(SparkCreateDedupRecord.class);
public SparkPrepareNewOrgs(ArgumentApplicationParser parser, SparkSession spark) {
super(parser, spark);
}
public static void main(String[] args) throws Exception {
ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
SparkPrepareNewOrgs.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/dedup/prepareNewOrgs_parameters.json")));
parser.parseArgument(args);
SparkConf conf = new SparkConf();
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
new SparkPrepareNewOrgs(parser, getSparkSession(conf))
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
}
@Override
public void run(ISLookUpService isLookUpService) throws IOException {
final String graphBasePath = parser.get("graphBasePath");
final String isLookUpUrl = parser.get("isLookUpUrl");
final String actionSetId = parser.get("actionSetId");
final String workingPath = parser.get("workingPath");
final int numConnections = Optional
.ofNullable(parser.get("numConnections"))
.map(Integer::valueOf)
.orElse(NUM_CONNECTIONS);
final String apiUrl = Optional
.ofNullable(parser.get("apiUrl"))
.orElse("");
final String dbUrl = parser.get("dbUrl");
final String dbTable = parser.get("dbTable");
final String dbUser = parser.get("dbUser");
final String dbPwd = parser.get("dbPwd");
log.info("graphBasePath: '{}'", graphBasePath);
log.info("isLookUpUrl: '{}'", isLookUpUrl);
log.info("actionSetId: '{}'", actionSetId);
log.info("workingPath: '{}'", workingPath);
log.info("numPartitions: '{}'", numConnections);
log.info("apiUrl: '{}'", apiUrl);
log.info("dbUrl: '{}'", dbUrl);
log.info("dbUser: '{}'", dbUser);
log.info("table: '{}'", dbTable);
log.info("dbPwd: '{}'", "xxx");
final String mergeRelPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, "organization");
final String entityPath = DedupUtility.createEntityPath(graphBasePath, "organization");
Dataset<OrgSimRel> newOrgs = createNewOrgs(spark, mergeRelPath, entityPath);
final Properties connectionProperties = new Properties();
connectionProperties.put("user", dbUser);
connectionProperties.put("password", dbPwd);
newOrgs
.repartition(numConnections)
.write()
.mode(SaveMode.Append)
.jdbc(dbUrl, dbTable, connectionProperties);
if (!apiUrl.isEmpty())
updateSimRels(apiUrl);
}
public static Dataset<OrgSimRel> createNewOrgs(
final SparkSession spark,
final String mergeRelsPath,
final String entitiesPath) {
// <id, json_entity>
Dataset<Tuple2<String, Organization>> entities = spark
.read()
.textFile(entitiesPath)
.map(
(MapFunction<String, Tuple2<String, Organization>>) it -> {
Organization entity = OBJECT_MAPPER.readValue(it, Organization.class);
return new Tuple2<>(entity.getId(), entity);
},
Encoders.tuple(Encoders.STRING(), Encoders.kryo(Organization.class)));
Dataset<Tuple2<String, String>> mergerels = spark
.createDataset(
spark
.read()
.load(mergeRelsPath)
.as(Encoders.bean(Relation.class))
.where("relClass == 'isMergedIn'")
.toJavaRDD()
.mapToPair(r -> new Tuple2<>(r.getSource(), r.getTarget()))
.rdd(),
Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
return entities
.joinWith(mergerels, entities.col("_1").equalTo(mergerels.col("_1")), "left")
.filter((FilterFunction<Tuple2<Tuple2<String, Organization>, Tuple2<String, String>>>) t -> t._2() == null)
.filter(
(FilterFunction<Tuple2<Tuple2<String, Organization>, Tuple2<String, String>>>) t -> !t
._1()
._1()
.contains("openorgs"))
.map(
(MapFunction<Tuple2<Tuple2<String, Organization>, Tuple2<String, String>>, OrgSimRel>) r -> new OrgSimRel(
"",
r._1()._2().getOriginalId().get(0),
r._1()._2().getLegalname() != null ? r._1()._2().getLegalname().getValue() : "",
r._1()._2().getLegalshortname() != null ? r._1()._2().getLegalshortname().getValue() : "",
r._1()._2().getCountry() != null ? r._1()._2().getCountry().getClassid() : "",
r._1()._2().getWebsiteurl() != null ? r._1()._2().getWebsiteurl().getValue() : "",
r._1()._2().getCollectedfrom().get(0).getValue(), ""),
Encoders.bean(OrgSimRel.class));
}
private static String updateSimRels(final String apiUrl) throws IOException {
log.info("Updating simrels on the portal");
final HttpGet req = new HttpGet(apiUrl);
try (final CloseableHttpClient client = HttpClients.createDefault()) {
try (final CloseableHttpResponse response = client.execute(req)) {
return IOUtils.toString(response.getEntity().getContent());
}
}
}
}

View File

@ -0,0 +1,263 @@
package eu.dnetlib.dhp.oa.dedup;
import com.google.common.collect.Lists;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.oa.dedup.model.OrgSimRel;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.Organization;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.dhp.utils.ISLookupClientFactory;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.Tuple2;
import scala.Tuple3;
import java.io.IOException;
import java.util.*;
public class SparkPrepareOrgRels extends AbstractSparkAction {
private static final Logger log = LoggerFactory.getLogger(SparkCreateDedupRecord.class);
public SparkPrepareOrgRels(ArgumentApplicationParser parser, SparkSession spark) {
super(parser, spark);
}
public static void main(String[] args) throws Exception {
ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
SparkCreateSimRels.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/dedup/prepareOrgRels_parameters.json")));
parser.parseArgument(args);
SparkConf conf = new SparkConf();
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
new SparkPrepareOrgRels(parser, getSparkSession(conf))
.run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl")));
}
@Override
public void run(ISLookUpService isLookUpService) throws IOException {
final String graphBasePath = parser.get("graphBasePath");
final String isLookUpUrl = parser.get("isLookUpUrl");
final String actionSetId = parser.get("actionSetId");
final String workingPath = parser.get("workingPath");
final int numConnections = Optional
.ofNullable(parser.get("numConnections"))
.map(Integer::valueOf)
.orElse(NUM_CONNECTIONS);
final String dbUrl = parser.get("dbUrl");
final String dbTable = parser.get("dbTable");
final String dbUser = parser.get("dbUser");
final String dbPwd = parser.get("dbPwd");
log.info("graphBasePath: '{}'", graphBasePath);
log.info("isLookUpUrl: '{}'", isLookUpUrl);
log.info("actionSetId: '{}'", actionSetId);
log.info("workingPath: '{}'", workingPath);
log.info("numPartitions: '{}'", numConnections);
log.info("dbUrl: '{}'", dbUrl);
log.info("dbUser: '{}'", dbUser);
log.info("table: '{}'", dbTable);
log.info("dbPwd: '{}'", "xxx");
final String mergeRelPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, "organization");
final String entityPath = DedupUtility.createEntityPath(graphBasePath, "organization");
Dataset<OrgSimRel> relations = createRelations(spark, mergeRelPath, entityPath);
final Properties connectionProperties = new Properties();
connectionProperties.put("user", dbUser);
connectionProperties.put("password", dbPwd);
relations
.repartition(numConnections)
.write()
.mode(SaveMode.Overwrite)
.jdbc(dbUrl, dbTable, connectionProperties);
}
public static Dataset<OrgSimRel> createRelations(
final SparkSession spark,
final String mergeRelsPath,
final String entitiesPath) {
Dataset<Tuple2<String, Organization>> entities = spark
.read()
.textFile(entitiesPath)
.map(
(MapFunction<String, Tuple2<String, Organization>>) it -> {
Organization entity = OBJECT_MAPPER.readValue(it, Organization.class);
return new Tuple2<>(entity.getId(), entity);
},
Encoders.tuple(Encoders.STRING(), Encoders.kryo(Organization.class)));
Dataset<Tuple3<String, String, String>> relations = spark
.createDataset(
spark
.read()
.load(mergeRelsPath)
.as(Encoders.bean(Relation.class))
.where("relClass == 'merges'")
.toJavaRDD()
.mapToPair(r -> new Tuple2<>(r.getSource(), r.getTarget()))
.filter(t -> !t._2().contains("openorgsmesh"))
.groupByKey()
.map(g -> Lists.newArrayList(g._2()))
.filter(l -> l.size() > 1)
.flatMap(l -> {
String groupId = "group::" + UUID.randomUUID();
List<String> ids = sortIds(l);
List<Tuple3<String, String, String>> rels = new ArrayList<>();
for (String source : ids) {
if (source.contains("openorgs____") || ids.indexOf(source) == 0)
for (String target : ids) {
rels.add(new Tuple3<>(source, target, groupId));
}
}
return rels.iterator();
})
.rdd(),
Encoders.tuple(Encoders.STRING(), Encoders.STRING(), Encoders.STRING()));
Dataset<Tuple2<String, OrgSimRel>> relations2 = relations // <openorgs, corda>
.joinWith(entities, relations.col("_2").equalTo(entities.col("_1")), "inner")
.map(
(MapFunction<Tuple2<Tuple3<String, String, String>, Tuple2<String, Organization>>, OrgSimRel>) r -> new OrgSimRel(
r._1()._1(),
r._2()._2().getOriginalId().get(0),
r._2()._2().getLegalname() != null ? r._2()._2().getLegalname().getValue() : "",
r._2()._2().getLegalshortname() != null ? r._2()._2().getLegalshortname().getValue() : "",
r._2()._2().getCountry() != null ? r._2()._2().getCountry().getClassid() : "",
r._2()._2().getWebsiteurl() != null ? r._2()._2().getWebsiteurl().getValue() : "",
r._2()._2().getCollectedfrom().get(0).getValue(),
r._1()._3()),
Encoders.bean(OrgSimRel.class))
.map(
(MapFunction<OrgSimRel, Tuple2<String, OrgSimRel>>) o -> new Tuple2<>(o.getLocal_id(), o),
Encoders.tuple(Encoders.STRING(), Encoders.bean(OrgSimRel.class)));
return relations2
.joinWith(entities, relations2.col("_1").equalTo(entities.col("_1")), "inner")
.map(
(MapFunction<Tuple2<Tuple2<String, OrgSimRel>, Tuple2<String, Organization>>, OrgSimRel>) r -> {
OrgSimRel orgSimRel = r._1()._2();
orgSimRel.setLocal_id(r._2()._2().getOriginalId().get(0));
return orgSimRel;
},
Encoders.bean(OrgSimRel.class));
}
// select best ids from the list. Priority: 1) openorgs, 2)corda, 3)alphabetic
public static List<String> sortIds(List<String> ids) {
ids.sort((o1, o2) -> {
if (o1.contains("openorgs____") && o2.contains("openorgs____"))
return o1.compareTo(o2);
if (o1.contains("corda") && o2.contains("corda"))
return o1.compareTo(o2);
if (o1.contains("openorgs____"))
return -1;
if (o2.contains("openorgs____"))
return 1;
if (o1.contains("corda"))
return -1;
if (o2.contains("corda"))
return 1;
return o1.compareTo(o2);
});
return ids;
}
public static Dataset<OrgSimRel> createRelationsFromScratch(
final SparkSession spark,
final String mergeRelsPath,
final String entitiesPath) {
// <id, json_entity>
Dataset<Tuple2<String, Organization>> entities = spark
.read()
.textFile(entitiesPath)
.map(
(MapFunction<String, Tuple2<String, Organization>>) it -> {
Organization entity = OBJECT_MAPPER.readValue(it, Organization.class);
return new Tuple2<>(entity.getId(), entity);
},
Encoders.tuple(Encoders.STRING(), Encoders.kryo(Organization.class)));
Dataset<Tuple2<String, String>> relations = spark
.createDataset(
spark
.read()
.load(mergeRelsPath)
.as(Encoders.bean(Relation.class))
.where("relClass == 'merges'")
.toJavaRDD()
.mapToPair(r -> new Tuple2<>(r.getSource(), r.getTarget()))
.groupByKey()
.flatMap(g -> {
List<Tuple2<String, String>> rels = new ArrayList<>();
for (String id1 : g._2()) {
for (String id2 : g._2()) {
if (!id1.equals(id2))
if (id1.contains("openorgs____") && !id2.contains("openorgsmesh"))
rels.add(new Tuple2<>(id1, id2));
}
}
return rels.iterator();
})
.rdd(),
Encoders.tuple(Encoders.STRING(), Encoders.STRING()));
Dataset<Tuple2<String, OrgSimRel>> relations2 = relations // <openorgs, corda>
.joinWith(entities, relations.col("_2").equalTo(entities.col("_1")), "inner")
.map(
(MapFunction<Tuple2<Tuple2<String, String>, Tuple2<String, Organization>>, OrgSimRel>) r -> new OrgSimRel(
r._1()._1(),
r._2()._2().getOriginalId().get(0),
r._2()._2().getLegalname() != null ? r._2()._2().getLegalname().getValue() : "",
r._2()._2().getLegalshortname() != null ? r._2()._2().getLegalshortname().getValue() : "",
r._2()._2().getCountry() != null ? r._2()._2().getCountry().getClassid() : "",
r._2()._2().getWebsiteurl() != null ? r._2()._2().getWebsiteurl().getValue() : "",
r._2()._2().getCollectedfrom().get(0).getValue(),
"group::" + r._1()._1()),
Encoders.bean(OrgSimRel.class))
.map(
(MapFunction<OrgSimRel, Tuple2<String, OrgSimRel>>) o -> new Tuple2<>(o.getLocal_id(), o),
Encoders.tuple(Encoders.STRING(), Encoders.bean(OrgSimRel.class)));
return relations2
.joinWith(entities, relations2.col("_1").equalTo(entities.col("_1")), "inner")
.map(
(MapFunction<Tuple2<Tuple2<String, OrgSimRel>, Tuple2<String, Organization>>, OrgSimRel>) r -> {
OrgSimRel orgSimRel = r._1()._2();
orgSimRel.setLocal_id(r._2()._2().getOriginalId().get(0));
return orgSimRel;
},
Encoders.bean(OrgSimRel.class));
}
}

View File

@ -28,8 +28,7 @@ public class SparkPropagateRelation extends AbstractSparkAction {
SOURCE, TARGET
}
public SparkPropagateRelation(ArgumentApplicationParser parser, SparkSession spark)
throws Exception {
public SparkPropagateRelation(ArgumentApplicationParser parser, SparkSession spark) throws Exception {
super(parser, spark);
}

View File

@ -12,6 +12,7 @@ import org.codehaus.jackson.annotate.JsonIgnore;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.oa.dedup.DedupUtility;
import eu.dnetlib.dhp.utils.DHPUtils;
import eu.dnetlib.pace.util.PaceException;
public class ConnectedComponent implements Serializable {
@ -36,7 +37,7 @@ public class ConnectedComponent implements Serializable {
if (docIds.size() > 1) {
final String s = getMin();
String prefix = s.split("\\|")[0];
ccId = prefix + "|dedup_wf_001::" + DedupUtility.md5(s);
ccId = prefix + "|dedup_wf_001::" + DHPUtils.md5(s);
return ccId;
} else {
return docIds.iterator().next();

View File

@ -0,0 +1,146 @@
package eu.dnetlib.dhp.oa.dedup.model;
import java.io.Serializable;
import java.util.Date;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
import com.google.common.collect.Sets;
import eu.dnetlib.dhp.oa.dedup.IdGenerator;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.oaf.KeyValue;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
public class Identifier implements Serializable, Comparable<Identifier> {
StructuredProperty pid;
Date date;
PidType type;
List<KeyValue> collectedFrom;
EntityType entityType;
String originalID;
boolean useOriginal = false; // to know if the top identifier won because of the alphabetical order of the original
// ID
public Identifier(StructuredProperty pid, Date date, PidType type, List<KeyValue> collectedFrom,
EntityType entityType, String originalID) {
this.pid = pid;
this.date = date;
this.type = type;
this.collectedFrom = collectedFrom;
this.entityType = entityType;
this.originalID = originalID;
}
public StructuredProperty getPid() {
return pid;
}
public void setPid(StructuredProperty pid) {
this.pid = pid;
}
public Date getDate() {
return date;
}
public void setDate(Date date) {
this.date = date;
}
public PidType getType() {
return type;
}
public void setType(PidType type) {
this.type = type;
}
public List<KeyValue> getCollectedFrom() {
return collectedFrom;
}
public void setCollectedFrom(List<KeyValue> collectedFrom) {
this.collectedFrom = collectedFrom;
}
public EntityType getEntityType() {
return entityType;
}
public void setEntityType(EntityType entityType) {
this.entityType = entityType;
}
public String getOriginalID() {
return originalID;
}
public void setOriginalID(String originalID) {
this.originalID = originalID;
}
public boolean isUseOriginal() {
return useOriginal;
}
public void setUseOriginal(boolean useOriginal) {
this.useOriginal = useOriginal;
}
@Override
public int compareTo(Identifier i) {
// priority in comparisons: 1) pidtype, 2) collectedfrom (depending on the entity type) , 3) date 4)
// alphabetical order of the originalID
Set<String> lKeys = Sets.newHashSet();
if (this.collectedFrom != null)
lKeys = this.collectedFrom.stream().map(KeyValue::getKey).collect(Collectors.toSet());
Set<String> rKeys = Sets.newHashSet();
if (i.getCollectedFrom() != null)
rKeys = i.getCollectedFrom().stream().map(KeyValue::getKey).collect(Collectors.toSet());
if (this.getType().compareTo(i.getType()) == 0) { // same type
if (entityType == EntityType.publication) {
if (isFromDatasourceID(lKeys, IdGenerator.CROSSREF_ID)
&& !isFromDatasourceID(rKeys, IdGenerator.CROSSREF_ID))
return 1;
if (isFromDatasourceID(rKeys, IdGenerator.CROSSREF_ID)
&& !isFromDatasourceID(lKeys, IdGenerator.CROSSREF_ID))
return -1;
}
if (entityType == EntityType.dataset) {
if (isFromDatasourceID(lKeys, IdGenerator.DATACITE_ID)
&& !isFromDatasourceID(rKeys, IdGenerator.DATACITE_ID))
return 1;
if (isFromDatasourceID(rKeys, IdGenerator.DATACITE_ID)
&& !isFromDatasourceID(lKeys, IdGenerator.DATACITE_ID))
return -1;
}
if (this.getDate().compareTo(i.getDate()) == 0) {// same date
if (this.originalID.compareTo(i.originalID) < 0)
this.useOriginal = true;
else
i.setUseOriginal(true);
// the minus because we need to take the alphabetically lower id
return -this.originalID.compareTo(i.originalID);
} else
// the minus is because we need to take the elder date
return -this.getDate().compareTo(i.getDate());
} else {
return this.getType().compareTo(i.getType());
}
}
public boolean isFromDatasourceID(Set<String> collectedFrom, String dsId) {
return collectedFrom.contains(dsId);
}
}

View File

@ -0,0 +1,108 @@
package eu.dnetlib.dhp.oa.dedup.model;
import java.io.Serializable;
public class OrgSimRel implements Serializable {
String local_id;
String oa_original_id;
String oa_name;
String oa_acronym;
String oa_country;
String oa_url;
String oa_collectedfrom;
String group_id;
public OrgSimRel() {
}
public OrgSimRel(String local_id, String oa_original_id, String oa_name, String oa_acronym, String oa_country,
String oa_url, String oa_collectedfrom, String group_id) {
this.local_id = local_id;
this.oa_original_id = oa_original_id;
this.oa_name = oa_name;
this.oa_acronym = oa_acronym;
this.oa_country = oa_country;
this.oa_url = oa_url;
this.oa_collectedfrom = oa_collectedfrom;
this.group_id = group_id;
}
public String getLocal_id() {
return local_id;
}
public void setLocal_id(String local_id) {
this.local_id = local_id;
}
public String getOa_original_id() {
return oa_original_id;
}
public void setOa_original_id(String oa_original_id) {
this.oa_original_id = oa_original_id;
}
public String getOa_name() {
return oa_name;
}
public void setOa_name(String oa_name) {
this.oa_name = oa_name;
}
public String getOa_acronym() {
return oa_acronym;
}
public void setOa_acronym(String oa_acronym) {
this.oa_acronym = oa_acronym;
}
public String getOa_country() {
return oa_country;
}
public void setOa_country(String oa_country) {
this.oa_country = oa_country;
}
public String getOa_url() {
return oa_url;
}
public void setOa_url(String oa_url) {
this.oa_url = oa_url;
}
public String getOa_collectedfrom() {
return oa_collectedfrom;
}
public void setOa_collectedfrom(String oa_collectedfrom) {
this.oa_collectedfrom = oa_collectedfrom;
}
public String getGroup_id() {
return group_id;
}
public void setGroup_id(String group_id) {
this.group_id = group_id;
}
@Override
public String toString() {
return "OrgSimRel{" +
"local_id='" + local_id + '\'' +
", oa_original_id='" + oa_original_id + '\'' +
", oa_name='" + oa_name + '\'' +
", oa_acronym='" + oa_acronym + '\'' +
", oa_country='" + oa_country + '\'' +
", oa_url='" + oa_url + '\'' +
", oa_collectedfrom='" + oa_collectedfrom + '\'' +
'}';
}
}

View File

@ -0,0 +1,17 @@
package eu.dnetlib.dhp.oa.dedup.model;
public enum PidType {
// from the less to the more important
undefined, original, orcid, ror, grid, pdb, arXiv, pmid, pmc, doi;
public static PidType classidValueOf(String s) {
try {
return PidType.valueOf(s);
} catch (Exception e) {
return PidType.undefined;
}
}
}

View File

@ -0,0 +1,44 @@
[
{
"paramName": "la",
"paramLongName": "isLookUpUrl",
"paramDescription": "address for the LookUp",
"paramRequired": true
},
{
"paramName": "asi",
"paramLongName": "actionSetId",
"paramDescription": "action set identifier (name of the orchestrator)",
"paramRequired": true
},
{
"paramName": "w",
"paramLongName": "workingPath",
"paramDescription": "path of the working directory",
"paramRequired": true
},
{
"paramName": "np",
"paramLongName": "numPartitions",
"paramDescription": "number of partitions for the similarity relations intermediate phases",
"paramRequired": false
},
{
"paramName": "purl",
"paramLongName": "postgresUrl",
"paramDescription": "the url of the postgres server",
"paramRequired": true
},
{
"paramName": "pusr",
"paramLongName": "postgresUser",
"paramDescription": "the owner of the postgres database",
"paramRequired": true
},
{
"paramName": "ppwd",
"paramLongName": "postgresPassword",
"paramDescription": "the password for the postgres user",
"paramRequired": true
}
]

View File

@ -0,0 +1,18 @@
<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>
</configuration>

View File

@ -0,0 +1,208 @@
<workflow-app name="Organization Dedup" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>graphBasePath</name>
<description>the raw graph base path</description>
</property>
<property>
<name>isLookUpUrl</name>
<description>the address of the lookUp service</description>
</property>
<property>
<name>actionSetId</name>
<description>id of the actionSet</description>
</property>
<property>
<name>workingPath</name>
<description>path for the working directory</description>
</property>
<property>
<name>dedupGraphPath</name>
<description>path for the output graph</description>
</property>
<property>
<name>cutConnectedComponent</name>
<description>max number of elements in a connected component</description>
</property>
<property>
<name>dbUrl</name>
<description>the url of the database</description>
</property>
<property>
<name>dbUser</name>
<description>the user of the database</description>
</property>
<property>
<name>dbTable</name>
<description>the name of the table in the database</description>
</property>
<property>
<name>dbPwd</name>
<description>the passowrd of the user of the database</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
<property>
<name>oozieActionShareLibForSpark2</name>
<description>oozie action sharelib for spark 2.*</description>
</property>
<property>
<name>spark2ExtraListeners</name>
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
<description>spark 2.* extra listeners classname</description>
</property>
<property>
<name>spark2SqlQueryExecutionListeners</name>
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
<description>spark 2.* sql query execution listeners classname</description>
</property>
<property>
<name>spark2YarnHistoryServerAddress</name>
<description>spark 2.* yarn history server address</description>
</property>
<property>
<name>spark2EventLogDir</name>
<description>spark 2.* event log dir location</description>
</property>
</parameters>
<global>
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<configuration>
<property>
<name>mapreduce.job.queuename</name>
<value>${queueName}</value>
</property>
<property>
<name>oozie.launcher.mapred.job.queue.name</name>
<value>${oozieLauncherQueueName}</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>${oozieActionShareLibForSpark2}</value>
</property>
</configuration>
</global>
<start to="resetWorkingPath"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="resetWorkingPath">
<fs>
<delete path="${workingPath}"/>
</fs>
<ok to="copyRelations"/>
<error to="Kill"/>
</action>
<action name="copyRelations">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<arg>-pb</arg>
<arg>${graphBasePath}/relation</arg>
<arg>${workingPath}/${actionSetId}/organization_simrel</arg>
</distcp>
<ok to="CreateSimRel"/>
<error to="Kill"/>
</action>
<action name="CreateSimRel">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Create Similarity Relations</name>
<class>eu.dnetlib.dhp.oa.dedup.SparkCreateSimRels</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--numPartitions</arg><arg>8000</arg>
</spark>
<ok to="CreateMergeRel"/>
<error to="Kill"/>
</action>
<action name="CreateMergeRel">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Create Merge Relations</name>
<class>eu.dnetlib.dhp.oa.dedup.SparkCreateMergeRels</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
<arg>--cutConnectedComponent</arg><arg>${cutConnectedComponent}</arg>
</spark>
<ok to="PrepareNewOrgs"/>
<error to="Kill"/>
</action>
<action name="PrepareNewOrgs">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Prepare New Organizations</name>
<class>eu.dnetlib.dhp.oa.dedup.SparkPrepareNewOrgs</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
<arg>--dbUrl</arg><arg>${dbUrl}</arg>
<arg>--dbTable</arg><arg>${dbTable}</arg>
<arg>--dbUser</arg><arg>${dbUser}</arg>
<arg>--dbPwd</arg><arg>${dbPwd}</arg>
<arg>--numConnections</arg><arg>20</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,18 @@
<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>
</configuration>

View File

@ -0,0 +1,240 @@
<workflow-app name="Organization Dedup" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>graphBasePath</name>
<description>the raw graph base path</description>
</property>
<property>
<name>isLookUpUrl</name>
<description>the address of the lookUp service</description>
</property>
<property>
<name>actionSetId</name>
<description>id of the actionSet</description>
</property>
<property>
<name>workingPath</name>
<description>path for the working directory</description>
</property>
<property>
<name>dedupGraphPath</name>
<description>path for the output graph</description>
</property>
<property>
<name>cutConnectedComponent</name>
<description>max number of elements in a connected component</description>
</property>
<property>
<name>dbUrl</name>
<description>the url of the database</description>
</property>
<property>
<name>dbUser</name>
<description>the user of the database</description>
</property>
<property>
<name>dbTable</name>
<description>the name of the table in the database</description>
</property>
<property>
<name>dbPwd</name>
<description>the passowrd of the user of the database</description>
</property>
<property>
<name>sparkDriverMemory</name>
<description>memory for driver process</description>
</property>
<property>
<name>sparkExecutorMemory</name>
<description>memory for individual executor</description>
</property>
<property>
<name>sparkExecutorCores</name>
<description>number of cores used by single executor</description>
</property>
<property>
<name>oozieActionShareLibForSpark2</name>
<description>oozie action sharelib for spark 2.*</description>
</property>
<property>
<name>spark2ExtraListeners</name>
<value>com.cloudera.spark.lineage.NavigatorAppListener</value>
<description>spark 2.* extra listeners classname</description>
</property>
<property>
<name>spark2SqlQueryExecutionListeners</name>
<value>com.cloudera.spark.lineage.NavigatorQueryListener</value>
<description>spark 2.* sql query execution listeners classname</description>
</property>
<property>
<name>spark2YarnHistoryServerAddress</name>
<description>spark 2.* yarn history server address</description>
</property>
<property>
<name>spark2EventLogDir</name>
<description>spark 2.* event log dir location</description>
</property>
</parameters>
<global>
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<configuration>
<property>
<name>mapreduce.job.queuename</name>
<value>${queueName}</value>
</property>
<property>
<name>oozie.launcher.mapred.job.queue.name</name>
<value>${oozieLauncherQueueName}</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>${oozieActionShareLibForSpark2}</value>
</property>
</configuration>
</global>
<start to="resetWorkingPath"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="resetWorkingPath">
<fs>
<delete path="${workingPath}"/>
</fs>
<ok to="copyRelations"/>
<error to="Kill"/>
</action>
<action name="copyRelations">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<arg>-pb</arg>
<arg>/tmp/graph_openorgs_and_corda/relation</arg>
<arg>${workingPath}/${actionSetId}/organization_simrel</arg>
</distcp>
<ok to="CreateSimRel"/>
<error to="Kill"/>
</action>
<action name="CreateSimRel">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Create Similarity Relations</name>
<class>eu.dnetlib.dhp.oa.dedup.SparkCreateSimRels</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--numPartitions</arg><arg>8000</arg>
</spark>
<ok to="CreateMergeRel"/>
<error to="Kill"/>
</action>
<action name="CreateMergeRel">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Create Merge Relations</name>
<class>eu.dnetlib.dhp.oa.dedup.SparkCreateMergeRels</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
<arg>--cutConnectedComponent</arg><arg>${cutConnectedComponent}</arg>
</spark>
<ok to="PrepareOrgRels"/>
<error to="Kill"/>
</action>
<action name="PrepareOrgRels">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Prepare Organization Relations</name>
<class>eu.dnetlib.dhp.oa.dedup.SparkPrepareOrgRels</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
<arg>--dbUrl</arg><arg>${dbUrl}</arg>
<arg>--dbTable</arg><arg>${dbTable}</arg>
<arg>--dbUser</arg><arg>${dbUser}</arg>
<arg>--dbPwd</arg><arg>${dbPwd}</arg>
<arg>--numConnections</arg><arg>20</arg>
</spark>
<ok to="PrepareNewOrgs"/>
<error to="Kill"/>
</action>
<action name="PrepareNewOrgs">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Prepare New Organizations</name>
<class>eu.dnetlib.dhp.oa.dedup.SparkPrepareNewOrgs</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
<arg>--apiUrl</arg><arg>${apiUrl}</arg>
<arg>--dbUrl</arg><arg>${dbUrl}</arg>
<arg>--dbTable</arg><arg>${dbTable}</arg>
<arg>--dbUser</arg><arg>${dbUser}</arg>
<arg>--dbPwd</arg><arg>${dbPwd}</arg>
<arg>--numConnections</arg><arg>20</arg>
</spark>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

View File

@ -0,0 +1,62 @@
[
{
"paramName": "i",
"paramLongName": "graphBasePath",
"paramDescription": "the base path of raw graph",
"paramRequired": true
},
{
"paramName": "w",
"paramLongName": "workingPath",
"paramDescription": "the working directory path",
"paramRequired": true
},
{
"paramName": "la",
"paramLongName": "isLookUpUrl",
"paramDescription": "the url of the lookup service",
"paramRequired": true
},
{
"paramName": "asi",
"paramLongName": "actionSetId",
"paramDescription": "the id of the actionset (orchestrator)",
"paramRequired": true
},
{
"paramName": "nc",
"paramLongName": "numConnections",
"paramDescription": "number of connections to the postgres db (for the write operation)",
"paramRequired": false
},
{
"paramName": "au",
"paramLongName": "apiUrl",
"paramDescription": "the url for the APIs of the openorgs service",
"paramRequired": false
},
{
"paramName": "du",
"paramLongName": "dbUrl",
"paramDescription": "the url of the database",
"paramRequired": true
},
{
"paramName": "dusr",
"paramLongName": "dbUser",
"paramDescription": "the user of the database",
"paramRequired": true
},
{
"paramName": "t",
"paramLongName": "dbTable",
"paramDescription": "the name of the table in the database",
"paramRequired": true
},
{
"paramName": "dpwd",
"paramLongName": "dbPwd",
"paramDescription": "the password for the user of the database",
"paramRequired": true
}
]

View File

@ -0,0 +1,56 @@
[
{
"paramName": "i",
"paramLongName": "graphBasePath",
"paramDescription": "the base path of raw graph",
"paramRequired": true
},
{
"paramName": "w",
"paramLongName": "workingPath",
"paramDescription": "the working directory path",
"paramRequired": true
},
{
"paramName": "la",
"paramLongName": "isLookUpUrl",
"paramDescription": "the url of the lookup service",
"paramRequired": true
},
{
"paramName": "asi",
"paramLongName": "actionSetId",
"paramDescription": "the id of the actionset (orchestrator)",
"paramRequired": true
},
{
"paramName": "nc",
"paramLongName": "numConnections",
"paramDescription": "number of connections to the postgres db (for the write operation)",
"paramRequired": false
},
{
"paramName": "du",
"paramLongName": "dbUrl",
"paramDescription": "the url of the database",
"paramRequired": true
},
{
"paramName": "dusr",
"paramLongName": "dbUser",
"paramDescription": "the user of the database",
"paramRequired": true
},
{
"paramName": "t",
"paramLongName": "dbTable",
"paramDescription": "the name of the table in the database",
"paramRequired": true
},
{
"paramName": "dpwd",
"paramLongName": "dbPwd",
"paramDescription": "the password for the user of the database",
"paramRequired": true
}
]

View File

@ -23,10 +23,13 @@ public class EntityMergerTest implements Serializable {
List<Tuple2<String, Publication>> publications;
List<Tuple2<String, Publication>> publications2;
List<Tuple2<String, Publication>> publications3;
List<Tuple2<String, Publication>> publications4;
List<Tuple2<String, Publication>> publications5;
String testEntityBasePath;
DataInfo dataInfo;
String dedupId = "dedup_id";
String dedupId = "00|dedup_id::1";
Publication pub_top;
@BeforeEach
@ -39,6 +42,9 @@ public class EntityMergerTest implements Serializable {
publications = readSample(testEntityBasePath + "/publication_merge.json", Publication.class);
publications2 = readSample(testEntityBasePath + "/publication_merge2.json", Publication.class);
publications3 = readSample(testEntityBasePath + "/publication_merge3.json", Publication.class);
publications4 = readSample(testEntityBasePath + "/publication_merge4.json", Publication.class);
publications5 = readSample(testEntityBasePath + "/publication_merge5.json", Publication.class);
pub_top = getTopPub(publications);
@ -48,6 +54,7 @@ public class EntityMergerTest implements Serializable {
@Test
public void softwareMergerTest() throws InstantiationException, IllegalAccessException {
List<Tuple2<String, Software>> softwares = readSample(
testEntityBasePath + "/software_merge.json", Software.class);
@ -55,6 +62,9 @@ public class EntityMergerTest implements Serializable {
.entityMerger(dedupId, softwares.iterator(), 0, dataInfo, Software.class);
assertEquals(merged.getBestaccessright().getClassid(), "OPEN SOURCE");
assertEquals(merged.getId(), "50|dedup_doi___::0968af610a356656706657e4f234b340");
}
@Test
@ -63,7 +73,8 @@ public class EntityMergerTest implements Serializable {
Publication pub_merged = DedupRecordFactory
.entityMerger(dedupId, publications.iterator(), 0, dataInfo, Publication.class);
assertEquals(dedupId, pub_merged.getId());
// verify id
assertEquals(pub_merged.getId(), "50|dedup_doi___::0968af610a356656706657e4f234b340");
assertEquals(pub_merged.getJournal(), pub_top.getJournal());
assertEquals(pub_merged.getBestaccessright().getClassid(), "OPEN");
@ -118,9 +129,40 @@ public class EntityMergerTest implements Serializable {
Publication pub_merged = DedupRecordFactory
.entityMerger(dedupId, publications2.iterator(), 0, dataInfo, Publication.class);
assertEquals(pub_merged.getAuthor().size(), 27);
// insert assertions here
// verify id
assertEquals("50|dedup_doi___::0ca46ff10b2b4c756191719d85302b14", pub_merged.getId());
assertEquals(pub_merged.getAuthor().size(), 27);
}
@Test
public void publicationMergerTest3() throws InstantiationException, IllegalAccessException {
Publication pub_merged = DedupRecordFactory
.entityMerger(dedupId, publications3.iterator(), 0, dataInfo, Publication.class);
// verify id
assertEquals("50|dedup_doi___::0ca46ff10b2b4c756191719d85302b14", pub_merged.getId());
}
@Test
public void publicationMergerTest4() throws InstantiationException, IllegalStateException, IllegalAccessException {
Publication pub_merged = DedupRecordFactory
.entityMerger(dedupId, publications4.iterator(), 0, dataInfo, Publication.class);
// verify id
assertEquals("50|dedup_wf_001::2d2bbbbcfb285e3fb3590237b79e2fa8", pub_merged.getId());
}
@Test
public void publicationMergerTest5() throws InstantiationException, IllegalStateException, IllegalAccessException {
Publication pub_merged = DedupRecordFactory
.entityMerger(dedupId, publications5.iterator(), 0, dataInfo, Publication.class);
// verify id
assertEquals("50|dedup_wf_001::584b89679c3ccd1015b647ec63cc2699", pub_merged.getId());
}
public DataInfo setDI() {

View File

@ -0,0 +1,159 @@
package eu.dnetlib.dhp.oa.dedup;
import static org.junit.jupiter.api.Assertions.assertEquals;
import java.io.BufferedReader;
import java.io.FileReader;
import java.io.IOException;
import java.nio.file.Paths;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Date;
import java.util.List;
import java.util.stream.Collectors;
import org.codehaus.jackson.map.ObjectMapper;
import org.junit.jupiter.api.*;
import com.google.common.collect.Lists;
import eu.dnetlib.dhp.oa.dedup.model.Identifier;
import eu.dnetlib.dhp.oa.dedup.model.PidType;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.oaf.KeyValue;
import eu.dnetlib.dhp.schema.oaf.Publication;
import eu.dnetlib.dhp.schema.oaf.Qualifier;
import eu.dnetlib.dhp.schema.oaf.StructuredProperty;
import eu.dnetlib.pace.util.MapDocumentUtil;
import scala.Tuple2;
@TestMethodOrder(MethodOrderer.OrderAnnotation.class)
public class IdGeneratorTest {
private static List<Identifier> bestIds;
private static List<Tuple2<String, Publication>> pubs;
private static List<Identifier> bestIds2;
private static List<Identifier> bestIds3;
private static String testEntityBasePath;
private static SimpleDateFormat sdf;
private static Date baseDate;
@BeforeAll
public static void setUp() throws Exception {
sdf = new SimpleDateFormat("yyyy-MM-dd");
baseDate = sdf.parse("2000-01-01");
bestIds = new ArrayList<>();
bestIds2 = Lists
.newArrayList(
new Identifier(pid("pid1", "original", "original"), baseDate, PidType.original,
keyValue("key", "value"), EntityType.publication, "50|originalID1"),
new Identifier(pid("pid2", "original", "original"), baseDate, PidType.original,
keyValue("key", "value"), EntityType.publication, "50|originalID2"),
new Identifier(pid("pid3", "original", "original"), baseDate, PidType.original,
keyValue("key", "value"), EntityType.publication, "50|originalID3"));
bestIds3 = Lists
.newArrayList(
new Identifier(pid("pid1", "original", "original"), baseDate, PidType.original,
keyValue("key", "value"), EntityType.publication, "50|originalID1"),
new Identifier(pid("pid2", "doi", "doi"), baseDate, PidType.doi, keyValue("key", "value"),
EntityType.publication, "50|originalID2"),
new Identifier(pid("pid3", "original", "original"), baseDate, PidType.original,
keyValue("key", "value"), EntityType.publication, "50|originalID3"));
testEntityBasePath = Paths
.get(SparkDedupTest.class.getResource("/eu/dnetlib/dhp/dedup/json").toURI())
.toFile()
.getAbsolutePath();
pubs = readSample(testEntityBasePath + "/publication_idgeneration.json", Publication.class);
}
@Test
@Order(1)
public void bestPidToIdentifierTest() {
List<String> typesForAssertions = Lists
.newArrayList(PidType.pmc.toString(), PidType.doi.toString(), PidType.doi.toString());
for (Tuple2<String, Publication> pub : pubs) {
List<Identifier> ids = IdGenerator.bestPidToIdentifier(pub._2());
assertEquals(typesForAssertions.get(pubs.indexOf(pub)), ids.get(0).getPid().getQualifier().getClassid());
bestIds.addAll(ids);
}
}
@Test
@Order(2)
public void generateIdTest1() {
String id1 = IdGenerator.generate(bestIds, "50|defaultID");
System.out.println("id list 1 = " + bestIds.stream().map(i -> i.getPid().getValue()).collect(Collectors.toList()));
assertEquals("50|dedup_wf_001::9c5cfbf993d38476e0f959a301239719", id1);
}
@Test
public void generateIdTest2() {
String id1 = IdGenerator.generate(bestIds2, "50|defaultID");
String id2 = IdGenerator.generate(bestIds3, "50|defaultID");
System.out.println("id list 2 = " + bestIds2.stream().map(i -> i.getPid().getValue()).collect(Collectors.toList()));
System.out.println("winner 2 = " + id1);
System.out.println("id list 3 = " + bestIds3.stream().map(i -> i.getPid().getValue()).collect(Collectors.toList()));
System.out.println("winner 3 = " + id2);
assertEquals("50|dedup_wf_001::2c56cc1914bffdb30fdff354e0099612", id1);
assertEquals("50|dedup_doi___::128ead3ed8d9ecf262704b6fcf592b8d", id2);
}
public static <T> List<Tuple2<String, T>> readSample(String path, Class<T> clazz) {
List<Tuple2<String, T>> res = new ArrayList<>();
BufferedReader reader;
try {
reader = new BufferedReader(new FileReader(path));
String line = reader.readLine();
while (line != null) {
res
.add(
new Tuple2<>(
MapDocumentUtil.getJPathString("$.id", line),
new ObjectMapper().readValue(line, clazz)));
// read next line
line = reader.readLine();
}
reader.close();
} catch (IOException e) {
e.printStackTrace();
}
return res;
}
public static StructuredProperty pid(String pid, String classid, String classname) {
StructuredProperty sp = new StructuredProperty();
sp.setValue(pid);
Qualifier q = new Qualifier();
q.setSchemeid(classid);
q.setSchemename(classname);
q.setClassname(classname);
q.setClassid(classid);
sp.setQualifier(q);
return sp;
}
public static List<KeyValue> keyValue(String key, String value) {
KeyValue kv = new KeyValue();
kv.setKey(key);
kv.setValue(value);
return Lists.newArrayList(kv);
}
}

View File

@ -1,22 +1,12 @@
package eu.dnetlib.dhp.oa.dedup;
import static java.nio.file.Files.createTempDirectory;
import static org.apache.spark.sql.functions.col;
import static org.apache.spark.sql.functions.count;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.Mockito.lenient;
import java.io.File;
import java.io.IOException;
import java.io.Serializable;
import java.net.URISyntaxException;
import java.nio.file.Paths;
import java.util.HashSet;
import java.util.Set;
import java.util.stream.Collectors;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import eu.dnetlib.pace.util.MapDocumentUtil;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
@ -35,16 +25,19 @@ import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.jupiter.MockitoExtension;
import com.fasterxml.jackson.databind.ObjectMapper;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.schema.oaf.Relation;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import eu.dnetlib.pace.util.MapDocumentUtil;
import scala.Tuple2;
import java.io.File;
import java.io.IOException;
import java.io.Serializable;
import java.net.URISyntaxException;
import java.nio.file.Paths;
import static java.nio.file.Files.createTempDirectory;
import static org.apache.spark.sql.functions.count;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.Mockito.lenient;
@ExtendWith(MockitoExtension.class)
@TestMethodOrder(MethodOrderer.OrderAnnotation.class)
public class SparkDedupTest implements Serializable {
@ -59,6 +52,7 @@ public class SparkDedupTest implements Serializable {
private static String testOutputBasePath;
private static String testDedupGraphBasePath;
private static final String testActionSetId = "test-orchestrator";
private static String testDedupAssertionsBasePath;
@BeforeAll
public static void cleanUp() throws IOException, URISyntaxException {
@ -73,6 +67,10 @@ public class SparkDedupTest implements Serializable {
testDedupGraphBasePath = createTempDirectory(SparkDedupTest.class.getSimpleName() + "-")
.toAbsolutePath()
.toString();
testDedupAssertionsBasePath = Paths
.get(SparkDedupTest.class.getResource("/eu/dnetlib/dhp/dedup/assertions").toURI())
.toFile()
.getAbsolutePath();
FileUtils.deleteDirectory(new File(testOutputBasePath));
FileUtils.deleteDirectory(new File(testDedupGraphBasePath));
@ -87,6 +85,7 @@ public class SparkDedupTest implements Serializable {
.getOrCreate();
jsc = JavaSparkContext.fromSparkContext(spark.sparkContext());
}
@BeforeEach
@ -157,6 +156,7 @@ public class SparkDedupTest implements Serializable {
SparkCreateSimRels.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json")));
parser
.parseArgument(
new String[] {
@ -171,27 +171,27 @@ public class SparkDedupTest implements Serializable {
long orgs_simrel = spark
.read()
.textFile(testOutputBasePath + "/" + testActionSetId + "/organization_simrel")
.load(testOutputBasePath + "/" + testActionSetId + "/organization_simrel")
.count();
long pubs_simrel = spark
.read()
.textFile(testOutputBasePath + "/" + testActionSetId + "/publication_simrel")
.load(testOutputBasePath + "/" + testActionSetId + "/publication_simrel")
.count();
long sw_simrel = spark
.read()
.textFile(testOutputBasePath + "/" + testActionSetId + "/software_simrel")
.load(testOutputBasePath + "/" + testActionSetId + "/software_simrel")
.count();
long ds_simrel = spark
.read()
.textFile(testOutputBasePath + "/" + testActionSetId + "/dataset_simrel")
.load(testOutputBasePath + "/" + testActionSetId + "/dataset_simrel")
.count();
long orp_simrel = spark
.read()
.textFile(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_simrel")
.load(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_simrel")
.count();
assertEquals(3082, orgs_simrel);
@ -203,6 +203,67 @@ public class SparkDedupTest implements Serializable {
@Test
@Order(2)
public void collectSimRelsTest() throws Exception {
ArgumentApplicationParser parser = new ArgumentApplicationParser(
IOUtils
.toString(
SparkCollectSimRels.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/dedup/collectSimRels_parameters.json")));
parser
.parseArgument(
new String[] {
"-asi", testActionSetId,
"-la", "lookupurl",
"-w", testOutputBasePath,
"-np", "50",
"-purl", "jdbc:postgresql://localhost:5432/dnet_dedup",
"-pusr", "postgres_user",
"-ppwd", ""
});
new SparkCollectSimRels(
parser,
spark,
spark.read().load(testDedupAssertionsBasePath + "/similarity_groups"),
spark.read().load(testDedupAssertionsBasePath + "/groups"))
.run(isLookUpService);
long orgs_simrel = spark
.read()
.load(testOutputBasePath + "/" + testActionSetId + "/organization_simrel")
.count();
long pubs_simrel = spark
.read()
.load(testOutputBasePath + "/" + testActionSetId + "/publication_simrel")
.count();
long sw_simrel = spark
.read()
.load(testOutputBasePath + "/" + testActionSetId + "/software_simrel")
.count();
long ds_simrel = spark
.read()
.load(testOutputBasePath + "/" + testActionSetId + "/dataset_simrel")
.count();
long orp_simrel = spark
.read()
.load(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_simrel")
.count();
assertEquals(3672, orgs_simrel);
assertEquals(10459, pubs_simrel);
assertEquals(3767, sw_simrel);
assertEquals(3865, ds_simrel);
assertEquals(10173, orp_simrel);
}
@Test
@Order(3)
public void cutMergeRelsTest() throws Exception {
ArgumentApplicationParser parser = new ArgumentApplicationParser(
@ -211,6 +272,7 @@ public class SparkDedupTest implements Serializable {
SparkCreateMergeRels.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json")));
parser
.parseArgument(
new String[] {
@ -297,7 +359,7 @@ public class SparkDedupTest implements Serializable {
}
@Test
@Order(3)
@Order(4)
public void createMergeRelsTest() throws Exception {
ArgumentApplicationParser parser = new ArgumentApplicationParser(
@ -306,6 +368,7 @@ public class SparkDedupTest implements Serializable {
SparkCreateMergeRels.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json")));
parser
.parseArgument(
new String[] {
@ -351,7 +414,7 @@ public class SparkDedupTest implements Serializable {
}
@Test
@Order(4)
@Order(5)
public void createDedupRecordTest() throws Exception {
ArgumentApplicationParser parser = new ArgumentApplicationParser(
@ -390,7 +453,7 @@ public class SparkDedupTest implements Serializable {
testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_deduprecord")
.count();
assertEquals(85, orgs_deduprecord);
assertEquals(84, orgs_deduprecord);
assertEquals(65, pubs_deduprecord);
assertEquals(51, sw_deduprecord);
assertEquals(97, ds_deduprecord);
@ -398,7 +461,7 @@ public class SparkDedupTest implements Serializable {
}
@Test
@Order(5)
@Order(6)
public void updateEntityTest() throws Exception {
ArgumentApplicationParser parser = new ArgumentApplicationParser(
@ -474,7 +537,7 @@ public class SparkDedupTest implements Serializable {
.count();
assertEquals(896, publications);
assertEquals(838, organizations);
assertEquals(837, organizations);
assertEquals(100, projects);
assertEquals(100, datasource);
assertEquals(200, softwares);
@ -514,7 +577,7 @@ public class SparkDedupTest implements Serializable {
}
@Test
@Order(6)
@Order(7)
public void propagateRelationTest() throws Exception {
ArgumentApplicationParser parser = new ArgumentApplicationParser(
@ -564,7 +627,7 @@ public class SparkDedupTest implements Serializable {
}
@Test
@Order(7)
@Order(8)
public void testRelations() throws Exception {
testUniqueness("/eu/dnetlib/dhp/dedup/test/relation_1.json", 12, 10);
testUniqueness("/eu/dnetlib/dhp/dedup/test/relation_2.json", 10, 2);

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long

View File

@ -292,9 +292,40 @@ public class MappersTest {
assertValidId(d.getCollectedfrom().get(0).getKey());
assertTrue(StringUtils.isNotBlank(d.getTitle().get(0).getValue()));
assertEquals(1, d.getAuthor().size());
assertEquals(0, d.getSubject().size());
assertEquals(1, d.getSubject().size());
assertEquals(1, d.getInstance().size());
assertEquals(1, d.getPid().size());
assertNotNull(d.getInstance().get(0).getUrl());
}
@Test
void testClaimFromCrossref() throws IOException {
final String xml = IOUtils.toString(getClass().getResourceAsStream("oaf_claim_crossref.xml"));
final List<Oaf> list = new OafToOafMapper(vocs, false).processMdRecord(xml);
System.out.println("***************");
System.out.println(new ObjectMapper().writeValueAsString(list));
System.out.println("***************");
final Publication p = (Publication) list.get(0);
assertValidId(p.getId());
assertValidId(p.getCollectedfrom().get(0).getKey());
System.out.println(p.getTitle().get(0).getValue());
assertTrue(StringUtils.isNotBlank(p.getTitle().get(0).getValue()));
}
@Test
void testODFRecord() throws IOException {
final String xml = IOUtils.toString(getClass().getResourceAsStream("odf_record.xml"));
List<Oaf> list = new OdfToOafMapper(vocs, false).processMdRecord(xml);
System.out.println("***************");
System.out.println(new ObjectMapper().writeValueAsString(list));
System.out.println("***************");
final Dataset p = (Dataset) list.get(0);
assertValidId(p.getId());
assertValidId(p.getCollectedfrom().get(0).getKey());
System.out.println(p.getTitle().get(0).getValue());
assertTrue(StringUtils.isNotBlank(p.getTitle().get(0).getValue()));
}
private void assertValidId(final String id) {

View File

@ -0,0 +1,68 @@
<?xml version="1.0" encoding="UTF-8"?>
<record xmlns:dc="http://purl.org/dc/elements/1.1/"
xmlns:dr="http://www.driver-repository.eu/namespace/dr"
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
xmlns:oaf="http://namespace.openaire.eu/oaf"
xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
<header xmlns="http://namespace.openaire.eu/">
<dri:objIdentifier>userclaim___::7f0f7807f17db50e5c2b5c452ccaf06d</dri:objIdentifier>
<dri:recordIdentifier>userclaim___::7f0f7807f17db50e5c2b5c452ccaf06d</dri:recordIdentifier>
<dri:dateOfCollection>2020-08-06T07:04:09.62Z</dri:dateOfCollection>
<dri:mdFormat/>
<dri:mdFormatInterpretation/>
<dri:repositoryId/>
<dr:objectIdentifier/>
<dr:dateOfCollection/>
<dr:dateOfTransformation>2020-08-06T07:20:57.911Z</dr:dateOfTransformation>
<oaf:datasourceprefix>openaire____</oaf:datasourceprefix>
</header>
<metadata xmlns="http://namespace.openaire.eu/">
<dc:title>A case report of serious haemolysis in a glucose-6-phosphate dehydrogenase-deficient COVID-19 patient receiving hydroxychloroquine</dc:title>
<dc:creator>Maillart, E.</dc:creator>
<dc:creator>Leemans, S.</dc:creator>
<dc:creator>Van Noten, H.</dc:creator>
<dc:creator>Vandergraesen, T.</dc:creator>
<dc:creator>Mahadeb, B.</dc:creator>
<dc:creator>Salaouatchi, M. T.</dc:creator>
<dc:creator>De Bels, D.</dc:creator>
<dc:creator>Clevenbergh, P.</dc:creator>
<dc:date/>
<dc:identifier>http://dx.doi.org/10.1080/23744235.2020.1774644</dc:identifier>
<dc:language/>
<dc:publisher>Informa UK Limited</dc:publisher>
<dc:source>Crossref</dc:source>
<dc:source>Infectious Diseases</dc:source>
<dc:subject>Microbiology (medical)</dc:subject>
<dc:subject>General Immunology and Microbiology</dc:subject>
<dc:subject>Infectious Diseases</dc:subject>
<dc:subject>General Medicine</dc:subject>
<dc:type>journal-article</dc:type>
<dr:CobjCategory type="publication">0001</dr:CobjCategory>
<oaf:dateAccepted>2020-06-04</oaf:dateAccepted>
<oaf:projectid/>
<oaf:accessrights>UNKNOWN</oaf:accessrights>
<oaf:hostedBy
id="openaire____::1256f046-bf1f-4afc-8b47-d0b147148b18" name="Unknown Repository"/>
<oaf:collectedFrom id="openaire____::crossref" name="Crossref"/>
<oaf:identifier identifierType="doi">10.1080/23744235.2020.1774644</oaf:identifier>
<oaf:journal eissn="2374-4243" ep="3" iss="" issn="2374-4235" sp="1" vol="">Infectious Diseases</oaf:journal>
</metadata>
<about xmlns:oai="http://www.openarchives.org/OAI/2.0/">
<provenance xmlns="http://www.openarchives.org/OAI/2.0/provenance" xsi:schemaLocation="http://www.openarchives.org/OAI/2.0/provenance http://www.openarchives.org/OAI/2.0/provenance.xsd">
<originDescription altered="true" harvestDate="2020-08-06T07:04:09.62Z">
<baseURL>file%3A%2F%2F%2Fsrv%2Fclaims%2Frecords%2Fpublication%2Fcrossref</baseURL>
<identifier/>
<datestamp/>
<metadataNamespace/>
</originDescription>
</provenance>
<oaf:datainfo>
<oaf:inferred>false</oaf:inferred>
<oaf:deletedbyinference>false</oaf:deletedbyinference>
<oaf:trust>0.9</oaf:trust>
<oaf:inferenceprovenance/>
<oaf:provenanceaction classid="user:claim" classname="user:claim"
schemeid="dnet:provenanceActions" schemename="dnet:provenanceActions"/>
</oaf:datainfo>
</about>
</record>

View File

@ -1,77 +1,75 @@
<?xml version="1.0" encoding="UTF-8"?>
<record xmlns:dr="http://www.driver-repository.eu/namespace/dr"
xmlns:oaf="http://namespace.openaire.eu/oaf" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
xmlns:oaf="http://namespace.openaire.eu/oaf"
xmlns:oai="http://www.openarchives.org/OAI/2.0/" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
<oai:header xmlns="http://namespace.openaire.eu/"
xmlns:dc="http://purl.org/dc/elements/1.1/"
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
xmlns:oai="http://www.openarchives.org/OAI/2.0/" xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance">
<dri:objIdentifier>r3f5b9831893::cca7367159bc3ff90cd2f75bf9dc21c4</dri:objIdentifier>
<dri:recordIdentifier>oai:nakala.fr:hdl_11280_847e01df</dri:recordIdentifier>
<dri:dateOfCollection>2020-08-01T00:16:24.742Z</dri:dateOfCollection>
xmlns:dri="http://www.driver-repository.eu/namespace/dri" xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance">
<dri:objIdentifier>r3f5b9831893::01a497c6c6b44289c52dcdf22b6c0fc0</dri:objIdentifier>
<dri:recordIdentifier>oai:nakala.fr:hdl_11280_50f302c6</dri:recordIdentifier>
<dri:dateOfCollection>2020-10-03T06:06:52.228Z</dri:dateOfCollection>
<oaf:datasourceprefix>r3f5b9831893</oaf:datasourceprefix>
<identifier xmlns="http://www.openarchives.org/OAI/2.0/">oai:nakala.fr:hdl_11280_847e01df</identifier>
<datestamp xmlns="http://www.openarchives.org/OAI/2.0/">2020-06-08T01:01:38Z</datestamp>
<setSpec xmlns="http://www.openarchives.org/OAI/2.0/">hdl_11280_2b09fc10</setSpec>
<setSpec xmlns="http://www.openarchives.org/OAI/2.0/">hdl_11280_c1bc48d0</setSpec>
<setSpec xmlns="http://www.openarchives.org/OAI/2.0/">hdl_11280_57c8db3a</setSpec>
<dr:dateOfTransformation>2020-08-01T00:31:35.625Z</dr:dateOfTransformation>
<identifier xmlns="http://www.openarchives.org/OAI/2.0/">oai:nakala.fr:hdl_11280_50f302c6</identifier>
<datestamp xmlns="http://www.openarchives.org/OAI/2.0/">2020-09-19T23:56:08Z</datestamp>
<setSpec xmlns="http://www.openarchives.org/OAI/2.0/">hdl_11280_96355742</setSpec>
<setSpec xmlns="http://www.openarchives.org/OAI/2.0/">hdl_11280_26914437</setSpec>
<setSpec xmlns="http://www.openarchives.org/OAI/2.0/">hdl_11280_86561837</setSpec>
<dr:dateOfTransformation>2020-10-19T15:39:52.151Z</dr:dateOfTransformation>
</oai:header>
<metadata>
<datacite:resource xmlns="http://www.openarchives.org/OAI/2.0/"
xmlns:datacite="http://datacite.org/schema/kernel-4"
xmlns:dc="http://purl.org/dc/elements/1.1/"
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
xmlns:oai="http://www.openarchives.org/OAI/2.0/"
xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance" xsi:schemaLocation="http://datacite.org/schema/kernel-4 https://schema.datacite.org/meta/kernel-4/metadata.xsd">
<datacite:alternateIdentifier identifierType="URL" xmlns:datacite="http://datacite.org/schema/kernel-4/">277</datacite:alternateIdentifier>
<datacite:identifier identifierType="Handle" xmlns:datacite="http://datacite.org/schema/kernel-4/">http://hdl.handle.net/11280/847e01df</datacite:identifier>
<alternateIdentifiers>
<alternateIdentifier alternateIdentifierType="URL">http://hdl.handle.net/http://hdl.handle.net/11280/847e01df</alternateIdentifier>
</alternateIdentifiers>
<datacite:alternateIdentifier identifierType="URL" xmlns:datacite="http://datacite.org/schema/kernel-4/">http://nakala.fr/data/11280/847e01df</datacite:alternateIdentifier>
<datacite:creators xmlns:datacite="http://datacite.org/schema/kernel-4/">
<alternateIdentifier alternateIdentifierType="URL" xmlns="http://datacite.org/schema/kernel-4">http://nakala.fr/data/11280/50f302c6</alternateIdentifier>
<identifier identifierType="Handle" xmlns="http://datacite.org/schema/kernel-4">11280/50f302c6</identifier>
<datacite:creators>
<datacite:creator>
<datacite:creatorName>DHAAP</datacite:creatorName>
<datacite:creatorName>Desbrosse, Xavier</datacite:creatorName>
</datacite:creator>
</datacite:creators>
<datacite:titles xmlns:datacite="http://datacite.org/schema/kernel-4/">
<datacite:title>CVP_Notice277-1 place du Docteur Antoine Béclère _PHO02.jpg</datacite:title>
<datacite:titles>
<datacite:title>Les rues Stalingrad en France (1945-2013)</datacite:title>
</datacite:titles>
<datacite:descriptions xmlns:datacite="http://datacite.org/schema/kernel-4/">
<datacite:description descriptionType="Abstract">Hôpital Saint-Antoine. Fragment de dalle funéraire trouvée en décembre 1932. Paris (XIIème arr.). Photographie d'Albert Citerne (1876-1970). Plaque de verre, 1932. Département Histoire de l'Architecture et Archéologie de Paris.</datacite:description>
<datacite:description descriptionType="Abstract">Nfa_1146</datacite:description>
<datacite:description descriptionType="Abstract">Hôpital Saint-Antoine. Fragment de dalle funéraire trouvée en décembre 1932. Paris (XIIème arr.). Photographie d'Albert Citerne (1876-1970). Plaque de verre, 1932. Département Histoire de l'Architecture et Archéologie de Paris.</datacite:description>
<datacite:subjects>
<datacite:subject>Rues Noms -- France</datacite:subject>
</datacite:subjects>
<datacite:descriptions>
<datacite:description descriptionType="Abstract">Cette carte appartient à la collection « Guerre froide vue den bas » élaborée dans le cadre de lenquête 2009-2013 du réseau des correspondants départementaux de lIHTP « La Guerre froide vue den bas : 1947-1967 », enquête conduite sous la direction de Philippe Buton Professeur dHistoire contemporaine à lUniversité de Reims, dOlivier Büttner Ingénieur de Recherche IHTP-CNRS et de Michel Hastings, Professeur de Science politique à lInstitut dEtudes Politiques de Lille.</datacite:description>
</datacite:descriptions>
<datacite:publisher xmlns:datacite="http://datacite.org/schema/kernel-4/">Nakala by Huma-Num</datacite:publisher>
<datacite:contributors xmlns:datacite="http://datacite.org/schema/kernel-4/">
<datacite:publisher>IHTP-CNRS</datacite:publisher>
<datacite:contributors>
<datacite:contributor contributorType="Other">
<datacite:contributorName>DHAAP, Pôle Archéologique</datacite:contributorName>
<datacite:contributorName>(CNRS), Institut d'Histoire du Temps Présent (IHTP) - Centre National de la Recherche Scientifique </datacite:contributorName>
</datacite:contributor>
</datacite:contributors>
<datacite:dates xmlns:datacite="http://datacite.org/schema/kernel-4/">
<datacite:date dateType="Created">1932</datacite:date>
<datacite:dates>
<datacite:date dateType="Created">2013</datacite:date>
</datacite:dates>
<datacite:resourceType resourceTypeGeneral="Image" xmlns:datacite="http://datacite.org/schema/kernel-4/">StillImage</datacite:resourceType>
<datacite:rightsList xmlns:datacite="http://datacite.org/schema/kernel-4/">
<datacite:rights rightsURI="info:eu-repo/semantics/openAccess"/>
</datacite:rightsList>
<datacite:resourceType resourceTypeGeneral="Image">Carte</datacite:resourceType>
<datacite:geoLocations>
<datacite:geoLocation>
<datacite:geoLocationPlace>France</datacite:geoLocationPlace>
</datacite:geoLocation>
</datacite:geoLocations>
</datacite:resource>
<oaf:identifier identifierType="handle">http://hdl.handle.net/11280/847e01df</oaf:identifier>
<oaf:identifier identifierType="handle">11280/50f302c6</oaf:identifier>
<oaf:concept id="dariah"/>
<dr:CobjCategory type="dataset">0025</dr:CobjCategory>
<oaf:dateAccepted/>
<oaf:accessrights>OPEN</oaf:accessrights>
<oaf:accessrights>UNKNOWN</oaf:accessrights>
<oaf:language>und</oaf:language>
<oaf:hostedBy id="re3data_____::r3d100012102" name="NAKALA"/>
<oaf:collectedFrom id="re3data_____::r3d100012102" name="NAKALA"/>
</metadata>
<about xmlns:dc="http://purl.org/dc/elements/1.1/"
xmlns:dri="http://www.driver-repository.eu/namespace/dri"
xmlns:oai="http://www.openarchives.org/OAI/2.0/" xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance">
xmlns:dri="http://www.driver-repository.eu/namespace/dri" xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance">
<provenance xmlns="http://www.openarchives.org/OAI/2.0/provenance" xsi:schemaLocation="http://www.openarchives.org/OAI/2.0/provenance http://www.openarchives.org/OAI/2.0/provenance.xsd">
<originDescription altered="true" harvestDate="2020-08-01T00:16:24.742Z">
<baseURL>https%3A%2F%2Fwww.nakala.fr%2Foai_oa%2F11280%2F8892ab4b</baseURL>
<identifier>oai:nakala.fr:hdl_11280_847e01df</identifier>
<datestamp>2020-06-08T01:01:38Z</datestamp>
<originDescription altered="true" harvestDate="2020-10-03T06:06:52.228Z">
<baseURL>https%3A%2F%2Fwww.nakala.fr%2Foai_oa%2F11280%2F92c4d30b</baseURL>
<identifier>oai:nakala.fr:hdl_11280_50f302c6</identifier>
<datestamp>2020-09-19T23:56:08Z</datestamp>
<metadataNamespace/>
</originDescription>
</provenance>

View File

@ -0,0 +1,102 @@
<?xml version="1.0" encoding="UTF-8"?>
<record xmlns:dr="http://www.driver-repository.eu/namespace/dr"
xmlns:oaf="http://namespace.openaire.eu/oaf"
xmlns:oai="http://www.openarchives.org/OAI/2.0/" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
<oai:header xmlns="http://namespace.openaire.eu/"
xmlns:dc="http://purl.org/dc/elements/1.1/"
xmlns:dri="http://www.driver-repository.eu/namespace/dri" xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance">
<dri:objIdentifier>r3a507cdacc5::03b31980d9bb3c4609e6005c4a3baba6</dri:objIdentifier>
<dri:recordIdentifier>oai:lindat.mff.cuni.cz:11372/LRT-1844</dri:recordIdentifier>
<dri:dateOfCollection>2020-09-04T14:36:48.411Z</dri:dateOfCollection>
<oaf:datasourceprefix>r3a507cdacc5</oaf:datasourceprefix>
<identifier xmlns="http://www.openarchives.org/OAI/2.0/">oai:lindat.mff.cuni.cz:11372/LRT-1844</identifier>
<datestamp xmlns="http://www.openarchives.org/OAI/2.0/">2016-12-07T11:10:30Z</datestamp>
<setSpec xmlns="http://www.openarchives.org/OAI/2.0/">hdl_11858_00-097C-0000-0007-710A-A</setSpec>
<setSpec xmlns="http://www.openarchives.org/OAI/2.0/">hdl_11858_00-097C-0000-0007-710B-8</setSpec>
<setSpec xmlns="http://www.openarchives.org/OAI/2.0/">openaire_data</setSpec>
<dr:dateOfTransformation>2020-09-04T14:39:16.458Z</dr:dateOfTransformation>
</oai:header>
<metadata>
<resource xmlns="http://datacite.org/schema/kernel-4"
xmlns:dc="http://purl.org/dc/elements/1.1/"
xmlns:dri="http://www.driver-repository.eu/namespace/dri" xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance">
<identifier identifierType="Handle">11372/LRT-1844</identifier>
<alternateIdentifiers>
<alternateIdentifier alternateIdentifierType="URL">http://hdl.handle.net/11372/LRT-1844</alternateIdentifier>
</alternateIdentifiers>
<creators>
<creator>
<creatorName>Hercig, Tomáš</creatorName>
</creator>
<creator>
<creatorName>Brychcín, Tomáš</creatorName>
</creator>
<creator>
<creatorName>Svoboda, Lukáš</creatorName>
</creator>
<creator>
<creatorName>Konkol, Michal</creatorName>
</creator>
<creator>
<creatorName>Steinberger, Josef</creatorName>
</creator>
</creators>
<titles>
<title>Restaurant Reviews CZ ABSA corpus v2</title>
</titles>
<publisher>University of West Bohemia, Department of Computer Science and Engineering</publisher>
<publicationYear>2016</publicationYear>
<contributors>
<contributor contributorType="Funder">
<contributorName>European Commission</contributorName>
<nameIdentifier nameIdentifierScheme="info">info:eu-repo/grantAgreement/EC/FP7/630786</nameIdentifier>
</contributor>
</contributors>
<dates>
<date dateType="Issued">2016</date>
<date dateType="Accepted">2016-12-07T11:10:30Z</date>
<date dateType="Available">2016-12-07T11:10:30Z</date>
</dates>
<resourceType resourceTypeGeneral="Dataset">corpus</resourceType>
<rightsList>
<rights rightsURI="info:eu-repo/semantics/openAccess"/>
<rights rightsURI="http://creativecommons.org/licenses/by-nc-sa/4.0/"/>
</rightsList>
<descriptions>
<description descriptionType="Abstract">Restaurant Reviews CZ ABSA - 2.15k reviews with their related target and category
The work done is described in the paper: https://doi.org/10.13053/CyS-20-3-2469</description>
</descriptions>
</resource>
<oaf:identifier identifierType="handle">11372/LRT-1844</oaf:identifier>
<oaf:embargoenddate>2016-12-07</oaf:embargoenddate>
<dr:CobjCategory type="dataset">0021</dr:CobjCategory>
<oaf:dateAccepted>2016-01-01</oaf:dateAccepted>
<oaf:accessrights>OPEN</oaf:accessrights>
<oaf:license>http://creativecommons.org/licenses/by-nc-sa/4.0/</oaf:license>
<oaf:language>und</oaf:language>
<oaf:projectid>corda_______::630786</oaf:projectid>
<oaf:hostedBy id="re3data_____::r3d100010386" name="LINDAT/CLARIN repository"/>
<oaf:collectedFrom id="re3data_____::r3d100010386" name="LINDAT/CLARIN repository"/>
</metadata>
<about xmlns:dc="http://purl.org/dc/elements/1.1/"
xmlns:dri="http://www.driver-repository.eu/namespace/dri" xmlns:prov="http://www.openarchives.org/OAI/2.0/provenance">
<provenance xmlns="http://www.openarchives.org/OAI/2.0/provenance" xsi:schemaLocation="http://www.openarchives.org/OAI/2.0/provenance http://www.openarchives.org/OAI/2.0/provenance.xsd">
<originDescription altered="true" harvestDate="2020-09-04T14:36:48.411Z">
<baseURL>https%3A%2F%2Flindat.mff.cuni.cz%2Frepository%2Foai%2Fopenaire_data</baseURL>
<identifier>oai:lindat.mff.cuni.cz:11372/LRT-1844</identifier>
<datestamp>2016-12-07T11:10:30Z</datestamp>
<metadataNamespace/>
</originDescription>
</provenance>
<oaf:datainfo>
<oaf:inferred>false</oaf:inferred>
<oaf:deletedbyinference>false</oaf:deletedbyinference>
<oaf:trust>0.9</oaf:trust>
<oaf:inferenceprovenance/>
<oaf:provenanceaction classid="sysimport:crosswalk:datasetarchive"
classname="sysimport:crosswalk:datasetarchive"
schemeid="dnet:provenanceActions" schemename="dnet:provenanceActions"/>
</oaf:datainfo>
</about>
</record>