From 1699d41d39a5d70e681589b3cf03b9d61d1a9a06 Mon Sep 17 00:00:00 2001 From: miconis Date: Thu, 5 Nov 2020 15:48:42 +0100 Subject: [PATCH 1/5] relations for openorgs: not it choose only one master --- .../dhp/oa/dedup/SparkPrepareOrgRels.java | 50 ++++++++++--------- 1 file changed, 26 insertions(+), 24 deletions(-) diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPrepareOrgRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPrepareOrgRels.java index e9933c4e5..19e6127f6 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPrepareOrgRels.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPrepareOrgRels.java @@ -1,13 +1,9 @@ + 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 java.io.IOException; +import java.util.*; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.MapFunction; @@ -17,12 +13,19 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +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 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); @@ -125,12 +128,11 @@ public class SparkPrepareOrgRels extends AbstractSparkAction { List ids = sortIds(l); List> 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)); - } + String source = ids.get(0); + for (String target : ids) { + rels.add(new Tuple3<>(source, target, groupId)); } + return rels.iterator(); }) .rdd(), @@ -235,14 +237,14 @@ public class SparkPrepareOrgRels extends AbstractSparkAction { .joinWith(entities, relations.col("_2").equalTo(entities.col("_1")), "inner") .map( (MapFunction, Tuple2>, 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()), + 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>) o -> new Tuple2<>(o.getLocal_id(), o), From 5af5a8ae42c5c3303bad4f0eeaccee518ef0480f Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Wed, 9 Nov 2022 14:20:59 +0100 Subject: [PATCH 2/5] added IdentifierComparator --- .../dhp/oa/dedup/DedupRecordFactory.java | 21 ++++- .../eu/dnetlib/dhp/oa/dedup/IdGenerator.java | 4 + .../dhp/oa/dedup/IdentifierComparator.java | 81 +++++++++++++++++++ .../dhp/oa/dedup/model/Identifier.java | 53 +----------- .../dhp/oa/dedup/SparkOpenorgsDedupTest.java | 4 +- .../dnetlib/dhp/oa/dedup/SparkStatsTest.java | 4 +- .../src/test/resources/log4j.properties | 25 ++++++ 7 files changed, 134 insertions(+), 58 deletions(-) create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/IdentifierComparator.java create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/resources/log4j.properties diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java index d345cf98f..c3b6751ba 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java @@ -2,8 +2,10 @@ package eu.dnetlib.dhp.oa.dedup; import java.util.Collection; +import java.util.Comparator; import java.util.Iterator; import java.util.List; +import java.util.stream.Collectors; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.api.java.function.MapGroupsFunction; @@ -13,11 +15,14 @@ import org.apache.spark.sql.SparkSession; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Iterators; 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.*; +import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; import scala.Tuple2; public class DedupRecordFactory { @@ -82,11 +87,19 @@ public class DedupRecordFactory { final Collection dates = Lists.newArrayList(); final List> authors = Lists.newArrayList(); - entities - .forEachRemaining( - t -> { - T duplicate = t._2(); + final Comparator> idComparator = new IdentifierComparator().reversed(); + final List entityList = Lists + .newArrayList(entities) + .stream() + .map(t -> Identifier.newInstance(t._2())) + .sorted(idComparator) + .map(Identifier::getEntity) + .collect(Collectors.toList()); + + entityList + .forEach( + duplicate -> { entity.mergeFrom(duplicate); if (ModelSupport.isSubClass(duplicate, Result.class)) { Result r1 = (Result) duplicate; diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/IdGenerator.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/IdGenerator.java index 81cd30f88..7e0d66062 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/IdGenerator.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/IdGenerator.java @@ -18,6 +18,10 @@ public class IdGenerator implements Serializable { if (pids == null || pids.isEmpty()) return defaultID; + return generateId(pids); + } + + private static String generateId(List> pids) { Identifier bp = pids .stream() .min(Identifier::compareTo) diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/IdentifierComparator.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/IdentifierComparator.java new file mode 100644 index 000000000..ba4e31128 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/IdentifierComparator.java @@ -0,0 +1,81 @@ + +package eu.dnetlib.dhp.oa.dedup; + +import java.util.Comparator; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import com.google.common.collect.Sets; + +import eu.dnetlib.dhp.oa.dedup.model.Identifier; +import eu.dnetlib.dhp.schema.common.EntityType; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.DataInfo; +import eu.dnetlib.dhp.schema.oaf.KeyValue; +import eu.dnetlib.dhp.schema.oaf.OafEntity; +import eu.dnetlib.dhp.schema.oaf.StructuredProperty; +import eu.dnetlib.dhp.schema.oaf.utils.OafMapperUtils; +import eu.dnetlib.dhp.schema.oaf.utils.PidComparator; +import eu.dnetlib.dhp.schema.oaf.utils.PidType; + +public class IdentifierComparator implements Comparator> { + + public static int compareIdentifiers(Identifier left, Identifier right) { + return new IdentifierComparator<>().compare(left, right); + } + + @Override + public int compare(Identifier left, Identifier i) { + // priority in comparisons: 1) pidtype, 2) collectedfrom (depending on the entity type) , 3) date 4) + // alphabetical order of the originalID + + Set lKeys = Optional + .ofNullable(left.getCollectedFrom()) + .map(c -> c.stream().map(KeyValue::getKey).collect(Collectors.toSet())) + .orElse(Sets.newHashSet()); + + final Optional> cf = Optional.ofNullable(i.getCollectedFrom()); + Set rKeys = cf + .map(c -> c.stream().map(KeyValue::getKey).collect(Collectors.toSet())) + .orElse(Sets.newHashSet()); + + if (left.getPidType().compareTo(i.getPidType()) == 0) { // same type + if (left.getEntityType() == EntityType.publication) { + if (isFromDatasourceID(lKeys, ModelConstants.CROSSREF_ID) + && !isFromDatasourceID(rKeys, ModelConstants.CROSSREF_ID)) + return -1; + if (isFromDatasourceID(rKeys, ModelConstants.CROSSREF_ID) + && !isFromDatasourceID(lKeys, ModelConstants.CROSSREF_ID)) + return 1; + } + if (left.getEntityType() == EntityType.dataset) { + if (isFromDatasourceID(lKeys, ModelConstants.DATACITE_ID) + && !isFromDatasourceID(rKeys, ModelConstants.DATACITE_ID)) + return -1; + if (isFromDatasourceID(rKeys, ModelConstants.DATACITE_ID) + && !isFromDatasourceID(lKeys, ModelConstants.DATACITE_ID)) + return 1; + } + + if (left.getDate().compareTo(i.getDate()) == 0) {// same date + // we need to take the alphabetically lower id + return left.getOriginalID().compareTo(i.getOriginalID()); + } else + // we need to take the elder date + return left.getDate().compareTo(i.getDate()); + } else { + return new PidComparator<>(left.getEntity()).compare(toSP(left.getPidType()), toSP(i.getPidType())); + } + } + + public boolean isFromDatasourceID(Set collectedFrom, String dsId) { + return collectedFrom.contains(dsId); + } + + private StructuredProperty toSP(PidType pidType) { + return OafMapperUtils.structuredProperty("", pidType.toString(), pidType.toString(), "", "", new DataInfo()); + } + +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/Identifier.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/Identifier.java index a25a853ef..0cba4fc3b 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/Identifier.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/Identifier.java @@ -11,6 +11,7 @@ import org.apache.commons.lang3.StringUtils; import com.google.common.collect.Sets; import eu.dnetlib.dhp.oa.dedup.DatePicker; +import eu.dnetlib.dhp.oa.dedup.IdentifierComparator; import eu.dnetlib.dhp.schema.common.EntityType; import eu.dnetlib.dhp.schema.common.ModelConstants; import eu.dnetlib.dhp.schema.common.ModelSupport; @@ -83,60 +84,12 @@ public class Identifier implements Serializable, Comparable return entity.getId(); } - private PidType getPidType() { + public PidType getPidType() { return PidType.tryValueOf(StringUtils.substringBefore(StringUtils.substringAfter(entity.getId(), "|"), "_")); } @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 lKeys = Optional - .ofNullable(getCollectedFrom()) - .map(c -> c.stream().map(KeyValue::getKey).collect(Collectors.toSet())) - .orElse(Sets.newHashSet()); - - final Optional> cf = Optional.ofNullable(i.getCollectedFrom()); - Set rKeys = cf - .map(c -> c.stream().map(KeyValue::getKey).collect(Collectors.toSet())) - .orElse(Sets.newHashSet()); - - if (this.getPidType().compareTo(i.getPidType()) == 0) { // same type - if (getEntityType() == EntityType.publication) { - if (isFromDatasourceID(lKeys, ModelConstants.CROSSREF_ID) - && !isFromDatasourceID(rKeys, ModelConstants.CROSSREF_ID)) - return -1; - if (isFromDatasourceID(rKeys, ModelConstants.CROSSREF_ID) - && !isFromDatasourceID(lKeys, ModelConstants.CROSSREF_ID)) - return 1; - } - if (getEntityType() == EntityType.dataset) { - if (isFromDatasourceID(lKeys, ModelConstants.DATACITE_ID) - && !isFromDatasourceID(rKeys, ModelConstants.DATACITE_ID)) - return -1; - if (isFromDatasourceID(rKeys, ModelConstants.DATACITE_ID) - && !isFromDatasourceID(lKeys, ModelConstants.DATACITE_ID)) - return 1; - } - - if (this.getDate().compareTo(i.getDate()) == 0) {// same date - // we need to take the alphabetically lower id - return this.getOriginalID().compareTo(i.getOriginalID()); - } else - // we need to take the elder date - return this.getDate().compareTo(i.getDate()); - } else { - return new PidComparator<>(getEntity()).compare(toSP(getPidType()), toSP(i.getPidType())); - } - - } - - private StructuredProperty toSP(PidType pidType) { - return OafMapperUtils.structuredProperty("", pidType.toString(), pidType.toString(), "", "", new DataInfo()); - } - - public boolean isFromDatasourceID(Set collectedFrom, String dsId) { - return collectedFrom.contains(dsId); + return IdentifierComparator.compareIdentifiers(this, i); } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsDedupTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsDedupTest.java index 9312d83b1..88c28ab2f 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsDedupTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsDedupTest.java @@ -143,7 +143,7 @@ public class SparkOpenorgsDedupTest implements Serializable { .load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "organization")) .count(); - assertEquals(288, orgs_simrel); + assertEquals(290, orgs_simrel); } @Test @@ -172,7 +172,7 @@ public class SparkOpenorgsDedupTest implements Serializable { .load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "organization")) .count(); - assertEquals(324, orgs_simrel); + assertEquals(326, orgs_simrel); } @Test diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkStatsTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkStatsTest.java index 1ba2c717c..b33b627e7 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkStatsTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkStatsTest.java @@ -168,11 +168,11 @@ public class SparkStatsTest implements Serializable { .textFile(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_blockstats") .count(); - assertEquals(477, orgs_blocks); + assertEquals(480, orgs_blocks); assertEquals(295, pubs_blocks); assertEquals(122, sw_blocks); assertEquals(191, ds_blocks); - assertEquals(171, orp_blocks); + assertEquals(178, orp_blocks); } @AfterAll diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/log4j.properties b/dhp-workflows/dhp-dedup-openaire/src/test/resources/log4j.properties new file mode 100644 index 000000000..ce37270c6 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/log4j.properties @@ -0,0 +1,25 @@ +# Root logger option +log4j.rootLogger=DEBUG, stdout + +# Direct log messages to stdout +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.Target=System.out +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n + +# Change this to set Spark log level +log4j.logger.org.apache.spark=ERROR +log4j.rootCategory=WARN + +# Silence akka remoting +log4j.logger.Remoting=WARN + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN + +log4j.logger.org.apache.hadoop.mapreduce.lib.output.FileOutputCommitterFactory=WARN +log4j.logger.org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter=WARN +log4j.logger.org.apache.parquet.hadoop.ParquetOutputFormat=WARN +log4j.logger.org.apache.parquet.hadoop.InternalParquetRecordWriter=WARN +log4j.logger.org.apache.hadoop.io.compress.CodecPool=WARN +log4j.logger.org.apache.parquet.hadoop.codec.CodecConfig=WARN \ No newline at end of file From ddff0e8999b392bdec7b9d6263e5c53e16a6bf59 Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Fri, 11 Nov 2022 16:10:25 +0100 Subject: [PATCH 3/5] merging duplicates using IdentifierComparator --- .../dhp/oa/dedup/DedupRecordFactory.java | 35 +- .../dhp/oa/dedup/EntityMergerTest.java | 15 +- .../oa/dedup/SparkDedupPublicationTest.java | 389 ++++++++++++++++++ .../dnetlib/dhp/oa/dedup/SparkDedupTest.java | 210 ++++++---- .../dedup/entities/publication/publication.gz | Bin 578793 -> 581926 bytes .../entities2/publication/publication.gz | Bin 0 -> 9056 bytes .../mock_orchestrator_publication.xml | 24 ++ .../src/test/resources/log4j.properties | 28 +- 8 files changed, 590 insertions(+), 111 deletions(-) create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupPublicationTest.java create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/entities2/publication/publication.gz create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/profiles/mock_orchestrator_publication.xml diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java index c3b6751ba..f9fc8a21a 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java @@ -1,12 +1,12 @@ package eu.dnetlib.dhp.oa.dedup; -import java.util.Collection; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; +import java.lang.reflect.InvocationTargetException; +import java.util.*; import java.util.stream.Collectors; +import org.apache.commons.beanutils.BeanUtils; +import org.apache.commons.lang3.StringUtils; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.api.java.function.MapGroupsFunction; import org.apache.spark.sql.Dataset; @@ -15,14 +15,12 @@ import org.apache.spark.sql.SparkSession; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.Iterators; 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.*; -import eu.dnetlib.dhp.schema.oaf.utils.IdentifierFactory; import scala.Tuple2; public class DedupRecordFactory { @@ -79,23 +77,25 @@ public class DedupRecordFactory { public static T entityMerger( String id, Iterator> entities, long ts, DataInfo dataInfo, Class clazz) - throws IllegalAccessException, InstantiationException { + throws IllegalAccessException, InstantiationException, InvocationTargetException { - T entity = clazz.newInstance(); - entity.setDataInfo(dataInfo); + final Comparator> idComparator = new IdentifierComparator<>(); - final Collection dates = Lists.newArrayList(); - final List> authors = Lists.newArrayList(); - - final Comparator> idComparator = new IdentifierComparator().reversed(); - - final List entityList = Lists + final LinkedList entityList = Lists .newArrayList(entities) .stream() .map(t -> Identifier.newInstance(t._2())) .sorted(idComparator) .map(Identifier::getEntity) - .collect(Collectors.toList()); + .collect(Collectors.toCollection(LinkedList::new)); + + final T entity = clazz.newInstance(); + final T first = entityList.removeFirst(); + + BeanUtils.copyProperties(entity, first); + + final Collection dates = Lists.newArrayList(); + final List> authors = Lists.newArrayList(); entityList .forEach( @@ -103,12 +103,11 @@ public class DedupRecordFactory { entity.mergeFrom(duplicate); if (ModelSupport.isSubClass(duplicate, Result.class)) { Result r1 = (Result) duplicate; - if (r1.getAuthor() != null && !r1.getAuthor().isEmpty()) + if (r1.getAuthor() != null && StringUtils.isNotBlank(r1.getDateofacceptance().getValue())) authors.add(r1.getAuthor()); if (r1.getDateofacceptance() != null) dates.add(r1.getDateofacceptance().getValue()); } - }); // set authors and date diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java index aa3722ce5..c9cfb8cb2 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/EntityMergerTest.java @@ -7,6 +7,7 @@ import java.io.BufferedReader; import java.io.FileReader; import java.io.IOException; import java.io.Serializable; +import java.lang.reflect.InvocationTargetException; import java.nio.file.Paths; import java.util.*; import java.util.stream.Collectors; @@ -54,7 +55,7 @@ class EntityMergerTest implements Serializable { } @Test - void softwareMergerTest() throws InstantiationException, IllegalAccessException { + void softwareMergerTest() throws InstantiationException, IllegalAccessException, InvocationTargetException { List> softwares = readSample( testEntityBasePath + "/software_merge.json", Software.class); @@ -69,7 +70,7 @@ class EntityMergerTest implements Serializable { } @Test - void publicationMergerTest() throws InstantiationException, IllegalAccessException { + void publicationMergerTest() throws InstantiationException, IllegalAccessException, InvocationTargetException { Publication pub_merged = DedupRecordFactory .entityMerger(dedupId, publications.iterator(), 0, dataInfo, Publication.class); @@ -134,7 +135,7 @@ class EntityMergerTest implements Serializable { } @Test - void publicationMergerTest2() throws InstantiationException, IllegalAccessException { + void publicationMergerTest2() throws InstantiationException, IllegalAccessException, InvocationTargetException { Publication pub_merged = DedupRecordFactory .entityMerger(dedupId, publications2.iterator(), 0, dataInfo, Publication.class); @@ -146,7 +147,7 @@ class EntityMergerTest implements Serializable { } @Test - void publicationMergerTest3() throws InstantiationException, IllegalAccessException { + void publicationMergerTest3() throws InstantiationException, IllegalAccessException, InvocationTargetException { Publication pub_merged = DedupRecordFactory .entityMerger(dedupId, publications3.iterator(), 0, dataInfo, Publication.class); @@ -156,7 +157,8 @@ class EntityMergerTest implements Serializable { } @Test - void publicationMergerTest4() throws InstantiationException, IllegalStateException, IllegalAccessException { + void publicationMergerTest4() + throws InstantiationException, IllegalStateException, IllegalAccessException, InvocationTargetException { Publication pub_merged = DedupRecordFactory .entityMerger(dedupId, publications4.iterator(), 0, dataInfo, Publication.class); @@ -166,7 +168,8 @@ class EntityMergerTest implements Serializable { } @Test - void publicationMergerTest5() throws InstantiationException, IllegalStateException, IllegalAccessException { + void publicationMergerTest5() + throws InstantiationException, IllegalStateException, IllegalAccessException, InvocationTargetException { System.out .println( diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupPublicationTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupPublicationTest.java new file mode 100644 index 000000000..c657d1865 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupPublicationTest.java @@ -0,0 +1,389 @@ + +package eu.dnetlib.dhp.oa.dedup; + +import static java.nio.file.Files.createTempDirectory; + +import static org.apache.spark.sql.functions.count; +import static org.junit.jupiter.api.Assertions.*; +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.*; +import java.util.stream.Collectors; + +import com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FilterFunction; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.*; +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.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Sets; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; + +@ExtendWith(MockitoExtension.class) +@TestMethodOrder(MethodOrderer.OrderAnnotation.class) +public class SparkDedupPublicationTest implements Serializable { + + @Mock(serializable = true) + ISLookUpService isLookUpService; + + private static SparkSession spark; + private static JavaSparkContext jsc; + + private static String testGraphBasePath; + private static String testOutputBasePath; + private static String testDedupGraphBasePath; + private static final String testActionSetId = "test-orchestrator"; + + @BeforeAll + public static void cleanUp() throws IOException, URISyntaxException { + + testGraphBasePath = Paths + .get(SparkDedupPublicationTest.class.getResource("/eu/dnetlib/dhp/dedup/entities2").toURI()) + .toFile() + .getAbsolutePath(); + testOutputBasePath = createTempDirectory(SparkDedupPublicationTest.class.getSimpleName() + "-") + .toAbsolutePath() + .toString(); + + testDedupGraphBasePath = createTempDirectory(SparkDedupPublicationTest.class.getSimpleName() + "-") + .toAbsolutePath() + .toString(); + + FileUtils.deleteDirectory(new File(testOutputBasePath)); + FileUtils.deleteDirectory(new File(testDedupGraphBasePath)); + + final SparkConf conf = new SparkConf(); + conf.set("spark.sql.shuffle.partitions", "10"); + spark = SparkSession + .builder() + .appName(SparkDedupPublicationTest.class.getSimpleName()) + .master("local[*]") + .config(conf) + .getOrCreate(); + + jsc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + } + + @BeforeEach + public void setUp() throws IOException, ISLookUpException { + + lenient() + .when(isLookUpService.getResourceProfileByQuery(Mockito.contains(testActionSetId))) + .thenReturn(classPathResourceAsString("/eu/dnetlib/dhp/dedup/profiles/mock_orchestrator_publication.xml")); + + lenient() + .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("publication"))) + .thenReturn(classPathResourceAsString("/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json")); + } + + @Test + @Order(1) + void createSimRelsTest() throws Exception { + + ArgumentApplicationParser parser = new ArgumentApplicationParser( + classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json")); + + parser + .parseArgument( + new String[] { + "--graphBasePath", testGraphBasePath, + "--actionSetId", testActionSetId, + "--isLookUpUrl", "lookupurl", + "--workingPath", testOutputBasePath, + "--numPartitions", "5" + }); + + new SparkCreateSimRels(parser, spark).run(isLookUpService); + + long pubs_simrel = spark + .read() + .load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "publication")) + .count(); + + assertEquals(62, pubs_simrel); + } + + @Test + @Order(2) + void cutMergeRelsTest() throws Exception { + + ArgumentApplicationParser parser = new ArgumentApplicationParser( + classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json")); + + parser + .parseArgument( + new String[] { + "--graphBasePath", testGraphBasePath, + "--actionSetId", testActionSetId, + "--isLookUpUrl", "lookupurl", + "--workingPath", testOutputBasePath, + "--cutConnectedComponent", "3" + }); + + new SparkCreateMergeRels(parser, spark).run(isLookUpService); + + long pubs_mergerel = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/publication_mergerel") + .as(Encoders.bean(Relation.class)) + .filter((FilterFunction) r -> r.getRelClass().equalsIgnoreCase("merges")) + .groupBy("source") + .agg(count("target").alias("cnt")) + .select("source", "cnt") + .where("cnt > 3") + .count(); + + assertEquals(0, pubs_mergerel); + + FileUtils.deleteDirectory(new File(testOutputBasePath + "/" + testActionSetId + "/publication_mergerel")); + } + + @Test + @Order(3) + void createMergeRelsTest() throws Exception { + + ArgumentApplicationParser parser = new ArgumentApplicationParser( + classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json")); + + parser + .parseArgument( + new String[] { + "--graphBasePath", testGraphBasePath, + "--actionSetId", testActionSetId, + "--isLookUpUrl", "lookupurl", + "--workingPath", testOutputBasePath + }); + + new SparkCreateMergeRels(parser, spark).run(isLookUpService); + + final Dataset pubs = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/publication_mergerel") + .as(Encoders.bean(Relation.class)); + + final List merges = pubs + .filter("source == '50|doi_dedup___::d5021b53204e4fdeab6ff5d5bc468032'") + .collectAsList(); + assertEquals(3, merges.size()); + Set dups = Sets + .newHashSet( + "50|doi_________::3b1d0d8e8f930826665df9d6b82fbb73", + "50|doi_________::d5021b53204e4fdeab6ff5d5bc468032", + "50|arXiv_______::c93aeb433eb90ed7a86e29be00791b7c"); + merges.forEach(r -> { + assertEquals(ModelConstants.RESULT_RESULT, r.getRelType()); + assertEquals(ModelConstants.DEDUP, r.getSubRelType()); + assertEquals(ModelConstants.MERGES, r.getRelClass()); + assertTrue(dups.contains(r.getTarget())); + }); + + final List mergedIn = pubs + .filter("target == '50|doi_dedup___::d5021b53204e4fdeab6ff5d5bc468032'") + .collectAsList(); + assertEquals(3, mergedIn.size()); + mergedIn.forEach(r -> { + assertEquals(ModelConstants.RESULT_RESULT, r.getRelType()); + assertEquals(ModelConstants.DEDUP, r.getSubRelType()); + assertEquals(ModelConstants.IS_MERGED_IN, r.getRelClass()); + assertTrue(dups.contains(r.getSource())); + }); + + assertEquals(24, pubs.count()); + } + + @Test + @Order(4) + void createDedupRecordTest() throws Exception { + + ArgumentApplicationParser parser = new ArgumentApplicationParser( + classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/createDedupRecord_parameters.json")); + parser + .parseArgument( + new String[] { + "--graphBasePath", testGraphBasePath, + "--actionSetId", testActionSetId, + "--isLookUpUrl", "lookupurl", + "--workingPath", testOutputBasePath + }); + + new SparkCreateDedupRecord(parser, spark).run(isLookUpService); + + final ObjectMapper mapper = new ObjectMapper() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + + final Dataset roots = spark + .read() + .textFile(testOutputBasePath + "/" + testActionSetId + "/publication_deduprecord") + .map( + (MapFunction) value -> mapper.readValue(value, Publication.class), + Encoders.bean(Publication.class)); + + assertEquals(2, roots.count()); + + final Dataset pubs = spark + .read() + .textFile(DedupUtility.createEntityPath(testGraphBasePath, "publication")) + .map( + (MapFunction) value -> mapper.readValue(value, Publication.class), + Encoders.bean(Publication.class)); + + verifyRoot_case_1(roots, pubs); + verifyRoot_case_2(roots, pubs); + } + + private static void verifyRoot_case_1(Dataset roots, Dataset pubs) { + Publication root = roots + .filter("id = '50|doi_dedup___::d5021b53204e4fdeab6ff5d5bc468032'") + .first(); + assertNotNull(root); + + Publication crossref_duplicate = pubs + .filter("id = '50|doi_________::d5021b53204e4fdeab6ff5d5bc468032'") + .collectAsList() + .get(0); + + assertEquals(crossref_duplicate.getJournal().getName(), root.getJournal().getName()); + assertEquals(crossref_duplicate.getJournal().getIssnPrinted(), root.getJournal().getIssnPrinted()); + assertEquals(crossref_duplicate.getPublisher().getValue(), root.getPublisher().getValue()); + + Set rootPids = root + .getPid() + .stream() + .map(StructuredProperty::getValue) + .collect(Collectors.toCollection(HashSet::new)); + Set dupPids = crossref_duplicate + .getPid() + .stream() + .map(StructuredProperty::getValue) + .collect(Collectors.toCollection(HashSet::new)); + + assertFalse(Sets.intersection(rootPids, dupPids).isEmpty()); + assertTrue(rootPids.contains("10.1109/jstqe.2022.3205716")); + + Optional instance_cr = root + .getInstance() + .stream() + .filter(i -> i.getCollectedfrom().getValue().equals("Crossref")) + .findFirst(); + assertTrue(instance_cr.isPresent()); + assertEquals("OPEN", instance_cr.get().getAccessright().getClassid()); + assertEquals("Open Access", instance_cr.get().getAccessright().getClassname()); + assertEquals(OpenAccessRoute.hybrid, instance_cr.get().getAccessright().getOpenAccessRoute()); + assertEquals( + "IEEE Journal of Selected Topics in Quantum Electronics", instance_cr.get().getHostedby().getValue()); + assertEquals("0001", instance_cr.get().getInstancetype().getClassid()); + assertEquals("Article", instance_cr.get().getInstancetype().getClassname()); + } + + private void verifyRoot_case_2(Dataset roots, Dataset pubs) throws JsonProcessingException { + Publication root = roots + .filter("id = '50|doi_dedup___::18aff3b55fb6876466a5d4bd82434885'") + .first(); + assertNotNull(root); + + Publication crossref_duplicate = pubs + .filter("id = '50|doi_________::18aff3b55fb6876466a5d4bd82434885'") + .first(); + + //System.err.println(new ObjectMapper().writeValueAsString(root)); + + assertEquals(crossref_duplicate.getJournal().getName(), root.getJournal().getName()); + assertEquals(crossref_duplicate.getJournal().getIssnOnline(), root.getJournal().getIssnOnline()); + assertEquals(crossref_duplicate.getJournal().getVol(), root.getJournal().getVol()); + + assertEquals(crossref_duplicate.getPublisher().getValue(), root.getPublisher().getValue()); + + Set dups_cf = pubs + .collectAsList() + .stream() + .flatMap(p -> p.getCollectedfrom().stream()) + .map(KeyValue::getValue) + .collect(Collectors.toCollection(HashSet::new)); + + Set root_cf = root + .getCollectedfrom() + .stream() + .map(KeyValue::getValue) + .collect(Collectors.toCollection(HashSet::new)); + + assertTrue(Sets.difference(root_cf, dups_cf).isEmpty()); + } + + @Test + @Order(6) + void updateEntityTest() throws Exception { + + ArgumentApplicationParser parser = new ArgumentApplicationParser( + classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json")); + parser + .parseArgument( + new String[] { + "-i", testGraphBasePath, "-w", testOutputBasePath, "-o", testDedupGraphBasePath + }); + + new SparkUpdateEntity(parser, spark).run(isLookUpService); + + long publications = jsc.textFile(testDedupGraphBasePath + "/publication").count(); + + long mergedPubs = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/publication_mergerel") + .as(Encoders.bean(Relation.class)) + .where("relClass=='merges'") + .javaRDD() + .map(Relation::getTarget) + .distinct() + .count(); + + assertEquals(14, publications); + + long deletedPubs = jsc + .textFile(testDedupGraphBasePath + "/publication") + .filter(this::isDeletedByInference) + .count(); + + assertEquals(mergedPubs, deletedPubs); + } + + @AfterAll + public static void finalCleanUp() throws IOException { + FileUtils.deleteDirectory(new File(testOutputBasePath)); + FileUtils.deleteDirectory(new File(testDedupGraphBasePath)); + } + + public boolean isDeletedByInference(String s) { + return s.contains("\"deletedbyinference\":true"); + } + + private static String classPathResourceAsString(String path) throws IOException { + return IOUtils + .toString( + SparkDedupPublicationTest.class + .getResourceAsStream(path)); + } + +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java index 9c9ec43d5..3de14f577 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupTest.java @@ -4,8 +4,7 @@ package eu.dnetlib.dhp.oa.dedup; 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.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.*; import static org.mockito.Mockito.lenient; import java.io.File; @@ -14,7 +13,11 @@ import java.io.IOException; import java.io.Serializable; import java.net.URISyntaxException; import java.nio.file.Paths; +import java.util.HashSet; import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; @@ -35,10 +38,13 @@ import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.junit.jupiter.MockitoExtension; +import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Sets; import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.*; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import eu.dnetlib.pace.util.MapDocumentUtil; @@ -105,57 +111,27 @@ public class SparkDedupTest implements Serializable { lenient() .when(isLookUpService.getResourceProfileByQuery(Mockito.contains(testActionSetId))) - .thenReturn( - IOUtils - .toString( - SparkDedupTest.class - .getResourceAsStream( - "/eu/dnetlib/dhp/dedup/profiles/mock_orchestrator.xml"))); + .thenReturn(classPathResourceAsString("/eu/dnetlib/dhp/dedup/profiles/mock_orchestrator.xml")); lenient() .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("organization"))) - .thenReturn( - IOUtils - .toString( - SparkDedupTest.class - .getResourceAsStream( - "/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json"))); + .thenReturn(classPathResourceAsString("/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json")); lenient() .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("publication"))) - .thenReturn( - IOUtils - .toString( - SparkDedupTest.class - .getResourceAsStream( - "/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json"))); + .thenReturn(classPathResourceAsString("/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json")); lenient() .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("software"))) - .thenReturn( - IOUtils - .toString( - SparkDedupTest.class - .getResourceAsStream( - "/eu/dnetlib/dhp/dedup/conf/sw.curr.conf.json"))); + .thenReturn(classPathResourceAsString("/eu/dnetlib/dhp/dedup/conf/sw.curr.conf.json")); lenient() .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("dataset"))) - .thenReturn( - IOUtils - .toString( - SparkDedupTest.class - .getResourceAsStream( - "/eu/dnetlib/dhp/dedup/conf/ds.curr.conf.json"))); + .thenReturn(classPathResourceAsString("/eu/dnetlib/dhp/dedup/conf/ds.curr.conf.json")); lenient() .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("otherresearchproduct"))) - .thenReturn( - IOUtils - .toString( - SparkDedupTest.class - .getResourceAsStream( - "/eu/dnetlib/dhp/dedup/conf/orp.curr.conf.json"))); + .thenReturn(classPathResourceAsString("/eu/dnetlib/dhp/dedup/conf/orp.curr.conf.json")); } @Test @@ -163,11 +139,7 @@ public class SparkDedupTest implements Serializable { void createSimRelsTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkCreateSimRels.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json"))); + classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json")); parser .parseArgument( @@ -207,7 +179,7 @@ public class SparkDedupTest implements Serializable { .count(); assertEquals(3076, orgs_simrel); - assertEquals(7040, pubs_simrel); + assertEquals(7046, pubs_simrel); assertEquals(336, sw_simrel); assertEquals(442, ds_simrel); assertEquals(6784, orp_simrel); @@ -223,11 +195,7 @@ public class SparkDedupTest implements Serializable { void whitelistSimRelsTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkWhitelistSimRels.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/whitelistSimRels_parameters.json"))); + classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/whitelistSimRels_parameters.json")); parser .parseArgument( @@ -264,7 +232,7 @@ public class SparkDedupTest implements Serializable { // entities simrels supposed to be equal to the number of previous step (no rels in whitelist) assertEquals(3076, orgs_simrel); - assertEquals(7040, pubs_simrel); + assertEquals(7046, pubs_simrel); assertEquals(442, ds_simrel); assertEquals(6784, orp_simrel); // System.out.println("orgs_simrel = " + orgs_simrel); @@ -306,11 +274,7 @@ public class SparkDedupTest implements Serializable { void cutMergeRelsTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkCreateMergeRels.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json"))); + classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json")); parser .parseArgument( @@ -402,11 +366,7 @@ public class SparkDedupTest implements Serializable { void createMergeRelsTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkCreateMergeRels.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json"))); + classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json")); parser .parseArgument( @@ -427,10 +387,10 @@ public class SparkDedupTest implements Serializable { .read() .load(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel") .count(); - long pubs_mergerel = spark + final Dataset pubs = spark .read() .load(testOutputBasePath + "/" + testActionSetId + "/publication_mergerel") - .count(); + .as(Encoders.bean(Relation.class)); long sw_mergerel = spark .read() .load(testOutputBasePath + "/" + testActionSetId + "/software_mergerel") @@ -445,8 +405,35 @@ public class SparkDedupTest implements Serializable { .load(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_mergerel") .count(); + final List merges = pubs + .filter("source == '50|doi_dedup___::d5021b53204e4fdeab6ff5d5bc468032'") + .collectAsList(); + assertEquals(3, merges.size()); + Set dups = Sets + .newHashSet( + "50|doi_________::3b1d0d8e8f930826665df9d6b82fbb73", + "50|doi_________::d5021b53204e4fdeab6ff5d5bc468032", + "50|arXiv_______::c93aeb433eb90ed7a86e29be00791b7c"); + merges.forEach(r -> { + assertEquals(ModelConstants.RESULT_RESULT, r.getRelType()); + assertEquals(ModelConstants.DEDUP, r.getSubRelType()); + assertEquals(ModelConstants.MERGES, r.getRelClass()); + assertTrue(dups.contains(r.getTarget())); + }); + + final List mergedIn = pubs + .filter("target == '50|doi_dedup___::d5021b53204e4fdeab6ff5d5bc468032'") + .collectAsList(); + assertEquals(3, mergedIn.size()); + mergedIn.forEach(r -> { + assertEquals(ModelConstants.RESULT_RESULT, r.getRelType()); + assertEquals(ModelConstants.DEDUP, r.getSubRelType()); + assertEquals(ModelConstants.IS_MERGED_IN, r.getRelClass()); + assertTrue(dups.contains(r.getSource())); + }); + assertEquals(1268, orgs_mergerel); - assertEquals(1444, pubs_mergerel); + assertEquals(1450, pubs.count()); assertEquals(286, sw_mergerel); assertEquals(472, ds_mergerel); assertEquals(738, orp_mergerel); @@ -463,11 +450,7 @@ public class SparkDedupTest implements Serializable { void createDedupRecordTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkCreateDedupRecord.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/createDedupRecord_parameters.json"))); + classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/createDedupRecord_parameters.json")); parser .parseArgument( new String[] { @@ -483,12 +466,18 @@ public class SparkDedupTest implements Serializable { new SparkCreateDedupRecord(parser, spark).run(isLookUpService); + final ObjectMapper mapper = new ObjectMapper() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + + final Dataset pubs = spark + .read() + .textFile(testOutputBasePath + "/" + testActionSetId + "/publication_deduprecord") + .map( + (MapFunction) value -> mapper.readValue(value, Publication.class), + Encoders.bean(Publication.class)); long orgs_deduprecord = jsc .textFile(testOutputBasePath + "/" + testActionSetId + "/organization_deduprecord") .count(); - long pubs_deduprecord = jsc - .textFile(testOutputBasePath + "/" + testActionSetId + "/publication_deduprecord") - .count(); long sw_deduprecord = jsc .textFile(testOutputBasePath + "/" + testActionSetId + "/software_deduprecord") .count(); @@ -499,11 +488,13 @@ public class SparkDedupTest implements Serializable { .count(); assertEquals(86, orgs_deduprecord); - assertEquals(67, pubs_deduprecord); + assertEquals(68, pubs.count()); assertEquals(49, sw_deduprecord); assertEquals(97, ds_deduprecord); assertEquals(92, orp_deduprecord); + verifyRoot_1(mapper, pubs); + // System.out.println("orgs_deduprecord = " + orgs_deduprecord); // System.out.println("pubs_deduprecord = " + pubs_deduprecord); // System.out.println("sw_deduprecord = " + sw_deduprecord); @@ -511,16 +502,63 @@ public class SparkDedupTest implements Serializable { // System.out.println("orp_deduprecord = " + orp_deduprecord); } + private static void verifyRoot_1(ObjectMapper mapper, Dataset pubs) { + Publication root = pubs + .filter("id = '50|doi_dedup___::d5021b53204e4fdeab6ff5d5bc468032'") + .first(); + assertNotNull(root); + + final Dataset publication = spark + .read() + .textFile(DedupUtility.createEntityPath(testGraphBasePath, "publication")); + + Publication crossref_duplicate = publication + .map( + (MapFunction) value -> mapper.readValue(value, Publication.class), + Encoders.bean(Publication.class)) + .filter("id = '50|doi_________::d5021b53204e4fdeab6ff5d5bc468032'") + .collectAsList() + .get(0); + + assertEquals(crossref_duplicate.getJournal().getName(), root.getJournal().getName()); + assertEquals(crossref_duplicate.getJournal().getIssnPrinted(), root.getJournal().getIssnPrinted()); + assertEquals(crossref_duplicate.getPublisher().getValue(), root.getPublisher().getValue()); + + Set rootPids = root + .getPid() + .stream() + .map(StructuredProperty::getValue) + .collect(Collectors.toCollection(HashSet::new)); + Set dupPids = crossref_duplicate + .getPid() + .stream() + .map(StructuredProperty::getValue) + .collect(Collectors.toCollection(HashSet::new)); + + assertFalse(Sets.intersection(rootPids, dupPids).isEmpty()); + assertTrue(rootPids.contains("10.1109/jstqe.2022.3205716")); + + Optional instance_cr = root + .getInstance() + .stream() + .filter(i -> i.getCollectedfrom().getValue().equals("Crossref")) + .findFirst(); + assertTrue(instance_cr.isPresent()); + assertEquals("OPEN", instance_cr.get().getAccessright().getClassid()); + assertEquals("Open Access", instance_cr.get().getAccessright().getClassname()); + assertEquals(OpenAccessRoute.hybrid, instance_cr.get().getAccessright().getOpenAccessRoute()); + assertEquals( + "IEEE Journal of Selected Topics in Quantum Electronics", instance_cr.get().getHostedby().getValue()); + assertEquals("0001", instance_cr.get().getInstancetype().getClassid()); + assertEquals("Article", instance_cr.get().getInstancetype().getClassname()); + } + @Test @Order(6) void updateEntityTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkUpdateEntity.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json"))); + classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json")); parser .parseArgument( new String[] { @@ -587,7 +625,7 @@ public class SparkDedupTest implements Serializable { .distinct() .count(); - assertEquals(898, publications); + assertEquals(902, publications); assertEquals(839, organizations); assertEquals(100, projects); assertEquals(100, datasource); @@ -640,11 +678,7 @@ public class SparkDedupTest implements Serializable { void propagateRelationTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkPropagateRelation.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/propagateRelation_parameters.json"))); + classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/propagateRelation_parameters.json")); parser .parseArgument( new String[] { @@ -714,4 +748,12 @@ public class SparkDedupTest implements Serializable { public boolean isDeletedByInference(String s) { return s.contains("\"deletedbyinference\":true"); } + + private static String classPathResourceAsString(String path) throws IOException { + return IOUtils + .toString( + SparkDedupTest.class + .getResourceAsStream(path)); + } + } diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/entities/publication/publication.gz b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/entities/publication/publication.gz index cf513d7302bda81e6bdae0dbb430f5852a91307f..a7457063bcbe59481e11c944d4dff7a3244e4831 100644 GIT binary patch delta 24067 zcmXVXWk8h8_qO!XDM)upcPt>?v2=HLO3KpRU4nFnG$>sQNSAbX2ut_N^ZUPF&o!Us z-1p46VvZ`POZ%t+RiFaTm00I3-OfDHvN`7Q3Z zxLP`1*KK~sv>u-;zzg;|o~tm7-;pCRZr!t3I89Oi^#!NF1XRq+rp)a;4h9n!Td4Ep zBwL0N-OTC5v&^F7pu@%;_;kQbGtV~%9<9#u%tA@IT!fVdQPTbB+omcCDC@gdQfibF!Z^^sm-qq!T~u&nM&<)Mz$Kjyg$;w;)CFh(N!5LOd8uCf|GS2)37m5wp+!NtIjQZPd<6sg!Eb>Ziu3LV9=oOh-vy!G==P&1t z{$gfou3t6cv895_QQ%3+l-bkXMhUuMEn%4(gB8z;-tsN`^kKO3DOouz_@GN721|UyV9yds)WC+Af=H06kBVZQ>NPfYmfg=;#;HiqO zv`@cbF|UvksPcf3G%FT>U!IQ+?2#ipfAXkuQx~)B_vzAI=ogyse+^@wk?`Q5{r1O zU>~CUEUwaMi@^SkU6mY|f%M57M}6BgCUk#`lLT>Lesk_b8q|05W~JH2F>a>Qb6jhP zUVFQ>$`SMI2kLXgl9vS(pC@(59Cv|~_4(#i%34hS6h9(}E{ico|JginfhoRB8X9gA zExyF8t8H1BKI5Hl5nC0iqd#|Ca{$e=49!aqyd2E5DT1^AJm!+1`%wAgCz&rHc^L!* z{7L>I3vX)>yPo{FH3nyewz*DsziHd!(tN8ig!YzaOpU%XXjg;|panB5O$*@7S_QLS zYvg@a03q9a7Nxw~;68ujK5oPcU+VV3li6h$^G@Z;lJVvU{|;ZxS5NcJMB>vJXV8UC z<*12HvJ?1yqa*m`_BuS$tZUj9AMwhllX`Ck1qc1z;J;^B?c%35g0B|JE*Zkp#;tdG zVXDeW|7zk><<&$v+^ASpky?3^-Ee2v6U|mQ-=OvL*(M;e=GC-&IW)_0oVm1G+#rO| zrudJ}wr*9{^$AH&=x|1p^KN~$W7j|!%(09$XaqOW^d&ZZnSqsykrr?qhx?+L)3*?x zJ?RLa&njLB8x_q&Vo3!Gk{qPO`j|Zps%bH`t)pk6i-_zBFQ4)VX(4Hr9O9#5>VjIZ zM@p`#so7jo%I zRym@{5~InI()BB)*|~OSo!FI@9i7uexokx8b9iUx?0z@-=g|H%%zCoad*O~UALtE_poTaIVL|a zK~^mlb}kK;)MEiCq^4}l%8cG-p#<17LK1f5MFk(MQo?ei5v z!(}Iqou+UVNDN)Q-W9@ye zo^0?hU80Wce={4?uT7dTBGP~i{PFjjPfu?JQ5U?VqYoILKG7}Mr|OIBfE0R$JbNGM z@h?rJC0vFYxXBMfKH>!HS!R?dg#mTNkzwQks^U}3Wi?s3Dw>Pae+bk{$=Jl_KU5;J z3y^EVIoA^1BQtouaDP*G1m|=#BD@4X*u8w2R?2}OMq@{1-DgH6WZm;vGrzY#9d<67 z_n=R{_%TQF#!IH&V+5y@;|)Z!^Mud{rtDKiE3i*qAU_RC!JO^;F{aN1F3*UE-DMU9bbaOS-(x*@EGD9CD6R-H@&L(|{wq{Zj9M_oVY@cn7_=m$V z#)qkhj0;mx%wKc~Cq)oqe->889F4Zy@eQ^ul# z##GE55=xB|FU!w6W^0cVpIxOkTQJlZy0arC%TKkX6IO&yLz3j;57&2TZEC|KC$n~R z=(I9*&5{^@dy%kM);V>;%qR)s-Cd+esK+8bdocGpSCmLyl)MbD3>PMs)2rGM zwcgevI2t9~78f1A-?rJE7@Tp8{fPC!Z(Pkg`iT1>`Pb7R*ogwWnXguy%C5ESh^+UK z4x*b^=J1L7E_8MFYg<{>Ny?a1lPkO*(e8I%vhQq*qdX*)9hNWsCCzRJaK^=XiN2NM z@fd0#H+O=fnSHgcG)Q)_65opRTXx13x$2QJyKl>-gH{3fpcnZ$1XCbbFe<`1{9b+m|nwH!HvqYEF{} zh6Kbya1=7BlGKhpLXJC#cKckZZ>`Fl97tRSi>Nv9mm#B-?9S&>n!+xi=Yce`iVtkJ zL|N;^L6jLz$Uk@PGj~a{wk6D?+LJVYhe&5b1ahK#hr2q^nE{ii{Ke~p9G4sk97tTEi2CF4175oB5c z#z==Q3Go{ibvy%v=e<=x*K_OMqhD0cqRQB1%6PN9)9KAVX>S#ya(o)dxrz=G4olUF zCa_{TR`Y9*s-q%|y3cboF$6F9W}puS@6|E4CmDN(m^DV`v<}9}^anLZxtSS!Dm_f8brH;jr>5UvH^q!6W2C z`{OA-)aaRbIzCk6rc_qK3v?36=!uRVs6|0TtMe;6o= zq@0N77ur5WcVEs98&Dh%CwO{5pPrS9Y^44_04d|Nc!_k@Nyr0rc#wF3H$J9Suf^`! z&ph+xkQMvlH!~Gn(rP2V0+`H#8F!x$_5ua=0?=bVI?Jn`XVv$o-5UqymJa4LrICkK zQX_X+tZzHN3rPMdc)l@A<@lXLheLd*a9eLY`&ljrNVen3ZK$-nL>kh>zg;Axe2%Km zI2i(@qS~xhnxGNMIfyc0j~v=b%~(vFWhZN3P%y27(Ow%e`%7mi0wx;VWgfj^#-&t; z(34>xJQzGnK`Hu3UZp8zKm#K_{uz+THCal_qpLz*BB0b=wf?guQ<)2FYgKa+@n1r| zz!5zN4tE&sVMWW9+9WsHq-p7jx{u1DUrQNrM;vqS>reRXaBeZf?9Jfbuf9FeMPDjc z#G{Ts6E`4!%o${g1wVg)!a*>($8GJQaN$Z7=IUp<0h^apyGV)T<-w9f|AD^|M5hc$ zMdk%7XItoq9v3+fZh13de_4Y1mopjfeLcQ@m9NeJ`s+w7JaCyV8JKCPI%MVm~62t9o->Ays<{{4BQLH&KE!9dpG$g-Ub zh=#vic^mi)N(MN|lPOV%BrJcE+E$@T(4;X&x+}j7XyLHzkD-fZYslK#bd3}DB85Egc0+B_v}-a}{JAR< zXQupf^1VOFpnmrsdvNi|xgB#9$t<*&k+OH=>hIg6$biVI>KwUr7uo;B!HR@%?`VBq_7 z0FL;JcPe$NLp(@kT%E;+c~}f(S_Q+Gc|EOBX>xykZ;bM|cQ(g4pEM<$pk>eM=Mz>S zQv-X9r67sa&bWr_Q|{)XkXw^+Qb;7j%`xX4F)(wRB4Pnu^tfmOmcTl428cGA>E(uxL9Yz$i&Ed_AV2{D5*pE ziD-bhvDnxcy4izEZ8{l`gNp#dU;DWU*I-<5O3GcO#c>ss3gTma@y7P>_2NG zqso?w3jaou>ap2mfqNQF!V^5?1u%@AY z>r3J1#4l^qu$t29ko<*RmA5M6HSYXYZ7hb-nCB7jjeA(n23w-R1KO{7!)bB!bTA$> zWAU%QecNAsJ|vY$g2+iMuz|AzZu(E5$1a^V?C?BwQ0qV2=ETs z10cyjQkg1mHSV*4YL+0#O{`QOtTfSpk&V`%VG*Lmf&@V{{hAKtB))d&si@wMu-g7R zssh31iDMC@lGvi>K>hd0*=MkVDK8SPSDh?5^v*;}X^1qr&%Nh6yGke(G44e+roD!| z4jxuyf{AN<^j5sZl+#S>=wmcjr`ve4qBOD*38rkWY-=no?6CpeO*i+~_}f&{d&F)PVCeB>%~q|Cy+1fSxx~q270G>!L*1a&YT{;F9J$ zP4I?t_je@(CZ-A}MMHyBqJ-Wb&)TK%X^02^Bx{8oYLB>Dj?3?eQitY#h@)^_oqhXA zW!^~2a^;-Pr+r}YFk_Tu32vTRCL9}+aH~A~*ld--r$hboWO}#Q=FBW-+q7#?pVw?U znzbfrk>_W5V$}Bs{3<0D=VEE9MaMk)7y(tboZ6($o3w~R&)x>K*_^~f@xm%wW&PIW zAC`jgH)``4M^X?+%GR)X#H1Bpd#96wex@6HOgrBdtu+=Gqx~v@I&fS?ElSMIfT<>+ zI%K~t@ps=$FzOWnnV1F(Y1>zzn6ZG(G4e}^DnhCO%D>N;~ zlJi)Cd$Y)yXqPyuD4aPc@PjWEGhfYN-cXx$Pjow4Sj5LIN-z(pI#>CLU9Fx*u@S2{ zFj+i>Y3wT+2iw*5E_T+EY>c-@fUm!}mE$;SXZ+KJo@P$Khm@0;`g1zuw(ckKrHY=Z7b{CodpwCYLN@$*z9MiqovhY|SxnOUM!K=`cGpc&E> z{_0}(i75TIn63SE7H*Bhs8*!aQvLGIo8Jw*ucy0z2P8f%e`f2H)LFNss@Y*>VeF{3iwau0DLu1o&p46B*R{D3369uG9G92dq13kc&%PE^D-{e>%v>FyDZ+Ylxbx0MWyq zXsTV1u8M(ikC17DLvuLzGYB)^Vw|t)Tu4_CS(9tCvLciH_smo$u{+z&u7#)q zVipCSQ7~5rA9i5gt{iZinQ9S1T&u#%%^-GOjo6L=&P$9`FST6ShhbF0F*QO2Rwk~} zv7xce*YW7koS_0(msm*}U#pQmf!6W%Zx2K-*)e6AG8vG(aPjcC@8bMj8S6=)DQo0o zoPy^rI0mq9sS_lhwmYUbKyQiDjr)BfM7AL}4E%j!R~0n3f3B0`M^plhqf54?Q<2od zLGqcNwt!IT#7YV(j)a(4Am=BK(hy?z)F)yR0QKbd#5TF~(E2#3RNxjF6h__@9Nfmb z3rD@InT*W~wByP9d2uq5yIF>Rp`<4)6(QS2;Am`cq8YdH^f&qbDI}YB$S3*}JgCFc z0p9*XTmDpuJ%bsqxuK$BdsrcSiC048YsUt(R10~h@&lT5zIH0Wu!G$>R#`9BA?9T7 zA8MdwhXB2axmo-O z{=k(}X_QR4yTU0Yp-@&!dnL06_DQPay8ga0v7+-vH;V{wK^uG2YIP9rI%9m1qQ*(r zV>0ndYywFjKU~03X<}GIX|{#o8~CgHt2efr2;@NnTP!*k(@g|v=pmETuCO%`@Wloa*dBa;0EmBa7`v{F!L$PSUb(dK>RVSSe>* zj?dExfboaFsJf32`yc`fzIXnx`SyHfVaUI6cIdk@Zd88D71RPNpZ%{ORPBK%T2G#8^Uh9{ChW=|7{T{2h{*1q!->)|dNW-!MePHDJ}$4MM`*+9~oLN-eO zhjqvdl-#?@-^V(1(Md$cgpSZeP1we|>wHkakZUy9XD&3)-pSNhl^WE<0z~A}CMoiT zK{DtI!G|m!n&t*!xcYG4)v5(?geCI1hv>K$3DW-QnmuF-B#E<1rkk|<@EX%l z5U?{{EG{o`g%j`2O15q1N5I@}?Vy>$b4?N^6$V@ulS7ghte-P$%x$r%kV4MazYcT9 zde{DRkwyhCyjfB`Nn9$Vl>U9vTTzQ!L8zD4C6ecB}+b?k(!VGc;m-u5g?YHdz z<6+Z2bhbY~?(9W8k5WNJlxO{O9j5|y`@yaONFe}C`EoV!G_`kU&q)~U)6c6IgD;Zv z-+Yhs(`I;N?N=HF{b-tMt%hZzW9sJugQNOg`MIM*tCiA6{9SpX<22Q}x!+E@!u>@? z$T#w%_M;=(eG_S7rE00!`9@KCrB=3C5H`MKhD@qywdp1_7}`zBJiOtqi7jUxMJv=0 zxT?`2v{J&A%mYhyB2Tc=ke?P7dkk@{lW%q%m(t$cj?wf-hL9!d(L6E=%kKQV8Sd!3 zWlWVVp=Q23xBs@B?)H~)t%_5C=$qSP&}u-t&)eSfF7zIrbdCh+TW3eR@B?4FaL2QE ze7tR@xpIO^$6?C;{q8PwcYi0p=KN;l@@zZ<)kDeMz&+qb27Ge63#*kwtz)AMc{mz8 zVi|=Tk3vL#I&ZjXYJX?I!Ol<8Tx;4>A&wdm`Sx@hLR=yj7FcjYtHKj%o=?GGxkXpz z)5(BJ-EGhUk`Gg3N>zK8+D`Iz_Od&u8{<-Hs^7XE-Q0aVlxqX%>qRN%UwUuKM1PeHubDy8 zaX#A-pugK3{fx)WilJIoW67eV)eEC8HFC~BR_y`=zO;WFtx3(;)5;n6!BAnBTzuSA ze$S0iSX_D>8-Y2s|1Oi4k-BAJ`oMOhkGSUd5)pqa0^?h1pfg?;3!$!kO&jqwpu86{ z)y;EJd)`ZzU_;Q<_T#=hRrM?ZCFoTSt$pzlu6@cT^bV>zs&poh_^7GfDGSf#UR&xr) z1ojtVaL=`<{ejM%17cZ`t>fDA&=T5hm6)Gh=GQS7bVu{4S7z)itdO8`lPi@Te}uF$ zq_`1kif*%QnJ)p@iT_v@R?k`3+P|>}`qxl+ur5n}>5>b4&Zi_`JbittNBVHk5II1N zEfu!!l53f34e0*I6hk=eDY=*vT6+>+s0y;M`bnFuv0un7;WYv8ZH6 z-O;7)@e*vAMF9pR(0c)DsGGV-ZC;KacilKtKgppWncgVL9E=@%>GK(cbWHXhs}N9D z$JhIQG<3z*lF@m&F!Ag^wy|3AX#Ek^dUVo!(P7(MUi;=}uJgjH0*&<0#T&y|`v3EQ7_=B7S3WMOfvk{VUF02p(x4gL~BWB#ER_SySILf0ER@10aV%0~TQS z`XCn%_xVynsB)>mS6_RLWr|3UEH1oB>Z$`l9qu-;#m3F9hpv}KjX0F{eC@!x69E};2G1nivy(JN#uJBV|Ajb zLyaJDi*0@FQ==ect(#)_>XFcavE(T7s0^uX%UPKvYctCb8=R;#I_%HnJ_E>a9P$<# zI5O~gRrKO{>7mhWaK;J~U`q(4QIsS~4ALqls?4|2PUoK=9;wn(Omdsepq&_@J;%_6PdMd{Bxb2}hnUxkdmb8`XRBLFrxw$!K~ z-+4?O17Z1R!o%okhP)Is|L_`c0q&YC5Bq;i76WHc7SBAacc0(EFj=P+Ek*6bN5|P0 zJ9$7BN4p&sRn81d6SWaY^ZGzxr3B6FAYF5?k8eSM-rvaF<9!d8^MRoMv+9y@YU7ae ziL0S)6AO&`?V94fGnr7|zzov{J0a;tK=Kg#1V^@8gumJa(4}f{t)ijSr;Lc1iOs+X z_%kJu?uE(AI@$P}j?b6qOtmsI0Hq`e9GyFi@M6*g{BI!Sm>I zJUUYW5g8aiIm=YnC)>h@7poswL-Cpx%dVIAz#*(~_`?)jc@p_tr-O*~;EV>{IIaG9 zW3D3kmJ|94jvj>bgMV2PM%_|6ASAqa^`_`ahkNwYx^-fm$uzxL z7rm@P6=P0&2nb=aN^B(UN&Rp zaB+t5gyHcfl8O1kA0=4{I12kpE_y1>vm66krM|sOz1YPvm2H_J*MJ%7h%X6;Rx(A7 zD@(9KX1}vZr#&Q`v$7TVtYkzohAbZ;Bbd5{vPO&wWS+W^s#6HAfY}TaA5^*P8A&NI z6HEl9{=5LQD}(=l-355N+$-Pcy`XM3n`z&~eMqvpq~4IjPr<{-+Kwr%1|u28Zf|9x z72f-azP^&nMp94)heAXx7ok8(vm=obql90=`yF}2?2`xSXZphZtvoc6x9C=j6+uXN z^FvEj#MN_;J79CqGp;9qN7;LQ|F!k`(N-KF6Ug(FaPDd^PA zc`$F@W^4AIVLoLNk4rb=)|DfFMdPeRC`k>xL{rXNm(FK z6y++c1Wkap?DJ&ZW;xJI0_kR7>__v(hm=Wtq+9^aZMz%ix5t-1#JpJ_dnUP|;^I_Tkz3qpj9xS+^QXeU&fGc3q zKV|0d3;ZB`GM!surnuvLd;F*f|I*W=IINJ~ecevn7?iBPV!~L*LGSv<+rE~D1!gIhp+jzd!S8#a%hm!DN1;!a@xI|dJwcXxrgOGSnI?$LK()YtJit( z_9kieXz<}!or7pYbSwzXb$s1XcM8ZZYK2fc-O_rw6~!;C`Wk>+#rI<@GO{RrXMWr@ zr&vy2igpjR?@@+IH*I%jBAq`x4*FbZp(}ToPQxi9w*%_gzAkxoE)8;-n9xED6u}aZ z@|HvvO9WWiHeMZ(s-3L`{V*%s2k~_T8W|hWZ&sC&DkS2e-oKjKr^Yq9qxwTVdj~Hs zTCFP5F0I2ZZPEctW^YMyHi31(M0H@Ectn#9@uOO{SSQM{rfS{H`@U3Zb7&g2mcIbg z*xy^qRIXhC&EWkHgd+N;~U1?Pk?fl(ShGMMOe^)kjBExq)+DM zSnlAn$I-wAIyT^s9!;lX)SoD$L64%QYDiW1gziN2s;$OXZYK5-wNfXWGwf7#u#X$O zF~uEX`=@oS2SloX?vOboymA*^`{m;pnJqO#^HFaS_0tiPZq3dVaNdMSAygvW%9Z<% zspdd%OPmKrN3j4a#I;SiXy%inGNnm zKCG0Ia<%8xYYZ{2v3wD{6x}x7r#^c(6(ut0M`*Tz1k}u;~{v2f=6JwlzW_xJ_wfb9NTxm z@-1vDm)kC!6#&G=dc)`z?*<_!-y!MHS$n}2rpU1cP;U1)}ZJoCq-zZi-&V8Nh;bj?xHJooFWdH65j{NnH_`FsEmc~$)0M@*pn>M3rjjii-;Is4u6h2pNJ4|iDgWoV zZ@SZ&uWsfEw|GzlWOy2^#4k3PvkNp>4%%FXae9+DfcLCNP$0bcM^HoJmoKneiOBFu z|I_(aS=a%HoB%L{g1b&%Yw`|4!ORKB5g8RtqAVXucNe4KgjX^HC>K(Y)ze^6jltu< z$ih=#PYqO0xz^O$4^7}Ht4wj)!zkNS$h++fs;yXM=T4fvuT>;9rvGWY+V*F zN8q5WcHj&(4$4+aF)|dZ-?7JRL^hoslT$a?Ll%t>>DO-LYX9&~Av_tsJH>g7v($%f z!T~suKS&iOG?vU`+X2k5fAzR5biNMOmk({PUVh?CY#XR=B{5@!RGd!#ZcYVB zBAUChH6gAos|qLmt9X~BzQm_26*5lZ#sv0^A&L{u6lW4@lvoWOI3|d=2xzA^x|6ApLau#-ZNLLy`8BRryJVO0fR*yMF^iUJ)E>UgC>E5J@)n2 zh;}Y)|HeLVZS}%`3G2`TKtkj*}z;s zkM{DJ%J(0*_)XV{qTUN}YC*|| zTQGZxH_mZt;k?S$zEc0ILZe6XVTscHiNfo5e^%nek>bS{SEgq(gzxpsO`jT3f8Qby z<2t$x2trpN&N&e0?jGkWsoQWidPfX;KDUr^+=qX3mm6hCpv)?oXO!nL%{G3iaCZ&e zc|i6p79eYX@o;l@z1f-^D~g#l_tz%c&DyBzG6>-B5bo@dY~_Zqlx1`quNmM4h_!ML zv#g1-u_`+`ze?tFkI=8Jq@Z5Qx&xuZ^W%g}@Fz_0&f8#i6RyXel+E~>a)7!7xz4h> ztgWRH`qz;Z!k_C$#t(p>K3^k0!O&l9Dy7_StmmwEVTxY`dkp>R6SpE$tXUYyN z0Nfq+yL*Y|B}|DVTJr|^Y#j+^evZmSdiAR_Blb^>^IF}}?7O>nX|c-R zBlIhipAAU*LAIN|Qg7{^Wbh7*7EzT=p9_CwEf%a0Y}rWZ(o}Y>YM820;&L%eNe1m} z!DnkP(r2I=y76pdM{{i`=T@A=+v)RqmRycVfqN6aV#R#TUG4M>zhRV=E1xx3KSN@e zbp@40364ul*=E~CbU2O|)GgW)Jyv;UXPdUOEb{Oz+NSp@;^Sm)BD^WD6Hqs$+H?$d zsLFGriYjtvH{EZU%x9HUqTOUE&;`s{^GfHpHwO9|I_@|8xO#wGhwx_ZWD@J`shl zHf|IOUn}kCM5und8CI=oF>h8J`^h`>C^|?LW6FnpfdZ+^`uyd!f9M{2+n8g6dLTgt z4R2Vbj$D#Tb%W-|t1XXB*zbJgnSB35SF%2Z@q!y5F~BFh38Kc@LjAIHkb() z-rY<+9S7LCPrb6~2zSSv)&)@z^!?)^qk5Mi1^bTw3kH_ND~+?G*J#Le`D6DT1FE!5 z@uUC4B0f}|=*heiq?UGP8%93uoFdaA6+kV4&pu zM-9NV!FY8y_`Eb1rI=mNmyUq9ccLq7KNliJdvk9{7pTC<5-top|lv&4OT!FN-%>(2BBcI)C!(1Eq0kSY2mWB1}=XZ~eG& z&oEl8!*`H`=WnuX+d7U^Q$WV828+cV5p6GBTeox$^sgfII2KRGjQ`8XH%zPhZ86Gw zUe6hMkIrcngELNM##({LsJCbOwg1bTRTRd_P%)8oU|$+!vf&w+UTRW?wRazk5k|9< zYY=x@N~G7PS<5a$$UiGG?T);M+YC8kt&`{Dag(g5%*Jx6L2W<(?*eBr8c&ak)mzJD zbFP;DIrLeVw1JyrV}dY0w-xGbj!Db<8x%AI?#mWOA--bKB-UO35~mkyZPpgFMKq89 z3mAW{h9~%A3MLHWzg|9sgiC1JrFtxk5fuDa)AUu4D1bvDT^B#Rz^%W{PDZNq()i_= zGju|WsV_^T!O~cwr}eYuuk2;wjX-rB^Jf1~=5vo;s;|Q_aDoYifUkAC!TsU%9bk2c zAK$#9pL3~1l{yO7tj8_tNc5&`%_hmZC~EIHcNyq%Ny?g zeWpf#g}n~Tk5629S*N%@Wd$Cl&cHpA+%h76ssj3L7<0ysVh}5#{3n!eGbH)NVQ6BZ zVI9I-->#M1{Zs0Zq6n|hzR!@9)kj{^+)9pe!KVLc>@&T@D99k%EuozJ6a$9;k3=CC z^gj*kz{puF4kHVe2}}^8h(^h!T1Rby!a?pK2JYOtF8(-?LV5V#63%U7{myaPVvDI# zb15E6I0AwIKPIW-NAEgD@9NZl{*o=*82_H^mRc>?V&DVvdv0E}PG1?{{*7nrXteJS z_e6?6K{ER4E&n84c^4%X22QowUwQg-uG90otGvZ~JeZQ&@RP7Eco1$8t6MZ%3sT+t z;Ky%HonHTxB@`$ShA@^>OBt50^QT~M28*c+bQu>hz&C7qLWc8CFWnyKCHrjm^;7dk z`OA>|Ht6Y_i_|&V^7IXFbE}K;?}tbC?Y-UAtg&f%~l^#Pm60lqnAL}1F2jxQrsrPI_2^WEZMeAn$|-r` zsdzEOQ==3IJ&Oib2QN`TKCu3Wfv9X^f5%suztpV4u|;h+0A+>GnyMP|gQ(|anp**C zsgd0Q{5;w)@yWaUlVAy;bWJ4w02V~teQZntU zte?FUbcXmCVY?PslQKZ^&u!de?>5_qI{QldP7m*cYbwc0?7?B&P3%EYemrA$!SIDr z4^)|XhXNN=9>pq4o7;Vc*R)SAE*Jy%t_W>)J+~jH)&*^*0>HjNOz~Uq56@$X-<&y+ zyIX-sZ-JF{)#T<>OdB*BUBn3k>?$J^7!l0DaiT{F=c-ArbS&gTc>xbw9}Iz?{M|MzxM!$SDfeLwL}hLXnV&=&j()7o@`LH;dHQAL~17 zbG96YYXOVIP;hJqQ}Z0A&UZ_4#Bq#@#5{cdJeBc0F{SmLRHC49mUB0i#YB`PKBir) zszz@9gJE2b&}I;jq)0n^=c6%A09ixm4`-r-93!-XFVdCMlpUpnwLv4*8CO{DKZ$o! zQh-r1B)kwtAo6Kx7X9uyvEdvEL4kGlan@DQ7-kLI8W`8j-_!M*o1d?ruV>V&Z9I5A z->7+=+*3uQN%{RPR2}Q9n2Igg&1l;56 z<_>*2+8Y@^k|Q9UB^mZ`c8+&a5h>Ir+Dn@q8>_05`Gm(e_IXkJpltZQlzWt(&$uI> z9j?R)5j^Ubeh*}hY&epkmxBj0iY_sbTAG0noYj>XracA|2{`x(5+!B?D)N^ zzB(1T8HHJfG5^i!4zb%ABd!AVo|uC~oE*n^P&^D9nXz9@dhd}q@B&U?hx!%ozV{|7 zYmp5Y?m&*C4a0zX{mfKZZ^#g`?Tz4%SC&~L-cg&$wO1va=< zBZTWp=5#XqrAt{Bx3qQ3MpuZaj|U*MtX}2!A@$^*PTQD8T%7bzy5e#0yA<%8k_R=i8?-J z##`EFJL%aaWkf%)KBfLMQ$5>^tbe)21DEhh#XFQ~F4yX&rf}&zO8$m%)ChSI47rFh zh+<&1>*++;yAhZ`;t&{%ppKkRq!14VjQ#n~<4%&S+!rL8uK-a-<+THlsc0fAWebAf zv2i5d{Da@gV3GN;qjsawB$jR+eI)mqZF{M?vmzF0++X3UdhzQhyh=6kWifZKI0R6W zd60zN_vkjdUp=MxM}WY9>SdPZ@jy^{J2uRf%*7^xmU@DV(?pI=mjYHnHuYy}lt$i3 zaPjzrEV23JH?YCEoI&57K1K>&vGc7B>k{d=op!4Pq1qvy**o+#3=*ZXAi`@MHuj{h zB|f#D)N6D%*c<+!m`i@bh|?8ttHHcx8NXLa!{8%4X2-V6J&}f(?XU74CiKt6+lpF$elqpaDH{TpesaD;zR(iLsCF&V7sq=xQy5rXcE+QXnBx7K#N^V#R;riv^R0 z43~$lin>pDh4*j3e?TQ5JxURQ^P_XG#&hVn_jobBomcrnOY9;!IL=&ms7fv-u00w> z>1S%mmv~r#=s`jvehzppsv;9WhPn-ZSS+1#`WYiT z0>Vo?A4Dco=?(*Pgn1y%kSeX5{Qlx~%|GjzmX%)NI#$mgt1NS4Xn3-OF+=Jn=Vj6| z`aUm-nN_lKc+#a2&a+$}__OQuBFt|VqPU|l#{Eg*w*KjzClWdE+6r5l2o>rF?pxg^hRsSzrzE?~be*&(QW()~xxlNjKFL;=X(QQ+Jsa7=wQk7@u9&) z7DnaH{}&Y<>f$WaQjmN0<3ULBC#zun4wdM%ZDZ0G$aNyuS>4Ak=Q=lil)dIMs{!Xr z;id8n-T7z->CUFnogd>Eo_;=mXKvY|qLA-A^y49a`sjZl)rnN+p}eF!qB?b|Iq%L- zj;Nyv=UpHLa=`l;EF1G zGwzl-`zqp)k!uDIk#Sk&k*Yo$4dMq`mXRV`f@}%0B@J`hnrw+}I-2T#QfoC^vOLs6 zo&zPl7zjA@C%+E_o8ubYIVt*!t_wL4rxS1+((MH{?lvfr*{8$Tx$iR;j-}}ChN1^iyZBk)PD6Nnt2hEw3TT7%&mMjZMnJiK!B3pPE zu{?M6VA8`@N1e(u^hq^;lCAuNWJ=S>lygQrxM>k*+-D@t zZ?L@%-bC1br?CAQtf#}`*kwgJu?!PTn(EBdX1O43;-jBfq7vrI3#w7FxL+dpx#gYRYUg64Nc80Yw9TM!TU9zU3I>qG3VvRo(wAF ze4)7yu*YQ4Of|-T%N1?8FSj+ZrjX0U@hwrFxu-f=b>5j;V~9`eDdgmIp|Ou~7|ajA zRsNziriiJ%Wez)w@@)RWj#jl*r<~=)o4)`QiUC>{+id$Jm{YmG^Q zXM~0A#aD{;C3aS4UZpkWh>1DON<*F%dDiARZ9Se<*Db?;wyftNtfz~RT}5`a3Zpy7 zt|Gg72<)n5=@s#{uM5E zgQB2jBGQ3>OezaF%lfj0CfF)|@jaH`eHZX)@gsZ_=Na)?5(xNJ#=ICdT?BRxoTa=& z7#ZdvEa!1QPBLgA55`EO?u(sRj%b@hXDW;t5!@$`g zcg-!$aRSYC0>kvFt8vS;UtAzOv$W!pLe20*6;UOxY|15`i zBKBhlD^ghXQglpn90PwpBIueXx&}TShqBKDPM8nvqi-7Ho8q;B=Zt&8OGKEWOscL- z>YVj6F(+TSE>Ha2NHI0Frx0mnp4mT5;{l8ASX!&O8j6TVgUj2iHnY5q8bXNMIC&_# zXnpH{H8c=sFd`GN>~PZ4;V_=_%$Z2i6~dSrLWqe$g_LGc41r<@6hk!2Y3qt1G)<$k z`8;z;=gXiL6h)vY0!0xhig*;x(pz*B9xl3N$-|@{Q15_x zhw26y4;Nh{dwf~=s5~c*s05^ypHM2%G^xaYb)4}i6J^oePZB%odD|KxY!C7g8OL48@{1l15&nfFlM+Fgkxbobq{)4fos<~T>FkJyJX zvpanR_o(GEPEDKknC+U59Rv<@w4TM(Lg|b8h<(yWP>N?+*5<5aR{k6@a+MAy#~5yZ zoabLKBNIFGRLr%AX43pmVRmC1lUCB2LP{I+EuHg+ftX|C9(YJrr485MuS%1WAe$JP zZ}VoBkGqLIv4WAS!Drn_F0M4OK3`UY#QLv(yn*2wDgOVNFCh?*`>>mqaf=SkJO62)wVD)DIMR&tPvQ z;DhJz_f4N@Px19lf<;8qn}nnngZhJoxcAO*@1I(Y6G8CiPZK)>!8KKXb*ROtr>VB% zSXz(iW?<-UVEc3d1V?bS9=LiU??3Vw_hm?=RL`6%=dFzp))}h`DQx{cr?tM#lK~sf z9Q$<5vt)Cf{luU`N;C8A&2J<&D-P9rX}$5DeAx)1|Kx_KOpzJ2?_ zjp7S(9msVc*U>Pit;cn!s%2V+O3ez_F<&ERRfvVe2NEAuSjK_Ek|=Y zrrEY!XA5+P{c5^@L(a`DIXQ34dz?nHIze%xEF_;`N{^ENh*201nLk!YS0v;{$XF>L z#|irmHUg!vE)CsCn?uRX$e*sV4E`!2C&Q6Q_`gze480Z)Q_|(h18fHsLS5Br6~!G6 zo}$@Ne!{Nn@*LFr4$ci9;r? z+7F+KJ+0)#_cEMJR$Yd{0V1w)Q_fe%hW8tXSjjuP!3B2rqzd^lnv>)(x^fzrjY+a{tNkkuVy)IJ^o9x9gCVewVneao-RWE3;C}q zR%#*th5XlB@Lx^KT2q~l=C&Qncs&Q^YOg=z-7w9TW4i9fWZ35dflPS>8|5I*IBAhT zAhP6e1BpK%-Pnf>^teKq$(Ma8ZKfe_hP>J0T5zDe88TsOGGQA8pvZ1DiQVcx;%20O z0@g&+8aD?|Phriz!*~UERI`t&?z`LL_wU=9s=966!342=7&SX&33bizEN*II@I$#}ZPi9~>REpPtgtQ+whmN;UMEbyl~$}3W~y*LZpem=kl0;Pr`Vn6_zKAoVS-ZRU@St>_J0QEAy=#i7bl; zEDXg=tG7H^@o+WHh^*KwG$M=7SZ!)~Gj)!OWa&U`VA_1ybe9ct9-^oPMJ*_5X_(X2 zJ%iG*HB+aw!YRzvWnO^MP}zdYmMWV`QQ3mZmP1gsm}*CN+P1z0VTyo<>m>G=C*ZY*MV&Bk`8XA+ETs%j=y8(pP!zw<#suyp z5WS)UGQmP}$0JVok5M8cr#K>MmIs2T4fVx;g3e})!Cj8UILL=|P zcql>ttlH4^id5kmB+a8egtLX7Wg%7aF;k841_0E&KK@S9R41r{64Z z(|Wm0Fjn|2EGzn4EcWMr%Y(yyxtW`?NYLP^?(o?{Luc%s$rPSygC{F`(|NhcsX2w5 zEHadd7^G<#6g)caZh2Gg%S}zqf$u&d<%X0SQf>`%+M0Egb<1@ugX+)W7Ec!;+lFjg z6-3pMZ9}&05ZE?d?U=eev~LS^+v`t1rLL~EEKS{kt&{KQj zDV+WG!&5qDFxQvFTutN-X*OgtNNHxiy{|%KFl?*sU4@}o-dqj6%*395LJBMMt#vaNh&-L?QkJY} zt%hi3Voxa!onBOW#G#n;-f%@n){xCeYlYi}o1|&i(8%13$I&Ye?XDq@`JBUBjLH#I zj-Ya+VNP39Iigt*BipoK24B7il_RJesd9$~Do0Q`^3PX}(2i($DUE$Bu} z!)@7u{1Sok63oz6ULilJhO2XNDaucpBtJQZfCjphCq*iD zk%E1Gc|@*8FagMpma3BvB7yameBfcmT3vBBWCR?4%>_&n)1>Oz8RF48Oa3h$rZei3 zHo@JLad z5qXk%$h;LpV56<|iYab_L(bg6mnh)!$Rk)jm-p*>v)w8H_B)SSm;8G)0~LN?y>Sq1 zjJ`g98CA%{)I7>|6MH<#<} zNNHrDtvtzCFjKyh zAL5~{X{ys!EggdM|9>&eugqXyx!ljNuyxFT9;8ICn`8qxd^did)Es#Um@l4HK9i@~~_z9dRBgaLsT8vuy+oE5&w$!^6t^9kM8gZpd=j z?cBA=Rga9A>?ip9L@F0y_DeG&(&AyKg*@Qz%1)9u&42WTC$i$np?5geaxQpv5%zS}=1%WCV{F5Lm)^`v1U9 z(H zu^dJj`5q^Ol=RsHmxV~amwL?{x=j8%eY#)_qgO7qu(D42L*BxOWzcOAJiH%&-$lia ztmtwvgsHRYepw+UlzYCZPs=OhTR>pViKSSybXIW3TRnK#9EPNv!;9h8CPzWJ;bUPq zR;IU5+%&9XG4VK$J22CsJSpG6ndg)!0E_}TGGKZpY&4o`I^`?Iia!nckBakV*<+Hi zbkHWJMO?+CtCgaoaS-UbxyfA~4@}e9baQYhl3QrhAPb4Ii>k28(H~OuAc<>0K(y%hu4VmAE z?j|AlHJZNjN2iZonJ~?jbCKXd?&PEj-}S^DJORvHr!ZMVfEN@^K!3r4l}^S_fj)gM z@h$njba{yzliPSSnZN!shf$Lc$;o3{Q(DFXm3>?jcRet0$c-F0msJ;DPhHrZyo8tK zEO0rQ1tz1nI~l!ywIYK_VDMYnEz6JmI;8DT@ylm7!^@5c`v9>V%=6@`(7=$>K-K~) z{&?w=ZG3RpiV(2^J}(82wU+9%sClcZ9mDPz^#4}jzoJ*B@@;si3JsxqG*APFJ3Uud z9nG>VGw8X2iSqCNEh~C|g{w9F)oT_A^e1vf&X7zb zynHI?I>Yn=jajBj-5xxjWgC>)rm4H8@Axz1%9m&m$q4BNR?AdmaWY=ud>^lsC{x;b z_s*A*8>}p_CdfswC{sQ8gY=y@erL4RcOk!HAp}8ufmTfy>RUgvA;1@P-(S=@@V)|d z%h&Z8Hgwg0@OHWGanp4y)6;zF>$(}}TEMv0+I#pfkK#cpArt&(|_ z45F8I=u0pCdjcBFW(qwMFBH*9HhD{C6Vj@`Ssg7DZM+aH6xYmgBXYo4t?AJ{EAOuq z94_8n<@2|>H|OPhn;A+Y7HlgCFZ^RPaQvWGF==ZB`ha68pmJ)KS?AlXb$bcxt=WF!y zT^FN&5dzPD?(-Bp>)euXu)C8ftGd-J{Z~h{)s15#w@pPk{rfcgAKsSBZClq=)0P=s zQz^z=ZGjg^>#OUN#jVSggiW_BOIc%=P=EwWLdnYKZxc(pt?7xmEAMTpk_cKODHB{C zPR{=R4enJhq>gmz^x8qC>QD(#xt6Y~p2aMGt_PZSY=J@o(Tv>!!TrDtFjd~`GYJ(fSfHTkP z^~}KZe8X~7y|}~uk1GqTy;vONV*UiZlamPIO=PBhYdnEKl^KmStF~su`+g7jlM*=AhEFC|%jWJdT^Ul2A`f#G|;$M3n`G7ML5xrSyfj*mh` zM;;Y=wN!gksUD7sJsuSrj0(ePxvIXb7zxh)`!f5__;LR2o+wsHMHO3HD?1g)zLZYi z7=B)+Ho?ZI*i3dh%2@(W@}UNkMi~${@ek7)l1-v z83upsk zqhs0~O@TcWwNzUQZyab*WSzhXeS3)7_BNmd9U**I9_OsJo zn#nJ3bR6rFX-k=cab657QB5#THu01U_`jFUJ!{W~|47~GAFtD$zQh}C62&G@vb|kK zuq(Ns<-zP*)_~r9M$__r<-U9G4NG{hRznqOsq>Hi`02Ldym?{b5`ry%{y*(qOK*Ze z5dJG%yeY!&@>p)Brin?@8rz;5U?G~c($s@ZO#gdl7qF~=f{(_>CZ3Gh!DWX1X1ItkY1|2o?_~!!KjY_OM`(o77BsF zX?kminf^rHse|x~;d&T}^avL2y=%9P}MBm(z^t9~~tiV^WzBu&x zQ#vod7u(V}dVJ^eV5|_Sf2i6MTi}VU=PP=}J*#5=8gW)E@xRhT$c_m`C?=L=yQXdW zZpdQdGlVFOZHBOfdDmB=MLmJ2v_U3kFOyEL)X1g;%cNaU)t`UGk9zCyz{ Wyp`9}P;sD;*Uv8sZ#9~4YJ>o6NS#9f delta 20910 zcmXt9WmFtNwjSKw-F?vD?he6&yK4v@q;dBU+#$FV+@0Vq0Rq9@T_4%q_p46ds_r>c z-F54eTSdK82c=Z-WKr<&j_Fy30q`O45Fzl8A@EQk@X#UfFd^`;A@Fbw@bHkqz>4?c z?x3@U!Bw^KW8ZNH6+KRm01kVt#LkzLv)fwz`SLrBeuVg7ur_vHUsU-#)=?Lls64k@ zJ-x1$YUQ!^sKuzOxM^VX_LZf0g=K9OjAU_kRa){{YQT{VqehJ&Z%;@p3JT=Ylh86F zQG9sbDE+9KmT}b@nLSrxn3Yfj5TWlncaky1!{bbI^x4pyx(Fy(NqswUF^1_zB&TRH zQC^9~+i$gK3602V^6WX1>PGU%OWKk_J%`+YA?rpW^b1cmapUYbDI08K2q@Ay`#3qT zR375?V$|T=BY|s$ahMQ3iu=83Ibdbb;b9+20u)*!2nUkz$2RyO9_bhZgwTBsrgaX6 z%TR$85;ID8TU_+{h1w@me`R&!ESXwluk9^L*veP`Ogwbc>^B0jT^ich7| zb&`6I8=2s6P>`A|$hlQDoud{bZlJi}{o=G9EcHq9bj6O-*xu1tzO5BZ+eclE*yj6T zch|B?4vd>_^^Lx4IaD(b1y>5%cOs?gm2E6PS6bk9*IikLD)bdVUXG{hjOSjyTK_k`K!E*g-Dh1xtGqOCmOh3bji)+^UtCuVue8f>_!~G& zbMYir01o}X2XD#}3#|oxdHWvuck@IrpKquT(maC?ZyfYZ$eYb;t+x}0w)hQDKj9;6 zQ!&D3oKcfn-JEIyAWtc0$lJy zgdz(z1-WG#NL1zklIhe3YH_+tQbDzcM}!C{ zS}NIZ?RU7w>k|t0c-BVsV>$x|&qJ$LGDx>vqiV$-P#eP3@X1iIG4km49IJrcL20{Y za{kX;<`?MWE}P>wLBetOS8j5HbzGx*=u24qNgM``OW!W^!G9g-D#PMDP; z|9AmVFY`u`>d?QTmVnH$Nr#|Xs-wkaUV`1!>$ye~L7yBl(#`m}3$aPX&z9*5FZ7RZ z4D1hm*EyB49z&%{90X#eVhLtA??2OBT+^ndDkqSsJMYw|qD9^dH5VhC<{gl4#<3Hi zbu*)U84d53NRO=Ky?n4vlB%~s(x1b}i}3;yM^Mbb?nob;0)X@_vRX5=l6yh0vM8d8 zCgK#y`9l!Q$ANTQ1I${`n^I{h@~{JAi8;bBf4m#{(fF6&3*UW66uvVuSvc@-v?>W1 zyLTU3Z018)!j1VMr>=b#2wiThAlZgGd~9QHBhJwK@_FJ+tYDHr=xumcdmoJQ|Gfs$M91|{vB~T3A6X2o9|avP#3E* ztp7R%j7n)LWV2&SY3{lf%cphiS;%ZB!faqeM2>j407{h>jU@5PkG!v7R{uIR%+vT`w^7~t9kMd)=an~13>k zS!Dld^G&<2$aXoo_)w}Ll8WY6SlVf^$;yux+3|U8j3J(oOXT|86Y=BCMNtW>WPCW2o$G;F*)~`3H zMSaVtaBLCh_cl=xQ~wZk@?B4NO*$m_7LLOl_?F{V@rl04N3H-Di)fMN$cj3@Co^;- zu~3ftSEwB|^!E`Kg$%^64bYg7@#y}+xjM=#9DHQuRRw88YDI=Dkn^AtSaLox4JfiI z!h0Ci!$X_LqTk$ZR&b9HS8I->QObHmK?b{p`~M?@t99`ojAgv;p6J0p-6{9hs*Y9NRiOz1Em_AmAT+ z`}8f@VbP|g(NM(Z0SyyClcTk3-M#5iU{h*TaA1HyH}Kkzy0BYgUdnuKxaPd4h(l*7M~e%iWQ>>A(5{K*BS73Ch3qPIm-8jX5D3Sn7yN6E9)a z6tC43ue8l(>{xGUAKUl~%%5gu*zFLaYOUqJ{vFY6i9#|G{pg=PLLmb;g_T>3Ci-!_ zJl@Vtqr;(Dz*y<@X)X{a=j%(OM17#!79QjcLn6j zKF>BuwrN~La#%80$tAu!!LlNEZ!2TI$F6zv9RoG_80 zt|)lyWRK`wIuj$}%Uy+Kzp!H(anU!db~?b69Ypf=;)(#;zF}oWFm4WLEa#4vDP6>q);Caj~Sb>-wD zG_t8}sVM`3o4vnnp6+zA3XWpY@@q;C^L}&6LbHCA6rFnxzY-v?`tzJ;5$Eg{wb9Kv z*4^X2-WR*Kk9txbaCpOo=zJp)RenTnz3!s@k{C>}C6SFx(289T(?3s9g?f`jxcig& zh@rIcyrAJPC?zF|yb-n5jkg=d^hk!zc5y!Tu#p6C!0T~4S?h7r8n{b+27 zI0KV@c$Ku(UgXg`OTf^pd(P_r=#xq{+oIR-DoRw)yXVPO)}6-56;nK&!d5X@k{X9 zxD$-=5)>rHXT_n{`t;2U?PLTy1Iq$c+!(*`H&@M+xA3*U@p&HzrJy&D(XGOGz56m7 zEhyO4>j#2fZbSEPD8uU|GD-gRiL{#2ji08vuk;A;vizxWch+y8uhZNzYUn#&8%m9s zRG{QEzreCdxYwz&=mm#>+5t!iPxx+$ky#e??A_!vq$4@_8XksjVIjS(bg73@68f<> zg5j&bXZITs^@VP7{m~ehm^+KM3)dfi@9*Hpo!WCJSa~3Sdk&BLL+v->K*Nbt2Vd)- z$laQBu?&tPz{s-Zkh6A1;YWK2m$ae0CEP57n~-*xE7)R}Nl{Jy!2lpd1Ws7z2xD|xwGri)EF0RRnNgZ6ZbEn^9cZ?dorbH0;CU~~#5-oNASprepLC{zu z((IQv{xzNCM4qH5TKQySwD{CmOw@{!v~f-!Py|6+5{Xb1v80S9=Z7uprAJ@zD6eFz zQkOQ_+YRTB1|qXgp9LDXp+aM7&CE`egwMZVoI`9t?T{zH`hf1cWzGY356^2CRI?hn z=BK*iu$S7%VcbW1Lp3w7ea;LGvkTf5*Bt8-(T!K2WmQBWkF~oI78D`65kyO>HP8%K#jW{Xd1Qsrt2~vRfr zN|P1LjehCF*!?iY*$EA~1FNg&QF1Uip|~sUR<6AI_E!Qi3tscTS9kvx6)d2LOLn}3 z`cW|N@H=D0=YPMuldRRWsj*)j#GcY*>kVu{ZqbA4&Odfrh@D)uL|jb~}>= zY^5hx73h&jIApZeL&5Kz?%GotJbxnUec@wFcB#c58{qL(AXU2--Lm3m*y8PDh5-1A z`?tmlV85ip3ObGxg9IJZhHjQ4Wq&7JH4`Fz+;CRfbWZV!=ioF}M2$;3Z}g0U{*eYv z2u@(&f_9rq!zcT|i4avI#`tQ=A^3GGk@ZNbU{~?`yBy*H)hC`ag zqC0p+rINA?4%D#RVFiG`+lKB_WIL+lV^)5Y+tBIUDyyno@$+K><&vU{elKU(D8_j> zctak^*e$;$^v}27rhEG^j7|OOhw#P1#kIVtC_s5LSnAD%Wim0+=Eno>1eS&E>(%9Qi8D*%bRuVE?6GpZou78_&XN|;o%5SDA z@-foIV<)!1RG-E(6AdHohu)wm1-tjb7CyR4#4ehUcOfRfo4&Y%5uhnFi3W*w$Gm=3 z>Qt5ttFbv%5)2X6=M+FQG5ndL=g>E-5AQaKE3D7x%7$N#+E3ugT^NVHx=@$MKp&4V z?r+Wgry^(^SiJR!Oz&aw*%b3N)ia#r=c|g zF+v|r2bXnE)PdW8!uv=7Mdv8JkfD$VVjps9p0n6N&tZy#WaziXjfFc#4YRSHiUa!6 zf=qp-5t&x0p|{HHO5@?FzgOCsrD22WrwG#yjtb0fep^47a{y5ui6zG9Gqh@p>&8Nq ztp7))z>ikZsP|YgD!oJYNi-6Pf>9{K1iwUKZ1YqTpcPQkI;n+?{74?0x)q;#730;e zCuJFWLmwkzbDlpUXlbe|mNZ6i;c-Nw=3gBrta(+PsxZS_8@CW5Jp1z&6|1kg)Od^? zD>rWxYb$|$!-1KjF2v~=D|I;DLIPBR8hd(N4&U0Z#+yW+m0a{ckW%rPg@1&dR4gFI zN=(2*-%jxS5}nPbXcMljNt#tm_>3GenMS(RH*1~}Z(1Kc2vtjX-ot}(853O)N{x#I z=}gyaI*WK}6YU*JO*Sg(sf|f$I?K)l)mF7z<|TDFPYLu_VW%fCtVC#EVa5%yCZy@= z_Mgr_{9;INYM{U7CkP)ZTfMd#k+D0h`ZPthyNgX7bFD?cExnW6dA^TfFvrTwI>uma zXiz56FXEu31QIb^n3X8{_1s|RnDYnA1@9U826DCq@~JRN(bD%O}LBvoG-hh z&cEG93j>vo4S{o?E!D)-B@}Wr1*fnQ$9Qz;wVgAojtX(E!_@ooO@5&u>Wxlh+FD@M zNpmVl?+CoURg#H1oqe20Incg1^idJ5ILE5(->aPr6ihL~d228a?yy*{TBvi-V-0OP zj2wQ-jG=y56whQ@zfT0H2t%njSZQ{s>UL`@RDeT2OI5LutN~oI%`Q2PO!d82j6~za z;O)Kizs2G|o=P=gI(|cv(C>?SJM+pd_88mYGE` ziJ>8Wo#OM2NBOEU73Akn`qQ~VSrKfPm5P5RdS}zowGfd<#H_#-`Jwd)Ui4#qY8*qv zlmS3W!89+x_v^_v52fKMwyh9#TtOp95PLh1dsxBiIBXC?HLW-CDl1ISg zt31?4lJH`-g?AeNmjR0iU*CDWe8)GFL>6ld z#CGRn0#n5eIH&qV4Sp8!IgCR2*8sL=<7on*{g^0GI4FL6-YYm)HnOUq2bkboTN=*t z0viX3dH39K=R8b8S%Y*By$$>vsQ6aSRyEysEUY1Kmq=JZmvV~j%c{XHJH|U{dMxp2 zjcBgmqJBw?hGZ^g>ZJZT;#E>ahhNl#bHa*+PcBDZZ?-}RUCX})G8jh*AF#WnugFa; zp{0q9AQ559?EGuCI2YYr=qu8NLtep4j|ho=uObN3UdB<~n%ZAqcszZaamn36Bvsd{ z>)=mwymv5D@LV2fdD~dbfrGx_fjeTkx`TZ^zRM(94u>g(VJzJ3bfxYX5HGH z=S5pZrp`_fDdM|ndMPgGM~(3$kV7WNJ3gyKhiFQ50eauGhOvNe)}Ur zAb+l^ozppd`O>j>Ri$1lmdfIOo3BKJJ>8T+J+Cjy#`^d-n;}FaY!nfx_jD+zk-%vf1S7`OX*8I&;Z9A8mBn zmUkjS+)E&Jn;jd6cYliMW_7Hy9a!zUDo%QYa;1SSLz{Ur!VGovB!)9b7ndaye?*^g zitlSgIyh>LtUWZ{tBij=l0u&ei}&N}-xB>dfb4z5s+W1_H2ZjCzzIq4X_iOW=>W=% zUaLBAU9QXi>`1LR3+=nxi!^4n??LzrP^k5t)hAdRVkUunM&B?ZeWB=H$M1Q~gSp?+ z%0n7efPFxSktKRu^GE;gQ>EOE(eT_ z?Q+#Gb!4O`yXj&s5FEy(IaKU5H0?9#1pgLXc(T}B%CmF>>b8eGhKh_OY(MG$Hs>Fw zrL45kUSLrirB~$c4Kp+D51`8Oi_Sp&!2KtODKL+mw}kLy2E7xiVSdpCh#|Xximh-3 zX4|hrQ@@jK+l@9LO4D*ztv4J3fY}CA(U$3LwW#m78A#>wDo&g-9TzKm(IElR8drXX zmd>cG)NiWLHR{ROQW#btm>5iv9LjRj0QEjyT-g4e{{s|fY-wt)wOVKXSfXgnbkf{} zQQ1Ck8Jyt8sLekg=!qQ3T-5y}^&h@2MJjh-NO*Ex?{JJdSan>9-2#614!yyiC|-F_ zkT!hiw7X%r)$g0rADXe>BYG9xROk0R6cw}}euo~4d1lGT z1GhoJrb;>f|K`{acZVl4SXLTc{QXai2u+$k$@YgO%20Uro8>Jj2L6+`ach%shSY{D zeGv&W$_O)9T{!{Vm=yLQn2(n^b_n)Gx7bN~IvR_s3G4)ynWhL<;|KPROYvdBbme+g zx%E0VSq%FF+nX!RQdZQ+Ej0#ya5-CY7veE48bI4iT-txg7?AD_<`|9a3>80AKEZFrVb}!tTXZaf6t0JX3&v zsbamuzxJScTYl+a-*RQ}5p`RhaR0k#^<4W(onJ6q2j>3wq`RbVtjq$7EF9n4=bn${ zr#QG2MuYv@QgV%PBa$PFG%-}D1tqthmli|riLk#di|zTBU6Kdc407)}5JZO>e4FaT zfd=~XL(4O=caJ318E#8b))C`-h|%kYWWtZpX4TU}d(^}JGI$ofPyEERS6Pp7*WJnK z_3K@^)1J{(U(?g%dX=)1vT~2uxbxKA%2U`ZkQa73zR2`hv5(lr@5^tf9L^!SVxHnV zqc{7TQ{|7Or(Fg?cIjK`ku4RKpJyv} z@QzKfJB%IUSNp?1Wao}SUm_}|xqHO_uf4%QEE~+)vha9!OZj(EUDbC{CoX`QpUFin z0|sSOxvkAF?!at^cvBZut%&H(OV8SJPRG?Z!#@^C5jc%kp@`qV|J}aPEfQy{xA>J4 zt7T`=ym0O4ziR8u4{E^~u1p&LQqFIu60gA}r{Lb>r=-2=35>|u-FVaD3;fo0^| zwxHW3wv*()gt{lWFBW5JgP{^yk>g#ky9%maK6C6|@Wa&v_Y~w^#PMgWi>s&AEG@pA z5D)TEQz7%T(Ayc<@oygr!5D3C0ZZ@fKN0?dJt|+{g;!2pULC&cj46sB{|Mc{NzK)O z$A0`uap(v40U)q++?IOZX!+Dy+=^}>@^im6!XdosjH}g8X(lO8Bvoch2EJ0l+LtHq z=0D1KNUt^}F?{*rNmj=ES)*e!tou4*d6yxyi5q-*N&yo~@H|C@pO1-K-=VeTimOL% zNYlkr!+EPsLrv9Nxx4lYMN~cfjcmU~)W}Mn+qey2rgJ{&AQ4SsdOTVW>R%$lGFx9s z{OaBFv@oh1Ud_YM>Gx}sZie#W!xqVPx3#Ehe-6ADD~0=o)A~r4WJw5%+}Mvv1 zY+zI;J+fQ}JL2Sn7p>gL6qZ=uaoVqHD$HWZF)4-swwKP{YZZHJ$g3`1wll))wMDo2 zYj8gBE+yTOWTD5Em6eJczfM73d9?_^kDaOCD(kSnNpso?eWq=6hBjbLS!luTN;Ga? z-Cs^u6UK==9vfrWw{NLjaBKN)S8G1JJ~PtDIJcTcU|o5y{fXL_)}^x2rnuL{zJ-~l z%L~K{`Yd`X`feDLY^%6c{4RTPKkYt}0`UQ$FQTU)(O4>`iy(or<({v4pWZ1v3Q4Rq zl>K!33S=n+oIi7VCZUQIJHmdx+alN*km0XB$73ZF+)x1W_#6=-*di%n%e<;p9V`>Hwoimb^LIT4Mm5+iM5Xv}* z@kq3^rVA1HBD1k2bQG=W_hdi${-GDoYNFud|c#BF2^LLxM+C<7s^3sEESwLn6pqZE&xmz3jNy@f6x$y zpz?hk6GT^$Sm4A+oAy7DMWQkwhoe0NT0QQkQTyR0;?qz~%t8=aM5N9qeC$JyRbLWY zEE}|rG?U*w9U+NOKOm30qS$1|Bqc58%^Isx~)!EH-j?dFd@9 zmM->MW;W6-TP!7miNv#MzNPLk9L;%-bOoB{mYL@c4Vrb56Evgiz3J1vPZeAJyFZ8O zZweYvF{$B0UBSgXN;WaI?;u&oFo2H7CwET6(FR38U;-d&;PpmqEb<(`jbzQS1j8I+Gn( zq($D2_Sbh9$}V}f5&43s$e|o|;m>M7hdwlgAN)m%gfL<7Jw^W;1jXy%PJqekbXH^8 zDS48}0K^$T_pp0r^5LcjF_A{Ft#zt`r8FS5W0Pcg5Fb-lh<(9o5&U;^EN)pxT!!R! zs1-HgjcXPp`AUTK;N-rrbZhr9{FR5{5g1(l;Dzw`jSe%CF005|7VcZ#^KL}0XJV{2 zxDk0132P7pcg(bil>dT$@-{#1XX{lYb<;f$Hp0IsjbYB=MQz9zoHWlTr!fs%6Q~ArJ@BBPD53f2(FVkD{vWt5Bit#VL z)0_(B34K1yzJalJY~Ac2AdpKn*me+JMD%rZPG3JGRv&xN;&J%Es7^AWT`gJgFd;tq zV3>)R-8n7BClT#U6$(=wO1RDvAO#86jmhkPxsR)geI!94EB#4ppzQ-mW`j%i5TO=} z$sRzSGa&UM@OL0Qj@BCgA_waAstY0wIenFzW2d{-P5! z=IR+JD=c|Qk+}XY=cVl%ue4Sk5~+BMy+%?RQd>2wmv?^J65f@rr!Yqwh)#xO62BGdJSz?c zqsVcROWaC!3n*Qp#^fAO=T7@aDM5J_*<{Og(~LX+GQI>0Ya2e}z3d(TLa=COe`GyZ zcX{T-W(vRnL#OZJv^S_*yv-c{F4SRV&s&i*FVIku zTS_to6+jAq4T!#%ku;`tq?g(JE*kr@ER_ofDTHo*8%Vlea3k849j!TeKcK6pKJIZ8 zta$Rm(Dk4cR6p(xhEM+k7Y`){>*U1)i=$jV64Iz|&Zsx#5|7KJr0dFuv?^&WxkaHs zhw$|;Y#sKYf<7l^%61PAt$Gl{er&9&2lQdeF4_lg4WKKvi1b<$?yh!gv$J$%j;qwk z*hBzLFYs}vW{y?Gzy3?}tJ>;#e3FNSP;d4j+!2yDB4dSU390xEdYRLE22fmcbM33M zx9@C-dPi9QqO4(#itzIiC9kOxH|g`!bBz{60`T*C=w)FlzW-d!i~rpFsm%P=v1=Fz zKKCx@J$oQX{`Ql1goC-vBS!n;gFOAFz3h%2`9*v^Ck?c`(`l}V22%Uc#Ee0U(g0hu z_}}r=LLEib*AxLm3I=pXkz5mtxWDd0{mEzP$#&(8Zd^mX0aE&Xja+^HpP&AQto${v zgStp6`lr}t+GASwfvbZcm2HyL_hkXg(E#=Itt5)5oJ)uI_iU1l@%|+Q(-x?9&PPR& zkTsW@ILh~He3Oz;R>y65aB z0MiPAgH)m|Ixv$&VIIc_;O0Ms77<8Nb)LY+9%|bfN13eF`?7Z7Y%TsG;gHtsO5iQJzP+!yUha;IXVQ42EQB4b|bAl>tKSl^_)CEP4vP?h{q(rUbi=Zc6rJ_kKgjJHhROU`x;f+NbhuN{X>dB z`wt>Ip4f8cC6J8LFQqxUJE#3Mu(?ETGyf_Cose)=&^#g6s#k>@Upy`I1tHqcS56G3 zP#TUi_gK|-uF2KlO2g=r&{<#N?!;&eMsPlC_8KC%+%=_oi?NZsbXws~9GMyi4EP!pa zF~J1(%cRHs_veC|pYjIQ+$v1q+drYS-^qesgOho**4$4%Kh32Fwo0YTmga?lN!1%w z=3NZbfA1C<9#`5PN26IMq4vkt&n-1^#T;Z1dDj-_I)TtZ7~Sk7yAa^V}pH85dk$glLP_(?{4L`C4YiK_A)3Va%68}c?;vLu5Y)?+jqyt7^_nLXI8CzOp6X3n72)v|^N-UtMwF(j;# zP{P1qO}CY@EJXYb>1)CZJ6CoDIJXS;veiL)%KhxR@cA|i%aFL- z#CdcVm(;8QbeVkSA0L{)=v@St6eJdX!5B&CiA6r$I^>ND(D0+3@KLtlI1*L`*#0Q| z+t?IsM7HlxgekWp8jvzq{hy$`g;DMMZd~VUpfdJ_*}hAdF{Uq#0TAk?x^vc>5fp_# zKeQ!2cbCOjeO((v6S?Nf+U2M>YsP&17S~g7AoSIuQy~@e4 z93Sd?qjGfA^@GCWB#j(0-XSXKHWbFQi!L>RQ5t9N4EcZ=8a>&Nfg;;~*q6GRYhBG& zzhuaRRkZs}mt}$b6A&<}coK1|)zC=rqMeb!OA@Pf?^ti+U4P&`lFb`WO)e*z_^*uU zW7|}%tpE%6pWoZ>S%4ZrR*kRD8YBmqcnv{)@Cllmv6gC8bsUv@CA~E&2_J3=qo!bY zE%}B@u!c%DqO>(#Uz;0}$QG06kwdRPsxv8iKp)SFDk$uwW{D z+n|eJ_?(4`d3O-=iate7)YQLXMG>LHZ-c{`kE736!vCmF+^vpsmsDhOK4;>Im|YrlkpxxK z#(4ZXp_8Yi|dm@vDyx16^A+79H-gBfMZ*gXAdW5CH!>2Y3{PG70?PbRQD7> zS|T}LYLk}ydkB~7rF;5O`rY2i72qYm1l*%rp)P8|rKD@FDdel)7{fBN+{va6#uu+L zW=ZdV@cA8Nt^0KF!^Ck8QK4BC-z_#7y|y11=HztO&yY_(`#qiPD%|~jzJ*<#A#ciD zfA;eRvYud8XNa4wU)wu{eVgB~d}gR~u1S@vd#J`QhRK@lQgYEN%Fx^QOhk9Dfq>#3 zDjju#=Ep>Oa8 zi=W^-b#YFV9hecYyi&ccTUL)#Fch{Bbm^2T$Z<0XHK)qoz254GeTE&au6nk@{kzAq zp8qcEMo*c(B-VxHzk-wqUjgk|TN(-z6HT$|MqItlVQczK6%Cg3j(hJvgCmyZ*2wQn zPy_$B;jev-htEu!iFgm}#6Mt97gj}@M&K|Sv44td{x+N5THTr=s`=br1jf|j;rULa zF*Sz_&ncGMf)%iPvUV^&!`wnL3*VncMmI6%Cey%7)HZW}oxf4--VNwv48`UA$rWy) zM{rFqcY4AYan+x)2~@tqcTF$)-Q%d|a{lC&9{TE-Rb;8{IO;b3$UiQgDkn}2>nRQ7 zPkNy-Q>Qx$vtnb8ucE2P&`tonx^9+vj8MM2kpX1aX))7Qrw(ErmE11#`Bx|34?^AO z>3K2=R|Roi@fN8r*nxWSpY)m;8m<_^+1tJ2@TzW)^`b@P>|?>r%Ec5U|4vSP-T~u#WDz9g|HY__ zd+2@RJWUaQTkG0p*w}Gzy&3(2J%~LQ5-$?`oJTA1UPHHYF$WY`rafNsE{)@fRgjs! z?8L{R*7}9tkV#Y+)eD5K4Yqa!X}>+ir?iE z#wMNZ$vpDlgPEi{wkT4hw4sVk_60+UlL!%1F=|hIN9I-rsIy}gKnOC;>duTN@pWQ#t0cQ zvxPHtYT%LOMJ}IQiu$NP#540N*dFGgpX$dgM9ee-SxkM>VC8y*i0okPJt7^y||-r1KLLVZ<$76J6*QobUWigiUknf$=wEqbB{u{#H2WCyA() z3{!!NGb$!A_mb@Y+Ui7tZs^&)+HfD=3%vnFC75=799EGj>+XM5^JV4AyiY~M4##!$ z9@fm~nHiHf`X;ZMVl6EIYw`zk6K*iiqo_1Bt=(jNRlG=YhQ?HR-T=3^eg9tKys~L> zA(*l}qe7*by{RM@Paul<&qPW@v~EXqm1Bi??Vz3xNgZ45(gaObcGKnMV0KjVXP^=? zriBY;hZ|Rqk*BUT0I>m|b-5(;FXB9|calsT0z8T6K1e+5-LV;c7VGO_$+C{!#sCwyt6KmV#Fzea1Z2}h&TxQXzF*?$O(}0b3;2-($9om&M zVIZqYPLkBdPt=Mhvi}svK)n|D8$pV5tpj%7rhL}r4Y>~j=h-l7Rcnnp3Y?firj=1$ zdUtvf{(Qc*mcp2Q0!4-oFhF5r#FTO@nM+A#xiQjHrWb~LqF;>(DV1m-{X8}G@)%Hj z{T}_J`R{ROQ0u-VBa$wiPO4>Xs^?c5Tt(48pfC}#(b<;bz8p=Nm%>({sQjq)H!fBk zIO^|VD9oM+ndIGa!EpbL{h{h8qR;4|U8s_#=|i%-+vv!h6*&hVwP$gF>tAERoqryj z4`traeSAE${tSIev^hSa&kS)6bqjJpNK30w5!AVoix1W}D}oP247@-1lqoRs*zdH( z)v3V|891?*R`Bg}>cZgU@hwua8j~ZLY zc)QkFgdrIS??O4hvmT)s=Sed#iv$7^4Hk44=OC*o3>!HH%#OF_&eL7mQWj198LqP2 z!QfFW^CxcGxGub>G1IoRoD|AR=8?>^0l% znZ0M0`2-&z2hn3dFp<&;Gav2{jk4t|ZJVS)xmFmZXL!SM|q@2T_W#1G(f_SHGz! zS)C6s;yjONc$q7cg7VcEv+HxTB3`J;F#y2Bwh9s;rv5cf7Iw=hhpO?d_2;^W>&JSx zK&Vx^^qR6hFD^JbS1yYkx?DzY!K&TB`1j**qzE#K90cKe5`tum&77k)E7_oGyD~cm z{&p9449N?MNHn@`3GBE1{QLA-_MhB?0=#9Ga3Oj%Cc4ZL^+z%nPRsF#d*NKdtT3kv zCQoX>)F1xC+?QidQuyUQi1Q4I+m{{c=hs8xXR5N!x)(yM{7a?Ck-PiUm^x?_FJ8)o zrTc=OZ9Y5|RIJc!ct$_?o!WEP_)Ap6$h7$3`Ym-1z_4BIS$;L_AZKhMA;o3IOmTZI z4HHJ_o3&_v*ybjr@+(pP_1iZ6i>lDjQ|T;V+S>Y3Dw5COu4!Ltl*>O#;hHX#AK!_~ zn;kda8KCsL7@sIWiTP0N<02@8m=KX~@DPkL@o<5UH~RZsklgsb>p5Umn-fAUy^%5a zP&7H9MlQ88PtjaNP=yhsBT_)IZOSuDSFAHOq+uVAbs^X9nC_`~+SwyD?u_ACRGJO6 z9RBgK@jc4Q|F&|}HXC*ocC|uu{?X<6>fyQZre#IoWrkmBCpxE8^CtAmYoRlM}pi8JA~@-EU8`>yu8(XOkbjH&S<{*`z{x4x;yP(MiF)En(k3YVyfTMD@=8@*F$q9 z^C$W#H<_8#hd!DuqoweNO1o*YUB~qw`!g56gPNI$bqwmtbMfrrrJ!{rRd6bRcYC0b zG4&+9E;Ed?vyO2o5m!$5EMDs8t{+)m1+i^_7_MqZ*iLs{xMBmv`;JP_Pk_!IA)VhgLTR{H z;7nin!>fVyI^Xy4IloSdG{MVJP`nF|zxU~2EHnHpd~Q9L)m1W|T#QZ~!@BBMKw&=M zs5X7v66%w(8!PoR=5ZmIk}h2PvvZfQ6(=>;!YC6-`KG_#t`gfL$ z2#RXO?t7yErp3ORqHpwKw=~ZF;ty@%JU!ftcGy)J@@k}P6u#CmX|q}_C{bvC_>nQL zF`E@JN=+9Ml%pkjv3wpD$oF^&F$3p6b^LJ;igHyoCbqQ6VEn5HMcp5?aLzTGmLju7 zSjjbZFG?E5V52-0fB?ZpW<1y*zLH=s0A9?O?}so>x}FOQm<8gE6X9+3BDI)$TSrt% zfSOc2J)a^%T`nL#RXjh5I+*5CAJ3oLJBWMm^6Hgijmj{1)T3Qd4YW=?rClfDv(;jm z!(S!MHG7A$hDxk76hLq-C&C7?PE3f+M$-vFs}h5t|B?3n{3Er9pLtmo|3eGY59Wom z`&>KJuxfuq#1$_XEgNmIkd}eeSA*mThYQnD>PXomJFsQ{Z|c7THP$WWrxK~XCu|fq z{!$@|>6_-n?{u=+xy+p+MX=H~8vG8Bb&Em|wblveK%C4lH1%09DQi)l3-- z0JnbegOpWo3vgu}@3rp^&IY`4rW^JNDO!s+gRYiR(iqX8A}2y-`xR-b$v2yfCZ-Xb zE>c8VPYPzqncR^#e@p(%IHSazi-DYL6Asxr06eCOK^~w}>)(b}P^bIq`A3ZG%<2Wn zCaTi{B=a6_l}-Zw0;rfCqKzYBo+A)!nmCFE@)puwNWKXuVRqUoFu30{-uaO^#*F@# zTW}mCn8$;#s!~deCgR`n4aq_?QC_50($OCB{C?=b*`@BE3F3qJVw*JED50GGx48`;bC=X=S(M@BW$T z!I_=?ih@T7l&b!Mm_v%MbBZV~`xF{{-Ln`?iGsV)!;GQvk-ShlTk{ZHVdT zSKtB7IXuXGajT<>Ri()m_s^>X);H2N9m{%ncvdjS9uVaKN{_AN#(% ztU^W1bK}1Gezi4QdoOU|TKVGcSg!pk=GI7x9+lp(USR5#KD!#W_3z`73rX$xR4oqr zQ2yK<*D23=z^l;-Gqev4FMJS# z@2@ZYUw}FRbsoz@x+m1BW6k+=b=t*_W*Cm~mnJ;x0OUM`uCqVL8EBTJd18V$^+3a> zZ~MC8+D@NXIxfzD1;}|oprlYAYw${^<)k}XZ;3U9F%CRvEI0))J7(ifI9m$ucrw>C%T`t8-V6COvHZs#AFbpOja!rJn$%G!1{IToL4nH_doT zeS!kY&@+2xZPXKUDM4dS6DD3}60bxIokv5GpR;`Q2<5#UqPc_w!uHJ{N%k1Fcd?;q zHgTxt$YW*trr~0O`?_nJ9)#@=Eo`p~Z$j99uVMR3F`tg|laq!FEoV>MiEUh3DS4vG zF-f_xJa=NDncu0*3!i^$W=-Nrg*2DKm}q7>>EtsJb40nBufghOD&)81wiNXB%Tg;h z6OYD^SijWwiLz`)O%QJ=YkJuNXqS8*UxMzj43Pnz;_m>WZ~4Tbq(2@8=9G0 z)l@g^i|1=GyXtC9W6tKrUJWYfYN@#|Vvk9pxoV6zYua)*w>5vYrl6bU{w>->c)3=uPXs~mP2W$EGv>#l37N;%7^Jq2}_+v3TP zM$Z|q^ua0&6fXu>eX+KNM>qUP+SAnjU!{`psRiLZKKvzwzV_0q7adtpg&3DsPO~ZQ2G2NVV*2+S= z%I`h;!ibmB1pVLWGzz$QKha_aT8=HI323T0C5ZB;JQ?k1VXBSw>Ae7&x;PC0O@#zB znSfSfZ3#fMNkH@u9uJXN?2B%Y7u1YdGL%VW0VU}`*3f?xw(?*60R??CvS)OjYeq zuI=bLb}hp)F#g{dQqB&cnZ`KnM0h$27~0ip;HMSQp@qPd5Q8fr&Z+pe~Y zCJBw94@+J$;0$MfZhT04&@Cp94@*=_V}vhqw3_{RC2prb#94cuJ#` z6-9q{&t&*awD%xJfe|0eAQJP@=Q1Sd$X}wd{AH3H~}>TROUUF zr*>Z=3Elm4;`FXntT|2>^bv>9XZEL$pdPkdLa}M%KCxZXv4g-Nj@Gw`nk#)lA8|{x$q zQn&fFI{-HTZUEc>xE);$b4!|iPyT?fI`{O@-Rjy_=L-lN5vY#OEG-8-4 z^-WBH{~R9w!N7m2@}HZ@{}C;u<&9Z)R*1P4{+Sw7P+EU$3)Rgh+amBI$Xj{$fGgdv9^?Us{b5 zg5b>`CiWHt*HqQP7QvpT+Kzu?X?>!bfuXyB?c=2&I0RR#1y@hy`;Rzq{u<+lF4ptXL;;vpH$9s6`g)40CQerixbg_*_n7B>?0ibD;!_hF@>LXsuu zi-Z?31f?M;4b_?&=Co~fE>2xGI2-M z+ZMKV5T$aUOHewXNcKb| zJ40TWDU@{y9#Kd!Z3Qf<{XbzN(V zso7{dj^Ww{lwmrA-m`x{0)bVG`-W#YzT;UM!Fpc{0$asIQ*C2`K>QQ$^gm37Z(5UU>yx}+IFl%RV~vpRBV=5$6}3~bxSM& z9{@heVKEcH2Y`={03S`OmZLcx(`;L=vjf~=zZ-YRvAHD&=h}bV<2;hp3Gz2eLUbmE zbRJL8J$+{M=Sn7m0g4n*q|hS+7FjX=0~P@+qExJ$vu=f{$11xu^m(pc2%pD7k4;%M6;*-gnifLIjZMftQMC(!1T3U;g{)HEUe0r_}l$W z=e+m%rsLX9%QbP!c8r#d-Ifk0<`8<)9#M>;286hl;aPv0YG}S;Q4RaL?mL0w0E#(0 z6k`=x72v7Hd&kTYmfttrz;!ScQv=18hh3ZKE})+S1Pb`**pGNxeu}lt(uXL${;NEa@Py^RAmEke zcx5t3DNAzlk^+9Kir@ZVVKSVnG5A2^0c&;TUQMZq;|ik+DwzWQ1^f&6SF@b99skm7 z$HIT6j;*%>5w8~k{{sG1j+I)#zkq*z1pjJU)|%>cG`Hcf&MWj_JC! zVc10;flO#5Hp&4{DQclV#LAM2HxT+sb z9vNjc1W^l!T0qp&FsE&M2Bl+brjBumDJ;}wUJ9duvIUeaWtxdW*#gRzV^Fr3YDah4 zw!Q;li}8M8i)olG)3x;cCeoI(*o#RW5#fLGNx~9Y zN?CkDkD{1{EdPBvLE@bRte@Y2l#viU(ugAZ*Ep6dCmx|B%>qUf^pm9nlni*BA|A3w z9%W*=z*81bMBlC*F|01NGuj|)hy~QwX+1E zuGc`J3W#RPpRSi0`5f|*4Em?#hVIs+3Ri$MFZL94*WLV)F4lac3e0BukuH`Rxt{9@ zcVb)i<430IsuRw>S>2|!xlLg#_gh$0^toQXpPM^}y}6nDqDav2weElL zo^peyYwo7AxyhM11)VMv%9st4q*xTZIPP(EQ|{)bX6A(Nz5;RsVOML^qtwv~gZI?y(tZO1^{bhTsZ^3c8=;I{Xleu`aPYgw9FAGh6!nHAke z6#XZapE>3c^Z#o#LZg3{qd7*2qS;4H(Ey>X8T|pFEhDtLALW1%S*4Zc!jckDSffxF zjT3n&wnSa>WYK_0P9(8cN`|uHNS{T482Sm>4?n1`%5-LpVw#wDu`o#{P-)da-#~G znp%1?5Js*>Jcf$GA#Xb74?Hg_*_nzB87JZd-2eHVoP7<|^oAruGz+TUl(a zmy&>G$=od^@tS|uDu`yL_LTh4>1Cw{57~nEMr&?l1=)z6d1e7BUbJ~{55zP{7WEnH?IS90BFXKVLb5JF3ya zdRup_cPmGBpc^p_w`EUzRqZrw^dR4E(Vr5;BXq)IKVyGslGDQ8M67H^2|7RTpwl8J z4tpb@u}#(64so`QXK(?z#qF#yOLEmbF9SS;pS^o52gY4zCSh#=u;%ETa%CsqId MP?9@ueh~tM00Q_}dH?_b diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/entities2/publication/publication.gz b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/entities2/publication/publication.gz new file mode 100644 index 0000000000000000000000000000000000000000..96dd218174dbb1fa6ab55da289dc116bbceb9963 GIT binary patch literal 9056 zcmV-mBcI$KiwFp`FK%N118{X>Y-wX*bZKvHE_8Tw0PTJ2a@$Cf?*H=?6^_^mIffV9 zZ@oPy)Uw*G9b3}MvU|_YnNSpff(RP~H~=A8?d^Bp%qoxsNRXl=Qlg~lu-g(*tV?EA zR#xVhi63u9rJuybfaPyI^5e}FO-cqoy`lMk#aD0s?2tDh9n+&U%J}m*y^4$2Pq0ly zlY&8mQJ!6~l%@fr!8p!R-Y`gLQN$s3RooQuaFpd^PdqQ!_zirGw@B%b5BD3*uUIi= zAwDUB0UI*$_$PxyF`#1a< zgAU*!e2?MxeV7;x0#+1xd^s4;FYoj5-;YmzKVDc;nvDs)qH#izTy4t|on8#HkS$wL zqX|wXq3LBwFWLMo$~0VHrYQ4C%ad!C zc8}S$;L7CFcQm>Q`KTj4;6MHeKjt+320yFVSro-dOe=ZePopwP7CY^bswqp;*Y*MR+Xv*`b%^L>VXQJA z(=e-#wx5o(G|t+`Gu9YSblD-3j{fA|XGK=#0UO_p76faVc-KB(kSkOf2l1F)Ogdg7 zUyJQf&NA3$aY~aTzP>lA(lZ>>RQfdk9$)n|O>ugPr5pB}fAXDM1bIA~thc7Yz5plV zfF|7odKHgv$fq)oXu!IM!+1Q7(@XM!7A!Bw-j{<8B#SMphmKM$hm-h;U31$KogGH z5JWmG85w8TVVsk+9I`ws3mB`2bJQ02oe@qS1}@5AKxjdd?3#?SYnJmVltaGc4xcaq z^-v~av|_|q*nu{~5op^K2JR4=hJ?XlalS*EmOzGanZty7M9eC0shB>u)=)c6}N5WYM@1rLJ`ao+r)A7@E+iL4OR z!aePmpd81fOvX8lD2!N?eit-%RxnszNOIT^#ZNE^AWIg3PR8CZa|Yi5+Q1@%vET`= zzsna5gMoJLf`kDD!zN?)aDuGs0};Zv6OLqg(IaOpVCk4#XZg2+4CocCWsqH`eEB7i z4F8?1F1!Khje`Q{G@%DhVMND3wiFg0WS42o*?=UEhoCy^?1vv=YQq^%MGM>o1kRi+ z#73N1;U()vu-Fnr5_LEiw;tIKL*6}2lAF$C3VhOVj%CDWEX7HOI2BkG2r>+kW@!Q^ z0I(MhIoN<>KL4qx0?Sx_mBl%inUe2}y|aV2Fu9Rv&jky3b`!INPmz5OSdt*w;9wkc zvEtH_j|qw`6_Y(5fD|EJQLJ*1P~1j*b>E>;%zK7)!1{)KPa(&r+boK65Q;Ih;=iWs z8pS?fqcM&~$@{?{UHryu9TO%Av4nrdeC)kX$H#9$+~Ne79FKwQ1bG2_jFYdDNSrML zQlOOrd z*NyU+r6GPBMghL01uPeqK$G|nG0@&}+Tp5Tb9Q0iE?AIB1Z?x6(+qS|z?YDvNU^t_ z`K`M5{AM;$W0R9YdYEU`$to*^n@iaJ;_uB7IKM76@H51OIQJDt|L&^VBm#(2%r}nSVbj2 zM~8=pV8Eayk>Gy$1Uo-K!9LGMmHztA5~QXal0$rwisdFE=N+(&WfAPYw3u#c{F< zMO{q(^E};GLnU;W6S=zLXqIJ}VdRFE?`VRt2OpOh449Wh$y0!TK!QP^oR2 zx@!hb@TN}e&*`@mEu=FzEsJ=Z`GV*Y5{txxvW-X96CAy)3`% zS6?Xl{!QQLDgA_9(gchkvEj=+;nx48hwyt3IKjWT`a&Q#>T}x>{j+i!fSLg#0-1J;y}+%LakWbt(`-ChZY}UJ zZZQ^kn})Orfw4fp3xjIu{K^r2d2x2N&$lx-=k~cZWT`Q`)l#;R`As(4iur-nS+C8>*5(N=+~1^9yzF- zEW+hhyR{cVj0RC5*SFK_JLS>{hUd#&&p-eCKR@P=gjahznM74r#hf1BqOD&Xck~Dw6-Pw}m22sW;#<^W zdZ;;?PeWY|e2Zxfqw*lnKxkMbD6*_e2d?dw64fBWNX2y4Nmeav8OJftj+!zu_ZEKnf`(C1j(uQAF8%4#@*PE^Y>EJe`_MYB1a zp@uoB0Uqj*1_2KtO&H1@@F^2e3a7jD9^URf%<=$^DbNNO3~oO}&RD@{9t36ntb!{Y#-m`>@qIK`OTBDQ{4hTcae|t zJ6%AsNCWt_J|Jv@0+ajv8SitwYZ$KHbq&p0ln;)gBU6OlTByAVR8LC9GnNVsq{48z zuA&=K`#)pt-{9l??yeB4q=t%J8!I~j$cih-Gl&Tb;m-p2%iI}_2>$bP#M~Muncm=AF7R8pzpn~luj2y3 zX^7v>vT`iM;l|J7b{+RZ0^3W2V=5>q^;|JL)Alswzu!~}s^Qjc!qK@7;~x9TZ(qg6 z6bBQm>J)c4*FL~@+wN+X@?Ui&cF*060+A(r=3F}zBR&E@I$+cjjwsesjbe$8Rutk60PZs!y&3I)6BV;>7F}l!xxD= z{rzFM)7#=k_pxI08Eo${E!d^8pmt+++G^Gt4j@e5MRUG!-MKBg^!OP~?f1&(&bKhX z&u1SdWBi-G|8Lr_@P~azLFx(EC5!?eLI3>FM=d>Zzt>@DXgwyF4;I#T>ddP4I0-yX zdhF4w)2uD>w^qz5%xEqSAuBRmO^XcEv>eq^sS|{eK|@V*-N*_xyVECik8_I=>?-Qnof6le2E;5JB? zwBo>8986LYUX&zMAxjW|wc?eQsyVu9+6qtizCB*qlG@&RaKLhYki|sy<{0;gmCO&( zNL_FCem8gfLdWwt>T{iM!PBQL$YGkLr*H{9;|{weTXEwVIh#NII!!Me*c?+6piljx zzgF^$)NjwmPz^CQfrctjB*}tls@v3>OJOG-)vdV6t#a|EE@fan8Ca5Ua;txLE4zIc zFn_Cky29zyn8Yba%2L?emYuro_oxNG?ky>#q6P0$cid32go`ZU zB1^c)5-zfYi!9;tEK9h^6aS7U{y{dx9qN%st$EW7(eFy?%-R9>Wx;XQ0?<2z%^w#7 z85c*F$kG5ther#taFU>+I(SJ7!Puv8Npnr#vF*qWG{taT&2)X=v4Xj%xV%?NZ}Q>J!YOu*cTb@M24K3j?El}U zA8!mi>)1x$P#sk_w}Yx!29-eErgN_7p6+^v(=$wKB~}9Oe&{gjrCHbuX(^W!)`4?`0IHQGcch;GecJ~l*n?F(6LknezRRh z)n-MbO~Z1cNDo5GfOEz$R7JC0M^s4KENVwe z+)m>)-m?_$$;mxDE^%BeEa<2rD)Da!YQ+7cv%@E3Y~7k$40aZ*u!_@krdiieXu79gvQ#Yo_CL>L8uxk`5MT z^eYvKFTikHu@&7>6<DNt=%|c0}`h7|xeF@anTo@$vK7v3L9n{Qcm4KJwtEiyOFG-@w~(Lcb#RwqAl-?m!#90(G{L zy?dD@uQzwM#RqgJoR{cBH9b}HG`;6;3+DA2_ttL}YKwwte#tNbK(At&Vh|r%q<>Jd z_uJ|I(OZn;UdIVs&te1v#E9h>(yv)g*d$vcj0=QH3CIQ|0WAe?Do(M40S5hth`8Xe zsE~uG2y71@o_#9>isXQoI3Rxc#~(Bg$=)xpdolz(vFt4w(1N2oQs5vHSg&A!3+J$w z;9q6TyF@4lQYV;#i4a-9K&lu__=ZGfT0JTdUezt|luubQ#m|fog;I6PFFBK?!&$>B zY!2LZ!W%;2t`lMme46*nI}l&0BZ37*iXsbQtf(lcJpp#%A}$#p1;0J!&Lo6x&5;9f zWpoVvLI*q?gM1;snbS#5)?CpRJA>_`bm4z0VWaquECV&r!!SIE42fW!0}djhVW-(y zmq+Ii&N4rRYsMiUuxCaEOr%GC;f#eQXXoA1gEL@$l2kIn+X~bNE)c6TT4&GD6r1Z%sf4adGCbA59bZ8-@ri;IuA&|H{6WX*sMdAEaW8EWN4l zYr`_l_-CjAc;Om>G8)1~lZa9t6|HZRiy6ts@gE#RhTDFC`v@d)&+V(KSeZZq)msj4 zvH;j78rrjlVz&v^fz<%Yq!dKD&MMv^k`E@dFhkyZ5f=!03i(-pk_OTc7ehhjxTpkj zvY)~jR>6e{)O#@Qk&`kfG(^`s2TNw8IyMm1m@q{7fNF_T&?X$QR#3!qp@0WT$w72} zPGiJxB2zjffvzi@SOE=L$^~i6QB4y8;53{HwV#0i_&1fv@^9i~tHpvk07Y0|G>ndl zy4MC;Sy5L)Qw)qtXp?MDc{r_kI8CP1;zEI^OaM72Jyzv5g!iRcZlnHlt@H-(WHg-- zJo^x*81&{J;RNnof`es65J#6KoKQI0v;&D0dH_;r!*ROKePSDrg)w?XyL{R8Zj(5p zS47TT#2TGe_KYx!;CvOoVD1M3^ufp(+u5N4#wBmME=UmmXs+p;87_`F5oS%jxFk^b~3pN-v8Zgr}9N+3XOt-s+qlI0^ z3GHrZxRI?Ix}ipjgtWYZEX4_g|sSXVd)qfbI zMWP&D5z4_;EK6~lrAQ>op;m9J2(nRG^nz^Ik5kMj>toVq9`>N^|1E;i&yJ~Tx~kcl zW&8ybYg3jVz5Chuf4_dnF2Dc%j(CY05<&*kUqHAOy7N*W{w7`-d#c%8&wh~+^_Ps7 zkZzcbyJ0H3ti>#oQTE-Bvd>>y?-yk^3{TT~rYXbhGR!W+?CXZv|9`0|sxD*gGS)6* z?K0N>Y)+_$#@c15{iTH3@8unR^pJ!M#6Q{%PzK^aFUG;}qSoE*-Ws&$sRR_H-3eST9EqE_{heto8Ia?BWH!wCMifMmqFEUuJZ~H zP|PG`iWy+yMviKaGu*=MqyCz`n1uo>fh%5*EC$x6%W$oQ#+dgm^M< zhWwM%T1u@iCm%k4Jp5VJemeT}=`;NB)2H+Ov-8va_xLk>cKq{{jz66roqss|xh=C# zdU+n=9Ch}cLo0T|; zhLmW?XO4zc71hwJj;tb7hmCQYX@l=d;wDH9TPD@-GdI#@d8cji^$Edcg2l3m(}p;? zlPJ&ioXV2rD_JhzmGnYDhb=y!`&4mKJ;U@Ar>Co~tl}iAI7zsigv&{|oP^8C0!}aP zaGFd>zAQ;X`!*k_>)a=}(F6j@``s4rcd|y4tkJZakfSF;qwNsf=%to#;u)zzNfbO5 zbFzQb*Ln(8UAhQ^{xXP*qINaznoumy)O)7pw$^l-Vq7qwg#B|!OWdO~$4m>VMdh`k zG!MErjkQZURU`DPzP-?hY-UlTszsrxhPw@1xfS#P=Q!AA6F|Su zn5k2$t6`u8zHjPIpeljpTV_N}M{9cg-kt3KN4N5g74z7KM&wWTL$nMuU)QZDRG4qL zOjE-!a3U}nf-PVlc1hJp+(Y9~ z_Wgom5r{-Ty>#@Ge21NcKuHMH%YZ<=l4I)8;7``Z@lV`RDhAbsMKb)3cF8fKA{aRZ zDnwZlXZ)$el(7}Mm^fO892EsoJLGgk)UHML z@XGX7dLg#vh1eu6@0E~JV)7&=@8x3hq#k&%9vBUSg@vD%{ztfR9pJqbcH_~zP5SRo zKOQ)?WtnQ*yJ#c_@58}(5${m3aI>t_O~ue$&3)V=Zi>c}Zw*hkJXPr_rn$0+8-i)U z*dMYm5c1ng;{m=IRUlp~+-qYwhrS>rTbZxSV6h7#$1pcs*5`Zm=?>8;rgZ>WzsSSw2F#l<9A6a}syGS^iiO>s>pP~AvT zEnBf2O;=4l5+QjBYkd^}sY$F%9t$s=(T4%I_$8xXI-`$x1V-}uHSqesGs);z!RU7a zW>82@zjRKYR{l3u#P{L01h2m)@VcehaEI&2SI$q6?%RFo`~>N~_1DhNkFpEzotNnO zDsvpG4UX7o7}O)lMZe}_t%2p`Vg2C4T6)W$KL?-2TQ0LyWR{A|QhB;8l?}!-Ix!{xFGSWdHS&(lJMGUjW#VH>)lnu_L{hGqD^;y6@sWwy^2vVEk#M&|SE4k=|m z&zZ7fx4m&lSy66;IJxtBOO1BQP6#tjv39(!VaST6uNtn$N3ncqJj%Gfhvk*R3U9 z>%Hz=F8Yj_uHpDr*I~NdH5@JMI!r62X zg*u^2O-rR#U>K1RvPQ(B^=0Zn<#Z@9VvMM~z zeKRj4^y@W2zZ64P&H1{euM+z8D9A4f{Mvco*9LTVXR%aj{j6na|LzB34Ggv!Ov-Dw z)SmM7-q4+w`tUdL%GguQ?s_)S0*XxX@a2JrUOP`0JWi^!7DDLZI(rwc8qW#-gRj=M zd4*XGZkd0b9NR14KT@`97@w9acoZYV$8c3gwRN>W_0C<;G`yI2=Pt77qVmpN2=Cm* z%sY3%y>k~+@7x7?=Pst+Ir!-Uy>l1BJJ%b9(Hbx!!DtP*Ye`KHz6OPm&9<%%5QpQ zi4H)RX+NYnTY|!Yj!n$v(-qG&)3nk1}Wx2Pmi>jfdTKA#F&gKmJTin$a?WSZMxD_ZQYVh z5|AYUS38s=@DhZ}~v@=D5spiMuIWX0&p{&k*LO2pH^&q?y{L>@HA3l=3k1Rx4e=EHr z5+t=PkkqI1{j>Aa{r3_u^=Mum37C?AsV4_aiRWNkKMNUI2PO6RuES34WirptP2U3S z+Evif3i6^;sgl*kv}&I(*9yGBN8aZG_ZsKMW!BjW2LTu$RKhWq*mgc>fE_jy^97}ATfMGV;7vz>%~37C^O z;#Oe2R-?RrJ^8e-_+Mj|gxpQX4fB9;!{nR=1MzBM(qz(8%7LPjVFgLUISv?2#sgA_ zY2WLGsZuDOzfc&2UUFx%FrT?uje!%cDWr6~cIg16u_z8w8eXR~=pYU9*pQ+i6oaD5 zlR^qaO*eoo5h8Lq8)1S4MZ`f>AX?^{tetD3nC5HBH8CC2aBWp}RHw66uE{iOgw%6N zHlAuC5=Tfar(~m8)*K~12Rsg*(0Q$Te9htlUfD|nWmR)IOT*Vk&sz( z$qTL0yZP6%Cvm{iLOALdV4>VW8(%Sjf-3kcIjb)eegCF!^py6Tku2tISGc$5fhIpH zcU%>c5nlc>jqg*-z3SKV744#3%H6)0ezT&w&9Z` z;U1(jrC4r6Z%V=3kh&a#U>4HIl#22x`&&>vJ-gft8`{;`J9o?4C&0+ z3Cc=mhID4^G;yVk{s=l#iluaB?2uqeXNFXiPcM(p+?lbyf8xG5CmrdaxSNAw=3{E9 z`lZ-iO+Qx*Pc{YnP!(%t^x{zUYDg{7r8~#FY86Ce S(Trx%jQ<63B6;BU&H(_01)C@U literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/profiles/mock_orchestrator_publication.xml b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/profiles/mock_orchestrator_publication.xml new file mode 100644 index 000000000..b47d99b92 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/profiles/mock_orchestrator_publication.xml @@ -0,0 +1,24 @@ + +
+ + + + + +
+ + + + + + + + + + + + + + SECURITY_PARAMETERS + +
\ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/log4j.properties b/dhp-workflows/dhp-dedup-openaire/src/test/resources/log4j.properties index ce37270c6..d3e717dfa 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/resources/log4j.properties +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/log4j.properties @@ -19,7 +19,29 @@ log4j.logger.org.eclipse.jetty=WARN log4j.logger.org.apache.hadoop.mapreduce.lib.output.FileOutputCommitterFactory=WARN log4j.logger.org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter=WARN -log4j.logger.org.apache.parquet.hadoop.ParquetOutputFormat=WARN -log4j.logger.org.apache.parquet.hadoop.InternalParquetRecordWriter=WARN +#log4j.logger.org.apache.parquet.hadoop.ParquetOutputFormat=WARN +#log4j.logger.org.apache.parquet.hadoop.InternalParquetRecordWriter=WARN log4j.logger.org.apache.hadoop.io.compress.CodecPool=WARN -log4j.logger.org.apache.parquet.hadoop.codec.CodecConfig=WARN \ No newline at end of file +#log4j.logger.org.apache.hadoop.io.compress=WARN +#log4j.logger.org.apache.parquet.hadoop.codec.CodecConfig=WARN +log4j.logger.parquet.hadoop.ColumnChunkPageWriteStore=ERROR +log4j.logger.com.jayway.jsonpath.internal.path.CompiledPath=WARN +log4j.logger.org.apache.parquet.hadoop.ParquetRecordReader=ERROR +log4j.logger.parquet.hadoop=WARN +log4j.logger.org.eclipse.jetty.server.handler.ContextHandlerCollection=WARN +log4j.logger.org.spark_project.jetty.util.component.ContainerLifeCycle=WARN +log4j.logger.org.apache.hadoop.mapred.FileInputFormat=WARN +log4j.logger.org.spark_project.jetty.servlet.ServletHandler=WARN +log4j.logger.org.apache.commons.beanutils.converters.BooleanConverter=WARN +log4j.logger.org.apache.commons.beanutils.converters.StringConverter=WARN +log4j.logger.org.apache.commons.beanutils.converters.LongConverter=WARN +log4j.logger.org.apache.commons.beanutils.converters.ArrayConverter=WARN +log4j.logger.org.apache.commons.beanutils.converters.FloatConverter=WARN +log4j.logger.org.apache.commons.beanutils.converters.IntegerConverter=WARN +log4j.logger.org.apache.commons.beanutils.converters.DoubleConverter=WARN +log4j.logger.org.apache.commons.beanutils.converters.CharacterConverter=WARN +log4j.logger.org.apache.commons.beanutils.converters.ByteConverter=WARN +log4j.logger.org.apache.commons.beanutils.converters.BigIntegerConverter=WARN +log4j.logger.org.apache.commons.beanutils.converters.BigDecimalConverter=WARN +log4j.logger.org.apache.commons.beanutils.converters.ShortConverter=WARN +log4j.logger.org.apache.commons.beanutils.BeanUtils=WARN From 3dbc637d3e8e8a5a8c775821998b1991e10690bc Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 17 Nov 2022 09:55:41 +0100 Subject: [PATCH 4/5] code formatting --- .../oa/dedup/SparkDedupPublicationTest.java | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupPublicationTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupPublicationTest.java index c657d1865..773de65fa 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupPublicationTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupPublicationTest.java @@ -15,7 +15,6 @@ import java.nio.file.Paths; import java.util.*; import java.util.stream.Collectors; -import com.fasterxml.jackson.core.JsonProcessingException; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; @@ -32,6 +31,7 @@ import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.junit.jupiter.MockitoExtension; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Sets; @@ -299,17 +299,18 @@ public class SparkDedupPublicationTest implements Serializable { assertEquals("Article", instance_cr.get().getInstancetype().getClassname()); } - private void verifyRoot_case_2(Dataset roots, Dataset pubs) throws JsonProcessingException { + private void verifyRoot_case_2(Dataset roots, Dataset pubs) + throws JsonProcessingException { Publication root = roots .filter("id = '50|doi_dedup___::18aff3b55fb6876466a5d4bd82434885'") .first(); assertNotNull(root); Publication crossref_duplicate = pubs - .filter("id = '50|doi_________::18aff3b55fb6876466a5d4bd82434885'") - .first(); + .filter("id = '50|doi_________::18aff3b55fb6876466a5d4bd82434885'") + .first(); - //System.err.println(new ObjectMapper().writeValueAsString(root)); + // System.err.println(new ObjectMapper().writeValueAsString(root)); assertEquals(crossref_duplicate.getJournal().getName(), root.getJournal().getName()); assertEquals(crossref_duplicate.getJournal().getIssnOnline(), root.getJournal().getIssnOnline()); @@ -325,10 +326,10 @@ public class SparkDedupPublicationTest implements Serializable { .collect(Collectors.toCollection(HashSet::new)); Set root_cf = root - .getCollectedfrom() - .stream() - .map(KeyValue::getValue) - .collect(Collectors.toCollection(HashSet::new)); + .getCollectedfrom() + .stream() + .map(KeyValue::getValue) + .collect(Collectors.toCollection(HashSet::new)); assertTrue(Sets.difference(root_cf, dups_cf).isEmpty()); } From 0aa725083f66875b4eef7ad6513b24bf557eb5ec Mon Sep 17 00:00:00 2001 From: Claudio Atzori Date: Thu, 17 Nov 2022 16:13:43 +0100 Subject: [PATCH 5/5] extended dedup testing --- .../dhp/oa/dedup/DedupRecordFactory.java | 2 +- ...st.java => SparkPublicationRootsTest.java} | 283 +++++++++--------- .../oa/dedup/SparkPublicationRootsTest2.java | 251 ++++++++++++++++ .../entities2/publication/publication.gz | Bin 9056 -> 0 bytes .../alterations/publication/publication_1.gz | Bin 0 -> 1488 bytes .../entities/publication/publication_0.gz | Bin 0 -> 10874 bytes 6 files changed, 400 insertions(+), 136 deletions(-) rename dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/{SparkDedupPublicationTest.java => SparkPublicationRootsTest.java} (57%) create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkPublicationRootsTest2.java delete mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/entities2/publication/publication.gz create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/root/alterations/publication/publication_1.gz create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/root/entities/publication/publication_0.gz diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java index f9fc8a21a..82bf87cca 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupRecordFactory.java @@ -112,7 +112,7 @@ public class DedupRecordFactory { // set authors and date if (ModelSupport.isSubClass(entity, Result.class)) { - ((Result) entity).setDateofacceptance(DatePicker.pick(dates)); + // ((Result) entity).setDateofacceptance(DatePicker.pick(dates)); ((Result) entity).setAuthor(AuthorMerger.merge(authors)); } diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupPublicationTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkPublicationRootsTest.java similarity index 57% rename from dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupPublicationTest.java rename to dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkPublicationRootsTest.java index 773de65fa..3cff836eb 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkDedupPublicationTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkPublicationRootsTest.java @@ -11,16 +11,17 @@ import java.io.File; import java.io.IOException; import java.io.Serializable; import java.net.URISyntaxException; +import java.nio.file.Path; import java.nio.file.Paths; import java.util.*; import java.util.stream.Collectors; +import org.apache.commons.cli.ParseException; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.FilterFunction; -import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; @@ -31,7 +32,6 @@ import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.junit.jupiter.MockitoExtension; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Sets; @@ -44,48 +44,52 @@ import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; @ExtendWith(MockitoExtension.class) @TestMethodOrder(MethodOrderer.OrderAnnotation.class) -public class SparkDedupPublicationTest implements Serializable { +public class SparkPublicationRootsTest implements Serializable { @Mock(serializable = true) ISLookUpService isLookUpService; private static SparkSession spark; - private static JavaSparkContext jsc; + private static String workingPath; - private static String testGraphBasePath; - private static String testOutputBasePath; - private static String testDedupGraphBasePath; + private static String graphInputPath; + private static String graphOutputPath; private static final String testActionSetId = "test-orchestrator"; + private static Path testBaseTmpPath; + + private static final ObjectMapper MAPPER = new ObjectMapper() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + @BeforeAll - public static void cleanUp() throws IOException, URISyntaxException { + public static void init() throws IOException, URISyntaxException { - testGraphBasePath = Paths - .get(SparkDedupPublicationTest.class.getResource("/eu/dnetlib/dhp/dedup/entities2").toURI()) - .toFile() - .getAbsolutePath(); - testOutputBasePath = createTempDirectory(SparkDedupPublicationTest.class.getSimpleName() + "-") - .toAbsolutePath() - .toString(); + testBaseTmpPath = createTempDirectory(SparkPublicationRootsTest.class.getSimpleName() + "-"); - testDedupGraphBasePath = createTempDirectory(SparkDedupPublicationTest.class.getSimpleName() + "-") - .toAbsolutePath() - .toString(); + final File entitiesSources = Paths + .get(SparkPublicationRootsTest.class.getResource("/eu/dnetlib/dhp/dedup/root").toURI()) + .toFile(); - FileUtils.deleteDirectory(new File(testOutputBasePath)); - FileUtils.deleteDirectory(new File(testDedupGraphBasePath)); + FileUtils + .copyDirectory( + entitiesSources, + testBaseTmpPath.resolve("input").toFile()); + + workingPath = testBaseTmpPath.resolve("workingPath").toString(); + graphInputPath = testBaseTmpPath.resolve("input").resolve("entities").toString(); + graphOutputPath = testBaseTmpPath.resolve("output").toString(); + + FileUtils.deleteDirectory(new File(workingPath)); + FileUtils.deleteDirectory(new File(graphOutputPath)); final SparkConf conf = new SparkConf(); conf.set("spark.sql.shuffle.partitions", "10"); spark = SparkSession .builder() - .appName(SparkDedupPublicationTest.class.getSimpleName()) + .appName(SparkPublicationRootsTest.class.getSimpleName()) .master("local[*]") .config(conf) .getOrCreate(); - - jsc = JavaSparkContext.fromSparkContext(spark.sparkContext()); - } @BeforeEach @@ -100,55 +104,51 @@ public class SparkDedupPublicationTest implements Serializable { .thenReturn(classPathResourceAsString("/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json")); } + @AfterAll + public static void tearDown() throws IOException { + FileUtils.deleteDirectory(testBaseTmpPath.toFile()); + spark.close(); + } + @Test @Order(1) void createSimRelsTest() throws Exception { - - ArgumentApplicationParser parser = new ArgumentApplicationParser( - classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json")); - - parser - .parseArgument( - new String[] { - "--graphBasePath", testGraphBasePath, - "--actionSetId", testActionSetId, - "--isLookUpUrl", "lookupurl", - "--workingPath", testOutputBasePath, - "--numPartitions", "5" - }); - - new SparkCreateSimRels(parser, spark).run(isLookUpService); + new SparkCreateSimRels(args( + "/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json", + new String[] { + "--graphBasePath", graphInputPath, + "--actionSetId", testActionSetId, + "--isLookUpUrl", "lookupurl", + "--workingPath", workingPath, + "--numPartitions", "5" + }), spark) + .run(isLookUpService); long pubs_simrel = spark .read() - .load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "publication")) + .load(DedupUtility.createSimRelPath(workingPath, testActionSetId, "publication")) .count(); - assertEquals(62, pubs_simrel); + assertEquals(74, pubs_simrel); } @Test @Order(2) void cutMergeRelsTest() throws Exception { - - ArgumentApplicationParser parser = new ArgumentApplicationParser( - classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json")); - - parser - .parseArgument( - new String[] { - "--graphBasePath", testGraphBasePath, - "--actionSetId", testActionSetId, - "--isLookUpUrl", "lookupurl", - "--workingPath", testOutputBasePath, - "--cutConnectedComponent", "3" - }); - - new SparkCreateMergeRels(parser, spark).run(isLookUpService); + new SparkCreateMergeRels(args( + "/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json", + new String[] { + "--graphBasePath", graphInputPath, + "--actionSetId", testActionSetId, + "--isLookUpUrl", "lookupurl", + "--workingPath", workingPath, + "--cutConnectedComponent", "3" + }), spark) + .run(isLookUpService); long pubs_mergerel = spark .read() - .load(testOutputBasePath + "/" + testActionSetId + "/publication_mergerel") + .load(workingPath + "/" + testActionSetId + "/publication_mergerel") .as(Encoders.bean(Relation.class)) .filter((FilterFunction) r -> r.getRelClass().equalsIgnoreCase("merges")) .groupBy("source") @@ -159,49 +159,44 @@ public class SparkDedupPublicationTest implements Serializable { assertEquals(0, pubs_mergerel); - FileUtils.deleteDirectory(new File(testOutputBasePath + "/" + testActionSetId + "/publication_mergerel")); + FileUtils.deleteDirectory(new File(workingPath + "/" + testActionSetId + "/publication_mergerel")); } @Test @Order(3) void createMergeRelsTest() throws Exception { + new SparkCreateMergeRels(args( + "/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json", + new String[] { + "--graphBasePath", graphInputPath, + "--actionSetId", testActionSetId, + "--isLookUpUrl", "lookupurl", + "--workingPath", workingPath + }), spark) + .run(isLookUpService); - ArgumentApplicationParser parser = new ArgumentApplicationParser( - classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json")); - - parser - .parseArgument( - new String[] { - "--graphBasePath", testGraphBasePath, - "--actionSetId", testActionSetId, - "--isLookUpUrl", "lookupurl", - "--workingPath", testOutputBasePath - }); - - new SparkCreateMergeRels(parser, spark).run(isLookUpService); - - final Dataset pubs = spark + final Dataset merges = spark .read() - .load(testOutputBasePath + "/" + testActionSetId + "/publication_mergerel") + .load(workingPath + "/" + testActionSetId + "/publication_mergerel") .as(Encoders.bean(Relation.class)); - final List merges = pubs + final List mergeList = merges .filter("source == '50|doi_dedup___::d5021b53204e4fdeab6ff5d5bc468032'") .collectAsList(); - assertEquals(3, merges.size()); + assertEquals(3, mergeList.size()); Set dups = Sets .newHashSet( "50|doi_________::3b1d0d8e8f930826665df9d6b82fbb73", "50|doi_________::d5021b53204e4fdeab6ff5d5bc468032", "50|arXiv_______::c93aeb433eb90ed7a86e29be00791b7c"); - merges.forEach(r -> { + mergeList.forEach(r -> { assertEquals(ModelConstants.RESULT_RESULT, r.getRelType()); assertEquals(ModelConstants.DEDUP, r.getSubRelType()); assertEquals(ModelConstants.MERGES, r.getRelClass()); assertTrue(dups.contains(r.getTarget())); }); - final List mergedIn = pubs + final List mergedIn = merges .filter("target == '50|doi_dedup___::d5021b53204e4fdeab6ff5d5bc468032'") .collectAsList(); assertEquals(3, mergedIn.size()); @@ -212,47 +207,37 @@ public class SparkDedupPublicationTest implements Serializable { assertTrue(dups.contains(r.getSource())); }); - assertEquals(24, pubs.count()); + assertEquals(32, merges.count()); } @Test @Order(4) void createDedupRecordTest() throws Exception { - - ArgumentApplicationParser parser = new ArgumentApplicationParser( - classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/createDedupRecord_parameters.json")); - parser - .parseArgument( - new String[] { - "--graphBasePath", testGraphBasePath, - "--actionSetId", testActionSetId, - "--isLookUpUrl", "lookupurl", - "--workingPath", testOutputBasePath - }); - - new SparkCreateDedupRecord(parser, spark).run(isLookUpService); - - final ObjectMapper mapper = new ObjectMapper() - .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + new SparkCreateDedupRecord(args( + "/eu/dnetlib/dhp/oa/dedup/createDedupRecord_parameters.json", + new String[] { + "--graphBasePath", graphInputPath, + "--actionSetId", testActionSetId, + "--isLookUpUrl", "lookupurl", + "--workingPath", workingPath + }), spark) + .run(isLookUpService); final Dataset roots = spark .read() - .textFile(testOutputBasePath + "/" + testActionSetId + "/publication_deduprecord") - .map( - (MapFunction) value -> mapper.readValue(value, Publication.class), - Encoders.bean(Publication.class)); + .textFile(workingPath + "/" + testActionSetId + "/publication_deduprecord") + .map(asEntity(Publication.class), Encoders.bean(Publication.class)); - assertEquals(2, roots.count()); + assertEquals(3, roots.count()); final Dataset pubs = spark .read() - .textFile(DedupUtility.createEntityPath(testGraphBasePath, "publication")) - .map( - (MapFunction) value -> mapper.readValue(value, Publication.class), - Encoders.bean(Publication.class)); + .textFile(DedupUtility.createEntityPath(graphInputPath, "publication")) + .map(asEntity(Publication.class), Encoders.bean(Publication.class)); verifyRoot_case_1(roots, pubs); verifyRoot_case_2(roots, pubs); + verifyRoot_case_3(roots, pubs); } private static void verifyRoot_case_1(Dataset roots, Dataset pubs) { @@ -299,8 +284,7 @@ public class SparkDedupPublicationTest implements Serializable { assertEquals("Article", instance_cr.get().getInstancetype().getClassname()); } - private void verifyRoot_case_2(Dataset roots, Dataset pubs) - throws JsonProcessingException { + private void verifyRoot_case_2(Dataset roots, Dataset pubs) { Publication root = roots .filter("id = '50|doi_dedup___::18aff3b55fb6876466a5d4bd82434885'") .first(); @@ -334,57 +318,86 @@ public class SparkDedupPublicationTest implements Serializable { assertTrue(Sets.difference(root_cf, dups_cf).isEmpty()); } + private void verifyRoot_case_3(Dataset roots, Dataset pubs) { + Publication root = roots + .filter("id = '50|dedup_wf_001::31ca734cc22181b704c4aa8fd050062a'") + .first(); + assertNotNull(root); + + Publication pivot_duplicate = pubs + .filter("id = '50|od_______166::31ca734cc22181b704c4aa8fd050062a'") + .first(); + + assertEquals(pivot_duplicate.getPublisher().getValue(), root.getPublisher().getValue()); + + Set dups_cf = pubs + .collectAsList() + .stream() + .flatMap(p -> p.getCollectedfrom().stream()) + .map(KeyValue::getValue) + .collect(Collectors.toCollection(HashSet::new)); + + Set root_cf = root + .getCollectedfrom() + .stream() + .map(KeyValue::getValue) + .collect(Collectors.toCollection(HashSet::new)); + + assertTrue(Sets.difference(root_cf, dups_cf).isEmpty()); + } + @Test @Order(6) void updateEntityTest() throws Exception { + new SparkUpdateEntity(args( + "/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json", + new String[] { + "--graphBasePath", graphInputPath, + "--workingPath", workingPath, + "--dedupGraphPath", graphOutputPath + }), spark) + .run(isLookUpService); - ArgumentApplicationParser parser = new ArgumentApplicationParser( - classPathResourceAsString("/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json")); - parser - .parseArgument( - new String[] { - "-i", testGraphBasePath, "-w", testOutputBasePath, "-o", testDedupGraphBasePath - }); - - new SparkUpdateEntity(parser, spark).run(isLookUpService); - - long publications = jsc.textFile(testDedupGraphBasePath + "/publication").count(); + long publications = spark.read().textFile(graphOutputPath + "/publication").count(); long mergedPubs = spark .read() - .load(testOutputBasePath + "/" + testActionSetId + "/publication_mergerel") + .load(workingPath + "/" + testActionSetId + "/publication_mergerel") .as(Encoders.bean(Relation.class)) .where("relClass=='merges'") - .javaRDD() - .map(Relation::getTarget) + .map((MapFunction) Relation::getTarget, Encoders.STRING()) .distinct() .count(); - assertEquals(14, publications); + assertEquals(19, publications); // 16 originals + 3 roots - long deletedPubs = jsc - .textFile(testDedupGraphBasePath + "/publication") - .filter(this::isDeletedByInference) + long deletedPubs = spark + .read() + .textFile(graphOutputPath + "/publication") + .map(asEntity(Publication.class), Encoders.bean(Publication.class)) + .filter("datainfo.deletedbyinference == true") + .map((MapFunction) OafEntity::getId, Encoders.STRING()) + .distinct() .count(); assertEquals(mergedPubs, deletedPubs); } - @AfterAll - public static void finalCleanUp() throws IOException { - FileUtils.deleteDirectory(new File(testOutputBasePath)); - FileUtils.deleteDirectory(new File(testDedupGraphBasePath)); - } - - public boolean isDeletedByInference(String s) { - return s.contains("\"deletedbyinference\":true"); - } - private static String classPathResourceAsString(String path) throws IOException { return IOUtils .toString( - SparkDedupPublicationTest.class + SparkPublicationRootsTest.class .getResourceAsStream(path)); } + private static MapFunction asEntity(Class clazz) { + return value -> MAPPER.readValue(value, clazz); + } + + private ArgumentApplicationParser args(String paramSpecs, String[] args) throws IOException, ParseException { + ArgumentApplicationParser parser = new ArgumentApplicationParser(classPathResourceAsString(paramSpecs)); + parser.parseArgument(args); + return parser; + } + } diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkPublicationRootsTest2.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkPublicationRootsTest2.java new file mode 100644 index 000000000..9afe1e34b --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkPublicationRootsTest2.java @@ -0,0 +1,251 @@ + +package eu.dnetlib.dhp.oa.dedup; + +import static java.nio.file.Files.createTempDirectory; + +import static org.apache.spark.sql.functions.count; +import static org.junit.jupiter.api.Assertions.*; +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.Path; +import java.nio.file.Paths; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.commons.cli.ParseException; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +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.SparkSession; +import org.junit.jupiter.api.*; +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.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Sets; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.common.ModelConstants; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; + +@ExtendWith(MockitoExtension.class) +@TestMethodOrder(MethodOrderer.OrderAnnotation.class) +public class SparkPublicationRootsTest2 implements Serializable { + + @Mock(serializable = true) + ISLookUpService isLookUpService; + private static SparkSession spark; + + private static String workingPath; + + private static String graphInputPath; + + private static String graphOutputPath; + + private static final String testActionSetId = "test-orchestrator"; + + private static Path testBaseTmpPath; + + private static final ObjectMapper MAPPER = new ObjectMapper() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + + @BeforeAll + public static void init() throws IOException, URISyntaxException { + + testBaseTmpPath = createTempDirectory(SparkPublicationRootsTest2.class.getSimpleName() + "-"); + + final File entitiesSources = Paths + .get(SparkPublicationRootsTest2.class.getResource("/eu/dnetlib/dhp/dedup/root").toURI()) + .toFile(); + + FileUtils + .copyDirectory( + entitiesSources, + testBaseTmpPath.resolve("input").toFile()); + + FileUtils + .copyFileToDirectory( + Paths + .get( + SparkPublicationRootsTest2.class + .getResource( + "/eu/dnetlib/dhp/dedup/root/alterations/publication/publication_1.gz") + .toURI()) + .toFile(), + testBaseTmpPath.resolve("input").resolve("entities").resolve("publication").toFile()); + + workingPath = testBaseTmpPath.resolve("workingPath").toString(); + graphInputPath = testBaseTmpPath.resolve("input").resolve("entities").toString(); + graphOutputPath = testBaseTmpPath.resolve("output").toString(); + + final SparkConf conf = new SparkConf(); + conf.set("spark.sql.shuffle.partitions", "10"); + spark = SparkSession + .builder() + .appName(SparkPublicationRootsTest2.class.getSimpleName()) + .master("local[*]") + .config(conf) + .getOrCreate(); + } + + @BeforeEach + public void setUp() throws IOException, ISLookUpException { + + lenient() + .when(isLookUpService.getResourceProfileByQuery(Mockito.contains(testActionSetId))) + .thenReturn(classPathResourceAsString("/eu/dnetlib/dhp/dedup/profiles/mock_orchestrator_publication.xml")); + + lenient() + .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("publication"))) + .thenReturn(classPathResourceAsString("/eu/dnetlib/dhp/dedup/conf/pub.curr.conf.json")); + } + + @AfterAll + public static void tearDown() throws IOException { + FileUtils.deleteDirectory(testBaseTmpPath.toFile()); + } + + @Test + @Order(7) + void dedupAlteredDatasetTest() throws Exception { + + new SparkCreateSimRels(args( + "/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json", + new String[] { + "--graphBasePath", graphInputPath, + "--actionSetId", testActionSetId, + "--isLookUpUrl", "lookupurl", + "--workingPath", workingPath, + "--numPartitions", "5" + }), spark) + .run(isLookUpService); + + new SparkCreateMergeRels(args( + "/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json", + new String[] { + "--graphBasePath", graphInputPath, + "--actionSetId", testActionSetId, + "--isLookUpUrl", "lookupurl", + "--workingPath", workingPath + }), spark) + .run(isLookUpService); + + final Dataset merges = spark + .read() + .load(workingPath + "/" + testActionSetId + "/publication_mergerel") + .as(Encoders.bean(Relation.class)); + + assertEquals( + 3, merges + .filter("relclass == 'isMergedIn'") + .map((MapFunction) Relation::getTarget, Encoders.STRING()) + .distinct() + .count()); + assertEquals( + 4, merges + .filter("source == '50|doi_dedup___::b3aec7985136e36827176aaa1dd5082d'") + .count()); + + new SparkCreateDedupRecord(args( + "/eu/dnetlib/dhp/oa/dedup/createDedupRecord_parameters.json", + new String[] { + "--graphBasePath", graphInputPath, + "--actionSetId", testActionSetId, + "--isLookUpUrl", "lookupurl", + "--workingPath", workingPath + }), spark) + .run(isLookUpService); + + final Dataset roots = spark + .read() + .textFile(workingPath + "/" + testActionSetId + "/publication_deduprecord") + .map(asEntity(Publication.class), Encoders.bean(Publication.class)); + + assertEquals(3, roots.count()); + + final Dataset pubs = spark + .read() + .textFile(DedupUtility.createEntityPath(graphInputPath, "publication")) + .map(asEntity(Publication.class), Encoders.bean(Publication.class)); + + Publication root = roots + .filter("id = '50|doi_dedup___::b3aec7985136e36827176aaa1dd5082d'") + .first(); + assertNotNull(root); + + Publication crossref_duplicate = pubs + .filter("id = '50|doi_________::b3aec7985136e36827176aaa1dd5082d'") + .collectAsList() + .get(0); + + assertEquals(crossref_duplicate.getDateofacceptance().getValue(), root.getDateofacceptance().getValue()); + assertEquals(crossref_duplicate.getJournal().getName(), root.getJournal().getName()); + assertEquals(crossref_duplicate.getJournal().getIssnPrinted(), root.getJournal().getIssnPrinted()); + assertEquals(crossref_duplicate.getPublisher().getValue(), root.getPublisher().getValue()); + + Set rootPids = root + .getPid() + .stream() + .map(StructuredProperty::getValue) + .collect(Collectors.toCollection(HashSet::new)); + Set dupPids = crossref_duplicate + .getPid() + .stream() + .map(StructuredProperty::getValue) + .collect(Collectors.toCollection(HashSet::new)); + + assertFalse(Sets.intersection(rootPids, dupPids).isEmpty()); + assertTrue(rootPids.contains("10.1109/jstqe.2022.3205716")); + assertTrue(rootPids.contains("10.1109/jstqe.2023.9999999")); + + Optional instance_cr = root + .getInstance() + .stream() + .filter(i -> i.getCollectedfrom().getValue().equals("Crossref")) + .findFirst(); + assertTrue(instance_cr.isPresent()); + assertEquals("OPEN", instance_cr.get().getAccessright().getClassid()); + assertEquals("Open Access", instance_cr.get().getAccessright().getClassname()); + assertEquals(OpenAccessRoute.hybrid, instance_cr.get().getAccessright().getOpenAccessRoute()); + assertEquals( + "IEEE Journal of Selected Topics in Quantum Electronics", instance_cr.get().getHostedby().getValue()); + assertEquals("0001", instance_cr.get().getInstancetype().getClassid()); + assertEquals("Article", instance_cr.get().getInstancetype().getClassname()); + + } + + private static String classPathResourceAsString(String path) throws IOException { + return IOUtils + .toString( + SparkPublicationRootsTest2.class + .getResourceAsStream(path)); + } + + private static MapFunction asEntity(Class clazz) { + return value -> MAPPER.readValue(value, clazz); + } + + private ArgumentApplicationParser args(String paramSpecs, String[] args) throws IOException, ParseException { + ArgumentApplicationParser parser = new ArgumentApplicationParser(classPathResourceAsString(paramSpecs)); + parser.parseArgument(args); + return parser; + } + +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/entities2/publication/publication.gz b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/entities2/publication/publication.gz deleted file mode 100644 index 96dd218174dbb1fa6ab55da289dc116bbceb9963..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 9056 zcmV-mBcI$KiwFp`FK%N118{X>Y-wX*bZKvHE_8Tw0PTJ2a@$Cf?*H=?6^_^mIffV9 zZ@oPy)Uw*G9b3}MvU|_YnNSpff(RP~H~=A8?d^Bp%qoxsNRXl=Qlg~lu-g(*tV?EA zR#xVhi63u9rJuybfaPyI^5e}FO-cqoy`lMk#aD0s?2tDh9n+&U%J}m*y^4$2Pq0ly zlY&8mQJ!6~l%@fr!8p!R-Y`gLQN$s3RooQuaFpd^PdqQ!_zirGw@B%b5BD3*uUIi= zAwDUB0UI*$_$PxyF`#1a< zgAU*!e2?MxeV7;x0#+1xd^s4;FYoj5-;YmzKVDc;nvDs)qH#izTy4t|on8#HkS$wL zqX|wXq3LBwFWLMo$~0VHrYQ4C%ad!C zc8}S$;L7CFcQm>Q`KTj4;6MHeKjt+320yFVSro-dOe=ZePopwP7CY^bswqp;*Y*MR+Xv*`b%^L>VXQJA z(=e-#wx5o(G|t+`Gu9YSblD-3j{fA|XGK=#0UO_p76faVc-KB(kSkOf2l1F)Ogdg7 zUyJQf&NA3$aY~aTzP>lA(lZ>>RQfdk9$)n|O>ugPr5pB}fAXDM1bIA~thc7Yz5plV zfF|7odKHgv$fq)oXu!IM!+1Q7(@XM!7A!Bw-j{<8B#SMphmKM$hm-h;U31$KogGH z5JWmG85w8TVVsk+9I`ws3mB`2bJQ02oe@qS1}@5AKxjdd?3#?SYnJmVltaGc4xcaq z^-v~av|_|q*nu{~5op^K2JR4=hJ?XlalS*EmOzGanZty7M9eC0shB>u)=)c6}N5WYM@1rLJ`ao+r)A7@E+iL4OR z!aePmpd81fOvX8lD2!N?eit-%RxnszNOIT^#ZNE^AWIg3PR8CZa|Yi5+Q1@%vET`= zzsna5gMoJLf`kDD!zN?)aDuGs0};Zv6OLqg(IaOpVCk4#XZg2+4CocCWsqH`eEB7i z4F8?1F1!Khje`Q{G@%DhVMND3wiFg0WS42o*?=UEhoCy^?1vv=YQq^%MGM>o1kRi+ z#73N1;U()vu-Fnr5_LEiw;tIKL*6}2lAF$C3VhOVj%CDWEX7HOI2BkG2r>+kW@!Q^ z0I(MhIoN<>KL4qx0?Sx_mBl%inUe2}y|aV2Fu9Rv&jky3b`!INPmz5OSdt*w;9wkc zvEtH_j|qw`6_Y(5fD|EJQLJ*1P~1j*b>E>;%zK7)!1{)KPa(&r+boK65Q;Ih;=iWs z8pS?fqcM&~$@{?{UHryu9TO%Av4nrdeC)kX$H#9$+~Ne79FKwQ1bG2_jFYdDNSrML zQlOOrd z*NyU+r6GPBMghL01uPeqK$G|nG0@&}+Tp5Tb9Q0iE?AIB1Z?x6(+qS|z?YDvNU^t_ z`K`M5{AM;$W0R9YdYEU`$to*^n@iaJ;_uB7IKM76@H51OIQJDt|L&^VBm#(2%r}nSVbj2 zM~8=pV8Eayk>Gy$1Uo-K!9LGMmHztA5~QXal0$rwisdFE=N+(&WfAPYw3u#c{F< zMO{q(^E};GLnU;W6S=zLXqIJ}VdRFE?`VRt2OpOh449Wh$y0!TK!QP^oR2 zx@!hb@TN}e&*`@mEu=FzEsJ=Z`GV*Y5{txxvW-X96CAy)3`% zS6?Xl{!QQLDgA_9(gchkvEj=+;nx48hwyt3IKjWT`a&Q#>T}x>{j+i!fSLg#0-1J;y}+%LakWbt(`-ChZY}UJ zZZQ^kn})Orfw4fp3xjIu{K^r2d2x2N&$lx-=k~cZWT`Q`)l#;R`As(4iur-nS+C8>*5(N=+~1^9yzF- zEW+hhyR{cVj0RC5*SFK_JLS>{hUd#&&p-eCKR@P=gjahznM74r#hf1BqOD&Xck~Dw6-Pw}m22sW;#<^W zdZ;;?PeWY|e2Zxfqw*lnKxkMbD6*_e2d?dw64fBWNX2y4Nmeav8OJftj+!zu_ZEKnf`(C1j(uQAF8%4#@*PE^Y>EJe`_MYB1a zp@uoB0Uqj*1_2KtO&H1@@F^2e3a7jD9^URf%<=$^DbNNO3~oO}&RD@{9t36ntb!{Y#-m`>@qIK`OTBDQ{4hTcae|t zJ6%AsNCWt_J|Jv@0+ajv8SitwYZ$KHbq&p0ln;)gBU6OlTByAVR8LC9GnNVsq{48z zuA&=K`#)pt-{9l??yeB4q=t%J8!I~j$cih-Gl&Tb;m-p2%iI}_2>$bP#M~Muncm=AF7R8pzpn~luj2y3 zX^7v>vT`iM;l|J7b{+RZ0^3W2V=5>q^;|JL)Alswzu!~}s^Qjc!qK@7;~x9TZ(qg6 z6bBQm>J)c4*FL~@+wN+X@?Ui&cF*060+A(r=3F}zBR&E@I$+cjjwsesjbe$8Rutk60PZs!y&3I)6BV;>7F}l!xxD= z{rzFM)7#=k_pxI08Eo${E!d^8pmt+++G^Gt4j@e5MRUG!-MKBg^!OP~?f1&(&bKhX z&u1SdWBi-G|8Lr_@P~azLFx(EC5!?eLI3>FM=d>Zzt>@DXgwyF4;I#T>ddP4I0-yX zdhF4w)2uD>w^qz5%xEqSAuBRmO^XcEv>eq^sS|{eK|@V*-N*_xyVECik8_I=>?-Qnof6le2E;5JB? zwBo>8986LYUX&zMAxjW|wc?eQsyVu9+6qtizCB*qlG@&RaKLhYki|sy<{0;gmCO&( zNL_FCem8gfLdWwt>T{iM!PBQL$YGkLr*H{9;|{weTXEwVIh#NII!!Me*c?+6piljx zzgF^$)NjwmPz^CQfrctjB*}tls@v3>OJOG-)vdV6t#a|EE@fan8Ca5Ua;txLE4zIc zFn_Cky29zyn8Yba%2L?emYuro_oxNG?ky>#q6P0$cid32go`ZU zB1^c)5-zfYi!9;tEK9h^6aS7U{y{dx9qN%st$EW7(eFy?%-R9>Wx;XQ0?<2z%^w#7 z85c*F$kG5ther#taFU>+I(SJ7!Puv8Npnr#vF*qWG{taT&2)X=v4Xj%xV%?NZ}Q>J!YOu*cTb@M24K3j?El}U zA8!mi>)1x$P#sk_w}Yx!29-eErgN_7p6+^v(=$wKB~}9Oe&{gjrCHbuX(^W!)`4?`0IHQGcch;GecJ~l*n?F(6LknezRRh z)n-MbO~Z1cNDo5GfOEz$R7JC0M^s4KENVwe z+)m>)-m?_$$;mxDE^%BeEa<2rD)Da!YQ+7cv%@E3Y~7k$40aZ*u!_@krdiieXu79gvQ#Yo_CL>L8uxk`5MT z^eYvKFTikHu@&7>6<DNt=%|c0}`h7|xeF@anTo@$vK7v3L9n{Qcm4KJwtEiyOFG-@w~(Lcb#RwqAl-?m!#90(G{L zy?dD@uQzwM#RqgJoR{cBH9b}HG`;6;3+DA2_ttL}YKwwte#tNbK(At&Vh|r%q<>Jd z_uJ|I(OZn;UdIVs&te1v#E9h>(yv)g*d$vcj0=QH3CIQ|0WAe?Do(M40S5hth`8Xe zsE~uG2y71@o_#9>isXQoI3Rxc#~(Bg$=)xpdolz(vFt4w(1N2oQs5vHSg&A!3+J$w z;9q6TyF@4lQYV;#i4a-9K&lu__=ZGfT0JTdUezt|luubQ#m|fog;I6PFFBK?!&$>B zY!2LZ!W%;2t`lMme46*nI}l&0BZ37*iXsbQtf(lcJpp#%A}$#p1;0J!&Lo6x&5;9f zWpoVvLI*q?gM1;snbS#5)?CpRJA>_`bm4z0VWaquECV&r!!SIE42fW!0}djhVW-(y zmq+Ii&N4rRYsMiUuxCaEOr%GC;f#eQXXoA1gEL@$l2kIn+X~bNE)c6TT4&GD6r1Z%sf4adGCbA59bZ8-@ri;IuA&|H{6WX*sMdAEaW8EWN4l zYr`_l_-CjAc;Om>G8)1~lZa9t6|HZRiy6ts@gE#RhTDFC`v@d)&+V(KSeZZq)msj4 zvH;j78rrjlVz&v^fz<%Yq!dKD&MMv^k`E@dFhkyZ5f=!03i(-pk_OTc7ehhjxTpkj zvY)~jR>6e{)O#@Qk&`kfG(^`s2TNw8IyMm1m@q{7fNF_T&?X$QR#3!qp@0WT$w72} zPGiJxB2zjffvzi@SOE=L$^~i6QB4y8;53{HwV#0i_&1fv@^9i~tHpvk07Y0|G>ndl zy4MC;Sy5L)Qw)qtXp?MDc{r_kI8CP1;zEI^OaM72Jyzv5g!iRcZlnHlt@H-(WHg-- zJo^x*81&{J;RNnof`es65J#6KoKQI0v;&D0dH_;r!*ROKePSDrg)w?XyL{R8Zj(5p zS47TT#2TGe_KYx!;CvOoVD1M3^ufp(+u5N4#wBmME=UmmXs+p;87_`F5oS%jxFk^b~3pN-v8Zgr}9N+3XOt-s+qlI0^ z3GHrZxRI?Ix}ipjgtWYZEX4_g|sSXVd)qfbI zMWP&D5z4_;EK6~lrAQ>op;m9J2(nRG^nz^Ik5kMj>toVq9`>N^|1E;i&yJ~Tx~kcl zW&8ybYg3jVz5Chuf4_dnF2Dc%j(CY05<&*kUqHAOy7N*W{w7`-d#c%8&wh~+^_Ps7 zkZzcbyJ0H3ti>#oQTE-Bvd>>y?-yk^3{TT~rYXbhGR!W+?CXZv|9`0|sxD*gGS)6* z?K0N>Y)+_$#@c15{iTH3@8unR^pJ!M#6Q{%PzK^aFUG;}qSoE*-Ws&$sRR_H-3eST9EqE_{heto8Ia?BWH!wCMifMmqFEUuJZ~H zP|PG`iWy+yMviKaGu*=MqyCz`n1uo>fh%5*EC$x6%W$oQ#+dgm^M< zhWwM%T1u@iCm%k4Jp5VJemeT}=`;NB)2H+Ov-8va_xLk>cKq{{jz66roqss|xh=C# zdU+n=9Ch}cLo0T|; zhLmW?XO4zc71hwJj;tb7hmCQYX@l=d;wDH9TPD@-GdI#@d8cji^$Edcg2l3m(}p;? zlPJ&ioXV2rD_JhzmGnYDhb=y!`&4mKJ;U@Ar>Co~tl}iAI7zsigv&{|oP^8C0!}aP zaGFd>zAQ;X`!*k_>)a=}(F6j@``s4rcd|y4tkJZakfSF;qwNsf=%to#;u)zzNfbO5 zbFzQb*Ln(8UAhQ^{xXP*qINaznoumy)O)7pw$^l-Vq7qwg#B|!OWdO~$4m>VMdh`k zG!MErjkQZURU`DPzP-?hY-UlTszsrxhPw@1xfS#P=Q!AA6F|Su zn5k2$t6`u8zHjPIpeljpTV_N}M{9cg-kt3KN4N5g74z7KM&wWTL$nMuU)QZDRG4qL zOjE-!a3U}nf-PVlc1hJp+(Y9~ z_Wgom5r{-Ty>#@Ge21NcKuHMH%YZ<=l4I)8;7``Z@lV`RDhAbsMKb)3cF8fKA{aRZ zDnwZlXZ)$el(7}Mm^fO892EsoJLGgk)UHML z@XGX7dLg#vh1eu6@0E~JV)7&=@8x3hq#k&%9vBUSg@vD%{ztfR9pJqbcH_~zP5SRo zKOQ)?WtnQ*yJ#c_@58}(5${m3aI>t_O~ue$&3)V=Zi>c}Zw*hkJXPr_rn$0+8-i)U z*dMYm5c1ng;{m=IRUlp~+-qYwhrS>rTbZxSV6h7#$1pcs*5`Zm=?>8;rgZ>WzsSSw2F#l<9A6a}syGS^iiO>s>pP~AvT zEnBf2O;=4l5+QjBYkd^}sY$F%9t$s=(T4%I_$8xXI-`$x1V-}uHSqesGs);z!RU7a zW>82@zjRKYR{l3u#P{L01h2m)@VcehaEI&2SI$q6?%RFo`~>N~_1DhNkFpEzotNnO zDsvpG4UX7o7}O)lMZe}_t%2p`Vg2C4T6)W$KL?-2TQ0LyWR{A|QhB;8l?}!-Ix!{xFGSWdHS&(lJMGUjW#VH>)lnu_L{hGqD^;y6@sWwy^2vVEk#M&|SE4k=|m z&zZ7fx4m&lSy66;IJxtBOO1BQP6#tjv39(!VaST6uNtn$N3ncqJj%Gfhvk*R3U9 z>%Hz=F8Yj_uHpDr*I~NdH5@JMI!r62X zg*u^2O-rR#U>K1RvPQ(B^=0Zn<#Z@9VvMM~z zeKRj4^y@W2zZ64P&H1{euM+z8D9A4f{Mvco*9LTVXR%aj{j6na|LzB34Ggv!Ov-Dw z)SmM7-q4+w`tUdL%GguQ?s_)S0*XxX@a2JrUOP`0JWi^!7DDLZI(rwc8qW#-gRj=M zd4*XGZkd0b9NR14KT@`97@w9acoZYV$8c3gwRN>W_0C<;G`yI2=Pt77qVmpN2=Cm* z%sY3%y>k~+@7x7?=Pst+Ir!-Uy>l1BJJ%b9(Hbx!!DtP*Ye`KHz6OPm&9<%%5QpQ zi4H)RX+NYnTY|!Yj!n$v(-qG&)3nk1}Wx2Pmi>jfdTKA#F&gKmJTin$a?WSZMxD_ZQYVh z5|AYUS38s=@DhZ}~v@=D5spiMuIWX0&p{&k*LO2pH^&q?y{L>@HA3l=3k1Rx4e=EHr z5+t=PkkqI1{j>Aa{r3_u^=Mum37C?AsV4_aiRWNkKMNUI2PO6RuES34WirptP2U3S z+Evif3i6^;sgl*kv}&I(*9yGBN8aZG_ZsKMW!BjW2LTu$RKhWq*mgc>fE_jy^97}ATfMGV;7vz>%~37C^O z;#Oe2R-?RrJ^8e-_+Mj|gxpQX4fB9;!{nR=1MzBM(qz(8%7LPjVFgLUISv?2#sgA_ zY2WLGsZuDOzfc&2UUFx%FrT?uje!%cDWr6~cIg16u_z8w8eXR~=pYU9*pQ+i6oaD5 zlR^qaO*eoo5h8Lq8)1S4MZ`f>AX?^{tetD3nC5HBH8CC2aBWp}RHw66uE{iOgw%6N zHlAuC5=Tfar(~m8)*K~12Rsg*(0Q$Te9htlUfD|nWmR)IOT*Vk&sz( z$qTL0yZP6%Cvm{iLOALdV4>VW8(%Sjf-3kcIjb)eegCF!^py6Tku2tISGc$5fhIpH zcU%>c5nlc>jqg*-z3SKV744#3%H6)0ezT&w&9Z` z;U1(jrC4r6Z%V=3kh&a#U>4HIl#22x`&&>vJ-gft8`{;`J9o?4C&0+ z3Cc=mhID4^G;yVk{s=l#iluaB?2uqeXNFXiPcM(p+?lbyf8xG5CmrdaxSNAw=3{E9 z`lZ-iO+Qx*Pc{YnP!(%t^x{zUYDg{7r8~#FY86Ce S(Trx%jQ<63B6;BU&H(_01)C@U diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/root/alterations/publication/publication_1.gz b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/root/alterations/publication/publication_1.gz new file mode 100644 index 0000000000000000000000000000000000000000..89ed9671e145d630b3093b97cd8b5c86065e1fbc GIT binary patch literal 1488 zcmV;>1uyy^iwFo9Lv~{T18{X>Y-wX*bZKvHUoimfSZ#0HHW2^({z^ zC0!SHNneH)MLq@Y{zo!04;(o!>Vs0@5qPu^6-v#4wsdNKedy5 z`h4JIOtZ5pmrio$h-xkxkrv=2IV&1CaB3yzP_c>w<66kdNb!O-jmVtDX%>wrYpHb7 z%{HK&gKoZJC9dL}GBt-rLx$rGpFs&$F{?l)_iG<_+H0Am@*b(L2aZ;4gZLeKI6@^+ z<)9Squ|l0e0k~(nurq+VPJ$|cnhF}(Dh_&4L;o<8Qf~|WvC@L0HFjWc(?W1&09~z? z+zo%1AmXUaTA%KmKU!9ZT!89>$)woA@=8pFW(Ap~*T6M7%b?Po;tq5oyD|sdW6||2 z;+V6dH8UPP^gKE`x^DCzaOhJ%I2?7K^A)4Sgp3kttEK9~CPl9@&Y@|Pn9g(`{p9WV zd<*`hhKd|pg&lyqm99#e?ZMm=ux>11&MJ-33y;~6D_~vpZGyQU-3DT^;Y4V-vJ3A* z(wEqB#kb9Ni1Z#zQlM%&nEEw$tU|?9&1aT#6XIrAvzfBMGux$J z%oy{sETTi-I=Hp@(giJATDP^aix#B`;IeiSFOI{QQa_~rAUtqbt7kUI@0K}=dU;zE zt2AW!BFo6+7Ap4~ZadOy)hfz2hkF|MuUK`1+ZofWvj6`=zIF;#_auc>961T2Boozi z7jaxb156=2MOc-QlY2HI7$8qVfJpz;HFtm;2VU$(ejM2-ei|2XlNh(GxJe9c>xF`; zXtY}V;~w*#(q+%dRsA%5Et^@($c0Qn=?49>nWdYfJ^6K#`ahENStn`WhT$l1N1-2Y zd;S+>_iJ(r*BR?MeKoQvw)S`M!{5)^-#+>~9J(VKgg;36d6RO_RsGaFuC%O#B&V&? zER+58T#xmI*C`s>GO)s;%~}HU6zfd1i?P^N8%MWH@c#Vg^U25a9p_Xq7jS!py#y4y z&)S#3^Uh>ry<&DW!^kQJF7Mp2ZH_6eFZEwUJ zxGYNywnM3=M@#YMC|w+dhxDj`DJwX(Q)aDJ6=>15J{H+l=aHW9^M9^`HC4aNO&?h= zL+Ea2XY`;kdg8vw`TnliY~TG{$W~jAFH$9T@zOUp)T_+;ci%I9du_OEG(2;4$8sL^aHTUL6cvKc>IkGtYPC zBzjvG2CT{NrZ#IETF2ccRUs-HqER$-!zeVN0ak(<+n2zQ2+h~fdE=WH45OfU^jWNA7{r}a->|)jy>Cx_ft`;}oyZ0~|8)8n zxxf>Ub7+xQrK)FwleaTzu6a%_YjJ~hrk%x*JqLX&)eZS5^o%faiBkoT4`7Ig748X} qW4%T$TZQ%jcU%gsjrJSXV2ShM!^!3N%}X@e*S`TS+)w`P761UD_u|6< literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/root/entities/publication/publication_0.gz b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/root/entities/publication/publication_0.gz new file mode 100644 index 0000000000000000000000000000000000000000..85706bc31f8d9816f211cb79173864c530081e30 GIT binary patch literal 10874 zcmXweQ*{_o!VQmb}#b@%G0-qlSU z0|T@5;bsB??&xN0Yh_~OYUN-La^?5#vc;A3eY4jGmRh+ZYyAWQq&hXP<7-U@uGTf> zWPI-KrodAO$H0nPOD)7(*nhp)fTYO%HJWNQmBlJeiegalbmhd0yH@1<^5mrcq+s-> z4>&Cl+|^$yHoaZq&HQHV^?$K|?}0y0Bq~;>${jm*(;N05+>5eQVfHjiv{<7rvFf*E zj%`xw#-V7YK1}3P-^OwZNIVu*6kuAvCG?f={=L|HcFj6dr~T8EG(ZJ{f7RlC%4jlt zS09&JGGL8n7^WJud8v?aY>YH$P^UlnS z2T@IZ53J+ReL@MM?`mO;NXyZAp(+-!NSET<6;#sTh!pKLON% zaicu=NIH%=%AtE$eew5RTjII%tS=(I@6;I|V+j)>OFYzA)#n8zI|(H}Gxo$sw4HTN zljGGt`wJ2w9X-MkE(RY%dVfD$zkV6rfi7eh5LMs)!3rWf56KNl7jY*XLK9O?D>6zb z0BMG!hAJ>n!YW8fy|qt6_I3({>WNil7s{7t>}G23-#WAba6)A|j=zyC8cobIA0du% z%S;JOH8lqIMsuBR1_xO5M<5eoo6U%PIKWe4;S@Tqg_y?z5(nLZhgMK{pKUEY-32C4-?WtgY zk}oiRqVVN^u!kteSAe>-U#K;cj#TzS;X#;n=mo`jjf#>&>ahs%lk?zE)C+~4^`jLX zR>8g-nB=NWEC3mFd@u6BjWm^K|E7^(XcA~UaYE*m&>!ZA*W={x$n-r~672@|+DLj$ zFD3@ydxQ2CL&{;73MB(QHWuDrn~$1;M>fLaw+w};Q0r$RBv|L=g8(9F&e49B@{B7l zc>{OxQ9b@#Q|D*>&{xm z5TZnU%KkOi<`WO3QN431S;OvRR0Q;nLm5o}=tDXHsAVm<(yHNpSRs5qwyl;pr6BbC z`@3Pkqb%9M#W8_~e?R0q8NS-vMW0M!W@EzhYjUj{EEay&DJQIq94l8d=2pu(=tu@$ZPCX- z`bH#8*bK)v(X^tvMzHmTi-PDL>LgkmFrv!30j`Vm$j5Wn7o1-Zf`SqJaX+0ga!{hh z^#W|SRZSKTHV`>!({!%j%cHU&ueFFV0ek!YLVDy1XNTKN0=P`hEMJ! z_&oLMR1_iLnU2yUpDY>C*Zj4-Vn~5W!!lwW*#32I&AF&wyO{pN7#8F-HWXHTN(b~1<6sw_wEZkNT)ojSE*ypgjq+J-6&})wV!yw zGLh0*^Qd&D=kSVU$D;H?tF#W1V&08)xh0W>!ML?9LqfFsMaULx-M4tkM1Na=Jcs2X z6e&7X4$&mS!N~=RM9+UQ-uEn2;9*6Ot_-vsmzj)fiKres)QzQDL)ruo_v4`ka$yX7 zG}yC@)K0~ae*R`LKa;oDxxbH(zjO1n3ODQFb{(&A8E5BHr8VU)4r`152)wE(Te!0; zL6f_2(hIWmsX9yt3S+34J!EgU;=Zzap!j>sx@pYoAJer7x|v%9E-3 zX6b`AL{zwi;`BnH!m`SU!4Q#O=I2o6iS9T!4=4;s>1*@;vOu1(%*wsl1N{ecGSQcT z=-~4^ec$0}AUxLN;$O^10cxIiY+@oUBpbdObbmm&7lOCpdcW!LLFf8DtRElu%e?)u z>2Joi&59lvV|x@#YkwR5K2_2rpOgw z2c?;ZQmXxW@pKpM(62yiSVg2muY`%pmgww8A24Ip9ZdXeUjzorhmj7DXxA2hJ~LSV zGN0C^+o#`)*qf(r8 zM}GcOehOR>LAM#^6s|-x7b7j|#S}NUdeDHSpZouG*egga3ZqsYH(o9+$pCvM91J}E5t(N!;J{=CNuq+4EqS| z)6$5yN)c>Qo-o#fN<=}5xIroqJE<*%ZiOdN<_dbn6q1B z+(fOu5m&zu1r%zp;Toc?%h$si8p#8z`GReQd>-InVe}VX*Vxtg; ze_YYiMW_(ifAZzl9PLUO`q(?h1}2eVuiwgTE!_t6Rbi+N)og_p!4p#PJs0iuPKMEN zQUO+IejSLsgKXJ#)o26r+$f)iSEeD!WQWJY+^L}op&Q(U>VAN!(+aT2m)QZr6G^G| z27_7sB;e!Od^wSf4Vv`Kb0hbRR$~+X#_rg09u8#37J~1)^3hWoc-GPt8r_>X1F+&`4H70?onJn zrk@@t5J%(lk3`Sh82T_-4a~+t&TyEIIqbBI@+OwvG;{B9UGrh+d%wn(Zk$fapBjW{ zPMPibvZYAL=48JCm&`W2lKY2oL0e--<0d|pX?A+TA#b5=iyRq2TZu*6MgpNsvhWZN zv$C?{>mTKK z{+dOS^!pIsdyRQU{i?swOXNXwhwbT?obO$K>_{W(*t@7_v*)McBc`veS>~maD~&$VcDB^JLv+1ak%xTx%-7TT%qkDl?E$VzY4g#);c{}$2M`Nw<9h~ zusRvC63EG*?WRzuk;c5BQzq-P zsjOwy2+r;ZtlXN$zlj~u-9B2WIs?qlX$VB-LWk^URrI_iX74@vG?w6@ImXr@&Q$u& z7*qI)9=AquDoGRM9YryzlS~J=R_ttNO6t*&|7vWXsd4+MXtLA;5T++Mz2^J6l2&?= z0DbR%or~woIKwnjlX~mh(K($>bIPjzJJ2dpX6R7Q2u&tXW&E<1nByj1MXY!#M#-86ZsZ21#ew{l;3!Ip^ySIy2ATdQBkFm7UzG4HgN392|+sK8xGNizrQ&@^5jQnX!R^a*ua-*aqjknBb^1E^O$@F5CwFz z-Bz36>xXcOacYKC0-#g(+vQKeE^@EE!# zk8%h3v1f_oIQS=C$k@B{Dq|709VFD)K4AAer$NniYVpV+!niXBBHoIv?<$2dph0+3 zaBVz1N8ww;#vRt?*)5t8)vn=>OK-uDHG>H7!V8tvQzXJXoRP!Pl$*##7%+*CE)!mWyrjRLMIc-FVn z$zg26_Bgw0UWj+RR(5{lP9orPbm4rswsmm^62-pl_CAc*$f~avn-b}udFJmtNJ3Zm z5J2q`R$3gWOq<=>-gTI?yfEaQVgt@7zU+Ji1Qy;G)W$WmfxWDoT&~ z+L@B~3Zzaz7mK(ro6u)Wa9~gLW-18!hkkG1))Qnm-j7GD29Fa`%TZj*3Ca9b+D4DM zMpGXb4+~%*QP@M09m*1UX4yHRhdOheot|zU-7*&B(ns_i3`kGB)no|pR^p9U{6;YE zn!tD8X*-K6=tz5d0OaSG^W*s%y`*hQ4eBUlgpBk!lsG5@r9G{J7c4l@KOue{7*|O+0#mdL%tzobWHlwQ zCRRL|U~-@WJ$68hK)YY45(fc@C2pp}Ps`O^seh|4U^amx$huFRQmO4Ujw?_2jHhAb zvfqrpO9p3=lQ`Bt<~3Ic`dY$6rlnF$-AHnRiAmWa+0Q5tMMvd*utx-EffivepBQp9 zd4{qX1|`N7{s8I6F&08F(^1IafRKi4KlBY_90Giy2WOu@F!Dsf#Tel80TUA&(r9XE zjhuDjn)-uc4+sqG&&0)GPDFZ>X2vq$?y}6g&INr)E>1$ds}=LRjdG?)#5o!UCJ=5| z#kfHDgC%7X-sI!!r~w&3X@rH77L+&N@+BX1N$~2Py+uC@D;MbAsBkdJVMOB6vhj?y z7&g4}f)a0#erv%%;yJEIdEy5lp&im23TFO`xUuBTflHkqpLFj!rDlB`eu>ef7UyGb#R$h1`?Pw=|W&YAP90xYu+_^y{0e7#^$SkOS3q5hZ zw`qgeCNph%FioydsYBO6DN&VBEXr%##go4p3aOG~)L=Ue2rCw>!?za)u?>s=O5$du z4I-#1V-DAbF#>xcLf}0so<>FNxX=%zWL2}nCs+Y6$KQjlpq1szLm1cO6U>c3@=3{n z7CyM(xqMJr$iP$1yN03D?Te?u3FE=3ElGhuAdZP#S&)Ey*- zd7nmivcfkzE9%1{;`1>U#5MA-oHXyBu;DEtV8ij5p^bA}H_*0`_At#j<3HuulyHiX zhulT(Zy06Y=eGePk@qJkxwJGOoLtohkw$#*Ir9@m=7iuFxE0pCWo(a1m?dGN1#q}~pc9A(g}PboCC3t_hnk%4T!nnL4 z=cCW9K3H<&tMIG9YTE4%^-yv&*;$_`Lo~E9Vs;!s%;li{510_}n3$K?3HGdc<=VEw zi^A;2V6WUQ+keX=?2w>pB&IQ}6B%3){XFdEDIW2NG#3jMp`${osC@-&v|?!u7ZeD8 zoC@n;Od7=~Au+bnUvkL)rcM#9+0{Tb4QV`s(C))8<14D3N)~vi2fz7}L`Cp-l`A|K zYhs@Q(kWKUYx~!~6g_+k5}9l6ecwrSiy^`m1fM&QY{SiXRUikT*TwQ1w_-Q_QWN`R zIHF9!H$S%rOfqH}rBzVBze#TVK(0}YX~YjNvm#tlGjhi?*ED(-AG-WjwFaEguGSh~ z{Rj9>qWqkzwIWvzl*?ZfH)k;s6lx@2BMO1ED0N_WRmj6k7is;Pv@{RP@Wft!jtivT ztMmbl(VjjG7>TCBr?g$y&rS7b1{V)xcX5?SZ`IV@Fi_L|K6ku)ouo4x+2dUkD{2 zj5LaE(lC|fIpgMw0l3O2;%N!&(KiXr_7(A-Ks04_LZOivPG0+K!)pB2EuAyC1bq_t zy1iWy!6h*+j2|~^I6Zf@?#*`HUE`EQ5%ga?e|qDF=;b#!ITj?J+IPjTM#O7xK}KRl z!u`SCt=2^(^J!Qg;osO6561gbok z{j01uJL}H#6kptLwu6$lzr0YOo=O$*nKZ7p3O~Jz4$c+N_UoW;G9}cXI-7pJkTMO> zTF$g5dZx`#$udk=S&+^mCiIz#;-baiqS@f?{o07TcZm1HSGhIZdVNWWhMHRFzi_13 zVG|1T6GoU@=Bxg+Yld9|#gu8an0oHX>Lx}uLbhoBwRhT8tn!^M^d?*I#dI}GVykE2 zhipC4J0B>UR)$NMqXnO@fl-G+ZzgLuSNwAT#U?WJ>FKZi2;x~L_gO{-4}rV9gQg?mG1rfS!~H|Y?=kHRzsacw z0;V}WzLnM`STSpFqlNv6+)920OiGL4MP&vH{sIq7|L z(n;<)11R&t@vG4o%#x*k^2DpW=%2T#$oXa5JZ`;R9aWebVRX3}t* zZ68AV7c>KLR47V0HyWP3s_dC+aive81U+bYc5(X>4lTaP<5HT|Zy_S---hhIWEhe$mx)6+sND9Us)5BXk8TJ6QE zE23@ITkvt?=B39jOn`&V+MD(I8Y_RX7HT%>_TBzeXkp~~gb+Ipu*dh<44x7mSa610 zKUEK~VDOo?>B!9Z)kok}&gF+i*L@sj@_K~CAH^3@dtA%jT3zMH42O0(aw8lrQ~<&N zTs`tdu{FkZK|@89C7+-S{g22ZuAbIx0abec3wkWZaG{!G&bhO16)yHL)h_HZb1lY( zEUp&w&nBk@el?~EPJK9etV0{Mz2ufhrJCv8$F*yQI?bx&h`%Mz#m61XZ{9~bUgZYT z4jYiq)=&5I?SNfX_{$Z1EVui4{m*VI5nX)MO8b^J9GOd_hxT>9+jgLh%>?A-vP6}V zV)V`uk&_&jFdxI*_kv1)86{ZWi6;~>q;KUWb3Q0CKH-d`288cs>r=}SHdss`#@l&K zQQk74I~-UEiIM|2SPSLWa?C(pb;(g)Xo=cmC3uW6ko+piBtF{k45J*yOgcNN3{G|Q z+Uq6?&^Q(`p{p1L&uL_0EzajHK26K;;sQk#zbXPP9wY^ygM?zGd&V!w;-_Q~nd(KF z7B#hnra~*1JNTzZUoLkMd~Ovy#Gje{ZfQ|47=4XVFn!RPZ(XoE^MF_QHEmJrJ{KkEaM(3_{}08{hMDFT?mzgV|eg5CLxFylwV;Y4UqC z8c4IPvHxTTR;QbZXwjb5>Fgis`s=FXfxP&AvMwj@(v0vrNAv8J)+JjLpP*h?uv{`J z#(ri@2Yjk(=4~UZB=5Q{G{FVaZuPLYr_3nxo_?hY02cNmzuyFeWrb)EBm^j zZvIIrXpP}JPR))u7v5)Gl_3=uB4NKi1YHFz792@_PL35gC%~PjK#~F)xXZVq> zUBtQjmSc1M6<5Iy5+Y|~Q~^H4paThb$S8Le3#AD0h50^^XQ(u&0=;v&8KwnnC^9yY zoeYbWN%i#jcqERzkbliAQ?y)xc_E`hSHPs!$_ToP%bC1t@Q9E7K}5K(74|I*F$a+B zCfkPg_Cz+$1o@%kv%iMs>S5EVX6;vJX7zH1Z;Xi0N+37cs%u^6)x*hlS|acv-_7zI zxnVF(l_~O@hNzeScE0t&b@E6oI}Al+Zf)bjae18rh}@^Mg@tL(&Smyx*KD^3{;k}o zKLniS3YPpC0mNZ+imXY#Ug!0YUb=PulhZX_I5o-ZLDS=s?y=h;dyvyb;B(5@;c@MJ znG=%tKceD&$`it@10q3zTLYSIhnz*2MIT6}Qmvsx3W=%mf1%)1IBa>8#mA%|WMJI% zBL2n6?G@-fAeM^`ZpAh! zoL!K|*BN+h`7K%a>FDVH1?j*4P4vxGx`T`jjUDkWC_uPaOwGR)!-m|}eD$cZPXVdm z?XzFJ7PuP-*7om0L)alVC3>X7;CD^JLDGJ;E2HlKN0Hb>w3Hwglgp}smP`(HyTD2nvF;%)UQKT@xdHwF z`keW3-@@3tD6~noHaR=e2(4zFS}mnYG>gk}&kT zG6K*?-sGmHs42@+T}ZkuQ>Gkq@`B*&Ew%Y2KI2n4GO5)Z;p*?cy&5Q2g83N3yf6E| zZg98C;cV7tO`p~>po62ZR}GuN4cLLX1MzuvrqwGFuTuGV?vXc^zirNO-mzK$MAq>q zOG-b292!pCRJwfYrnM<(Sn8xBpyIC$XI9y!Rw4<3IPu(>=v<&BwZoM=;_OSHJ)dDz z$(+7J_r&_Qi1KI=62Gsr%>U@d+u^2(&&=P7O0v2RImtzFmk;c5p#BSn5ldXL2i*c7 z1>qsbs*E;C`rgVMaiCeAt!eX2ur5iu+x%K}@5%g1?+Dq?=}BhzgD z;J+P|QwrP_W{LSu-V;fx3SwxmowAHxMe5$^0sKB_(DV$&MgDMDv;@s}?8(&SB6<_r z|Nl?O;xyK@!SjXa<(XpsPr@XjW|(PldY{};*|Kxp=F^8KGhwW?fbNvv;m=l4N__Fe zV|k&DjRv;M%eI{#m(99TLO;h!!8iYa3Mh0 z=1l#bZP@J&Eow)bKZQM`_e~<(ZgMR}n<_bzrv%8Vr2C;vi{NI+DEF4~-HXmc6(ySw z%H{?99!NGKo14i#9!l)XF)N|BcT1&ga>8>UsYt8au5ovP5Z>y`Iq%<3orCa9aSN{A zBaAZ}l|iYfo6Kl9^poIbO<^@*psvcs{a3FJt8=`;pUF|C{VODbE3$F_?bgd@-_B~{ z^gXQVqM$P=e8&Ynl9OO1vBG5A#{LP0PxMm~bL;}MZ`k7pylh;@{mrlQ`dcx^^F5Cb zkrdw77yB#G*TMZ88+})4q$H*sE}#zsYfGl|lhBHZ3g>o3p43S<=*V*UYnY*J?rwhdW(O4~uo&iyr`kd+Vr3ai8;ZmO!YWsqJn7){!S-RxJo3RU`3S&bbrXX z)6nlg*7Ch~yPOSFl%8R36S1&OHP=muykyU8aW5joqph(iw2v8D36%cOcgHekBi?Wp zz;Tj$aEriQh6NCcNtnK7mSnUZLm-}RS!6c9(1LNw_L>&ZzZ7QFo_O)Vme=45=UWYv z_Qu8a=PyTz*c$J$2?Qrw8uWRZBTpnc?3r>w4>O#aSbU{T7xXKYmxPsw;*6{D+3w9a!IyjiU z`4t6mN8JpLlyObUG{{>W@=KI zLqGMfl&Nt0JGU|8tK&Kp+wPTfs}k|g9X<@RnfD=YM0w2osVeW+AT-M$P3)?G8;lI zMfh!~%GCujP{j_|0V%HiI)tjX?xp-HO44GC2%-tnUZ<<;+#7yaTCC${{Nts_f&^d7 zzlYr(MKKTN&|U5ke=S-4b;C2Z;&kb3o;+oGeQAz*oI=|0@Ufdkn0A2n9Dzi03oMk|*qgrEPE`I3K|g60nS zl<1!BeJCy#>djJW&MM*ah#AAsB1FiZTpPk8XdgT1r?n9_W3a^&RbL&XQq?eHs1{7% zptKq2?5{2BA|fS}B%hnsDF<_+)iw$;h0NM$wR#Ulje84nxWIbSx6YElc*Sb4Vr6D$ zjphE~aFC>xHmR$`MU~o?FD87GWb#&FUzsk?5lTlJ5cwEuH?!84_| z3Bfe5ZL4^}C(P<^|C28T$y_TA>PDlRcp)&f!;JNRMPNi@<|1nN7Z2mVws~>lpA%-0-S=A5M23vpm;jvz zTN>pyRBmKTl)V*UJgWeCg2Ibp)KZ=#+2*_vZcd%WpCsb7PQ6@J)No_f@p8ESPkZAJ zJU?E9$^Xtmg@svHY(rDR(Y^>f=~IKGzULPU9Shk0yiva%dcHUKM{$SLuR|vNJUh0b zw-(to9V)gX#`ZRD2fqr_WRll`c9JF1-Rfyf2c;3|Yd>s`I`_(Bqc@x~$IR4(@H^X0 z2JcRfMS*(eMUXXr`^P0?duL$klA1g%4RH<=Pup=?LjhXFE+<-|HN-)|z`- zj9Hmub!ufLFU;r*5zz8(I7&#`I^!6xx_jEz^p}5h3a9p-R?T2!>R6o%uwJ{Q;)R(o zTjfburz~rxY5Q?f)JV5WJC&EGe`l>@W|#enpK9`x*ZE;xmo6VsgwI%rm9SccUM+gW z-BGGUtJ^SoWH3lP)wn=~el;_ic0{B^%N8O`J$eLhvQef%i3ZCdjrc#cnrr0n8w4Yn L#g!=;4CMa+cETas literal 0 HcmV?d00001