From 1e1aab83e30085b8763d9452f47f77fac4ed2c27 Mon Sep 17 00:00:00 2001 From: miconis Date: Mon, 21 Dec 2020 11:58:21 +0100 Subject: [PATCH 01/10] implementation of the raw wf for openorgs: still not complete, some functionalities are missing --- .../dhp/oa/dedup/SparkCopySimRels.java | 110 ++++++ .../dhp/oa/dedup/SparkRemoveDiffRels.java | 233 +++++++++++++ .../openorgs/oozie_app/config-default.xml | 18 + .../oa/dedup/openorgs/oozie_app/workflow.xml | 324 ++++++++++++++++++ 4 files changed, 685 insertions(+) create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopySimRels.java create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkRemoveDiffRels.java create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopySimRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopySimRels.java new file mode 100644 index 000000000..4409aaee7 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopySimRels.java @@ -0,0 +1,110 @@ +package eu.dnetlib.dhp.oa.dedup; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.DataInfo; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import eu.dnetlib.pace.config.DedupConfig; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +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.dom4j.DocumentException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Optional; + +//copy simrels (verified) from relation to the workdir in order to make them available for the deduplication +public class SparkCopySimRels extends AbstractSparkAction{ + private static final Logger log = LoggerFactory.getLogger(SparkCopySimRels.class); + + public SparkCopySimRels(ArgumentApplicationParser parser, SparkSession spark) { + super(parser, spark); + } + + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json"))); + parser.parseArgument(args); + + SparkConf conf = new SparkConf(); + new SparkCreateSimRels(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } + + @Override + public void run(ISLookUpService isLookUpService) + throws DocumentException, IOException, ISLookUpException { + + // read oozie parameters + final String graphBasePath = parser.get("graphBasePath"); + final String isLookUpUrl = parser.get("isLookUpUrl"); + final String actionSetId = parser.get("actionSetId"); + final String workingPath = parser.get("workingPath"); + final int numPartitions = Optional + .ofNullable(parser.get("numPartitions")) + .map(Integer::valueOf) + .orElse(NUM_PARTITIONS); + + log.info("numPartitions: '{}'", numPartitions); + log.info("graphBasePath: '{}'", graphBasePath); + log.info("isLookUpUrl: '{}'", isLookUpUrl); + log.info("actionSetId: '{}'", actionSetId); + log.info("workingPath: '{}'", workingPath); + + // for each dedup configuration + for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) { + + final String entity = dedupConf.getWf().getEntityType(); + final String subEntity = dedupConf.getWf().getSubEntityValue(); + log.info("Copying simrels for: '{}'", subEntity); + + final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity); + removeOutputDir(spark, outputPath); + + final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation"); + + JavaRDD simRels = spark.read().textFile(relationPath).map(patchRelFn(), Encoders.bean(Relation.class)).toJavaRDD().filter(r -> filterRels(r, entity)); + + simRels.saveAsTextFile(outputPath); + } + } + + private static MapFunction patchRelFn() { + return value -> { + final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class); + if (rel.getDataInfo() == null) { + rel.setDataInfo(new DataInfo()); + } + return rel; + }; + } + + private boolean filterRels(Relation rel, String entityType) { + + switch(entityType) { + case "result": + if (rel.getRelClass().equals("isSimilarTo") && rel.getRelType().equals("resultResult") && rel.getSubRelType().equals("dedup")) + return true; + break; + case "organization": + if (rel.getRelClass().equals("isSimilarTo") && rel.getRelType().equals("organizationOrganization") && rel.getSubRelType().equals("dedup")) + return true; + break; + default: + return false; + } + return false; + } +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkRemoveDiffRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkRemoveDiffRels.java new file mode 100644 index 000000000..030f3b783 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkRemoveDiffRels.java @@ -0,0 +1,233 @@ +package eu.dnetlib.dhp.oa.dedup; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.oa.dedup.graph.ConnectedComponent; +import eu.dnetlib.dhp.oa.dedup.graph.GraphProcessor; +import eu.dnetlib.dhp.oa.dedup.model.Block; +import eu.dnetlib.dhp.schema.oaf.DataInfo; +import eu.dnetlib.dhp.schema.oaf.Qualifier; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import eu.dnetlib.pace.config.DedupConfig; +import eu.dnetlib.pace.model.MapDocument; +import eu.dnetlib.pace.util.MapDocumentUtil; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.graphx.Edge; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.dom4j.DocumentException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Tuple2; + +import java.io.IOException; +import java.util.*; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +import static eu.dnetlib.dhp.oa.dedup.SparkCreateMergeRels.DNET_PROVENANCE_ACTIONS; +import static eu.dnetlib.dhp.oa.dedup.SparkCreateMergeRels.PROVENANCE_ACTION_CLASS; +import static eu.dnetlib.dhp.oa.dedup.SparkCreateMergeRels.hash; + +public class SparkRemoveDiffRels extends AbstractSparkAction { + + private static final Logger log = LoggerFactory.getLogger(SparkRemoveDiffRels.class); + + public SparkRemoveDiffRels(ArgumentApplicationParser parser, SparkSession spark) { + super(parser, spark); + } + + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json"))); + parser.parseArgument(args); + + SparkConf conf = new SparkConf(); + new SparkCreateSimRels(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } + + @Override + public void run(ISLookUpService isLookUpService) + throws DocumentException, IOException, ISLookUpException { + + // read oozie parameters + final String graphBasePath = parser.get("graphBasePath"); + final String isLookUpUrl = parser.get("isLookUpUrl"); + final String actionSetId = parser.get("actionSetId"); + final String workingPath = parser.get("workingPath"); + final int numPartitions = Optional + .ofNullable(parser.get("numPartitions")) + .map(Integer::valueOf) + .orElse(NUM_PARTITIONS); + + log.info("numPartitions: '{}'", numPartitions); + log.info("graphBasePath: '{}'", graphBasePath); + log.info("isLookUpUrl: '{}'", isLookUpUrl); + log.info("actionSetId: '{}'", actionSetId); + log.info("workingPath: '{}'", workingPath); + + // for each dedup configuration + for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) { + + final String entity = dedupConf.getWf().getEntityType(); + final String subEntity = dedupConf.getWf().getSubEntityValue(); + log.info("Removing diffrels for: '{}'", subEntity); + + final String mergeRelsPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, subEntity); + + final String relationPath = DedupUtility.createEntityPath(graphBasePath, subEntity); + + final int maxIterations = dedupConf.getWf().getMaxIterations(); + log.info("Max iterations {}", maxIterations); + + JavaRDD mergeRelsRDD = spark + .read() + .load(mergeRelsPath) + .as(Encoders.bean(Relation.class)) + .where("relClass == 'merges'") + .toJavaRDD(); + + JavaRDD, String>> diffRelsRDD = spark + .read() + .textFile(relationPath) + .map(patchRelFn(), Encoders.bean(Relation.class)) + .toJavaRDD().filter(r -> filterRels(r, entity)) + .map(rel -> { + if (rel.getSource().compareTo(rel.getTarget()) < 0) + return new Tuple2<>(new Tuple2<>(rel.getSource(), rel.getTarget()), "diffRel"); + else + return new Tuple2<>(new Tuple2<>(rel.getTarget(), rel.getSource()), "diffRel"); + }); + + JavaRDD, String>> flatMergeRels = mergeRelsRDD + .mapToPair(rel -> new Tuple2<>(rel.getSource(), rel.getTarget())) + .groupByKey() + .flatMap(g -> { + List, String>> rels = new ArrayList<>(); + + List ids = StreamSupport + .stream(g._2().spliterator(), false) + .collect(Collectors.toList()); + + for (int i = 0; i < ids.size(); i++){ + for (int j = i+1; j < ids.size(); j++){ + if (ids.get(i).compareTo(ids.get(j)) < 0) + rels.add(new Tuple2<>(new Tuple2<>(ids.get(i), ids.get(j)), g._1())); + else + rels.add(new Tuple2<>(new Tuple2<>(ids.get(j), ids.get(i)), g._1())); + } + } + return rels.iterator(); + + }); + + JavaRDD purgedMergeRels = flatMergeRels.union(diffRelsRDD) + .mapToPair(rel -> new Tuple2<>(rel._1(), Arrays.asList(rel._2()))) + .reduceByKey((a, b) -> { + List list = new ArrayList(); + list.addAll(a); + list.addAll(b); + return list; + }) + .filter(rel -> rel._2().size() == 1) + .mapToPair(rel -> new Tuple2<>(rel._2().get(0), rel._1())) + .flatMap(rel -> { + List> rels = new ArrayList<>(); + String source = rel._1(); + rels.add(new Tuple2<>(source, rel._2()._1())); + rels.add(new Tuple2<>(source, rel._2()._2())); + return rels.iterator(); + }) + .distinct() + .flatMap(rel -> tupleToMergeRel(rel, dedupConf)); + + spark + .createDataset(purgedMergeRels.rdd(), Encoders.bean(Relation.class)) + .write() + .mode(SaveMode.Overwrite).parquet(mergeRelsPath); + } + } + + private static MapFunction patchRelFn() { + return value -> { + final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class); + if (rel.getDataInfo() == null) { + rel.setDataInfo(new DataInfo()); + } + return rel; + }; + } + + private boolean filterRels(Relation rel, String entityType) { + + switch(entityType) { + case "result": + if (rel.getRelClass().equals("isDifferentFrom") && rel.getRelType().equals("resultResult") && rel.getSubRelType().equals("dedup")) + return true; + break; + case "organization": + if (rel.getRelClass().equals("isDifferentFrom") && rel.getRelType().equals("organizationOrganization") && rel.getSubRelType().equals("dedup")) + return true; + break; + default: + return false; + } + return false; + } + + public Iterator tupleToMergeRel(Tuple2 rel, DedupConfig dedupConf) { + + List rels = new ArrayList<>(); + + rels.add(rel(rel._1(), rel._2(), "merges", dedupConf)); + rels.add(rel(rel._2(), rel._1(), "isMergedIn", dedupConf)); + + return rels.iterator(); + } + + private Relation rel(String source, String target, String relClass, DedupConfig dedupConf) { + + String entityType = dedupConf.getWf().getEntityType(); + + Relation r = new Relation(); + r.setSource(source); + r.setTarget(target); + r.setRelClass(relClass); + r.setRelType(entityType + entityType.substring(0, 1).toUpperCase() + entityType.substring(1)); + r.setSubRelType("dedup"); + + DataInfo info = new DataInfo(); + info.setDeletedbyinference(false); + info.setInferred(true); + info.setInvisible(false); + info.setInferenceprovenance(dedupConf.getWf().getConfigurationId()); + Qualifier provenanceAction = new Qualifier(); + provenanceAction.setClassid(PROVENANCE_ACTION_CLASS); + provenanceAction.setClassname(PROVENANCE_ACTION_CLASS); + provenanceAction.setSchemeid(DNET_PROVENANCE_ACTIONS); + provenanceAction.setSchemename(DNET_PROVENANCE_ACTIONS); + info.setProvenanceaction(provenanceAction); + + // TODO calculate the trust value based on the similarity score of the elements in the CC + // info.setTrust(); + + r.setDataInfo(info); + return r; + } +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/config-default.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/config-default.xml new file mode 100644 index 000000000..2e0ed9aee --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/config-default.xml @@ -0,0 +1,18 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml new file mode 100644 index 000000000..092a5c30e --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml @@ -0,0 +1,324 @@ + + + + graphBasePath + the raw graph base path + + + isLookUpUrl + the address of the lookUp service + + + actionSetId + id of the actionSet + + + workingPath + path for the working directory + + + dedupGraphPath + path for the output graph + + + cutConnectedComponent + max number of elements in a connected component + + + sparkDriverMemory + memory for driver process + + + sparkExecutorMemory + memory for individual executor + + + sparkExecutorCores + number of cores used by single executor + + + oozieActionShareLibForSpark2 + oozie action sharelib for spark 2.* + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + spark 2.* extra listeners classname + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + spark 2.* sql query execution listeners classname + + + spark2YarnHistoryServerAddress + spark 2.* yarn history server address + + + spark2EventLogDir + spark 2.* event log dir location + + + + + ${jobTracker} + ${nameNode} + + + mapreduce.job.queuename + ${queueName} + + + oozie.launcher.mapred.job.queue.name + ${oozieLauncherQueueName} + + + oozie.action.sharelib.for.spark + ${oozieActionShareLibForSpark2} + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + + + + + + + + + + yarn + cluster + Copy Similarity Relations + eu.dnetlib.dhp.oa.dedup.SparkCopySimRels + dhp-dedup-openaire-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.shuffle.partitions=3840 + + --graphBasePath${graphBasePath} + --workingPath${workingPath} + --isLookUpUrl${isLookUpUrl} + --actionSetId${actionSetId} + --numPartitions8000 + + + + + + + + + + + + yarn + cluster + Create Dedup Record + eu.dnetlib.dhp.oa.dedup.SparkCreateDedupRecord + dhp-dedup-openaire-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.shuffle.partitions=3840 + + --graphBasePath${graphBasePath} + --workingPath${workingPath} + --isLookUpUrl${isLookUpUrl} + --actionSetId${actionSetId} + + + + + + + + yarn + cluster + Update Entity + eu.dnetlib.dhp.oa.dedup.SparkUpdateEntity + dhp-dedup-openaire-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.shuffle.partitions=3840 + + --graphBasePath${graphBasePath} + --workingPath${workingPath} + --dedupGraphPath${dedupGraphPath} + + + + + + + + yarn + cluster + Create Similarity Relations + eu.dnetlib.dhp.oa.dedup.SparkCreateSimRels + dhp-dedup-openaire-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.shuffle.partitions=3840 + + --graphBasePath${graphBasePath} + --isLookUpUrl${isLookUpUrl} + --actionSetId${actionSetId} + --workingPath${workingPath} + --numPartitions8000 + + + + + + + + yarn + cluster + Create Merge Relations + eu.dnetlib.dhp.oa.dedup.SparkCreateMergeRels + dhp-dedup-openaire-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.shuffle.partitions=3840 + + --graphBasePath${graphBasePath} + --workingPath${workingPath} + --isLookUpUrl${isLookUpUrl} + --actionSetId${actionSetId} + --cutConnectedComponent${cutConnectedComponent} + + + + + + + + yarn + cluster + Create Merge Relations + eu.dnetlib.dhp.oa.dedup.SparkRemoveDiffRels + dhp-dedup-openaire-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.shuffle.partitions=3840 + + --graphBasePath${graphBasePath} + --isLookUpUrl${isLookUpUrl} + --actionSetId${actionSetId} + --workingPath${workingPath} + --numPartitions8000 + + + + + + + + yarn + cluster + Prepare Organization Relations + eu.dnetlib.dhp.oa.dedup.SparkPrepareOrgRels + dhp-dedup-openaire-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.shuffle.partitions=3840 + + --graphBasePath${graphBasePath} + --workingPath${workingPath} + --isLookUpUrl${isLookUpUrl} + --actionSetId${actionSetId} + --dbUrl${dbUrl} + --dbTable${dbTable} + --dbUser${dbUser} + --dbPwd${dbPwd} + --numConnections20 + + + + + + + + yarn + cluster + Prepare New Organizations + eu.dnetlib.dhp.oa.dedup.SparkPrepareNewOrgs + dhp-dedup-openaire-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.shuffle.partitions=3840 + + --graphBasePath${graphBasePath} + --workingPath${workingPath} + --isLookUpUrl${isLookUpUrl} + --actionSetId${actionSetId} + --apiUrl${apiUrl} + --dbUrl${dbUrl} + --dbTable${dbTable} + --dbUser${dbUser} + --dbPwd${dbPwd} + --numConnections20 + + + + + + + \ No newline at end of file From 8fea29177ccd9d98940151862cea853167bdc045 Mon Sep 17 00:00:00 2001 From: miconis Date: Mon, 18 Jan 2021 16:48:08 +0100 Subject: [PATCH 02/10] refactoring, minor changes and implementation of the wf for openorgs with integration of organization phases into the scan wf --- .../dhp/oa/dedup/SparkCopyOpenorgs.java | 100 ++++++++++++++++++ ...arkCopySimRels.java => SparkCopyRels.java} | 53 +++++----- .../dhp/oa/dedup/copyOpenorgs_parameters.json | 26 +++++ .../dhp/oa/dedup/copyRels_parameters.json | 32 ++++++ .../oa/dedup/openorgs/oozie_app/workflow.xml | 65 +----------- .../dhp/oa/dedup/scan/oozie_app/workflow.xml | 55 ++++++++++ 6 files changed, 247 insertions(+), 84 deletions(-) create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgs.java rename dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/{SparkCopySimRels.java => SparkCopyRels.java} (70%) create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/copyOpenorgs_parameters.json create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/copyRels_parameters.json diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgs.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgs.java new file mode 100644 index 000000000..12ae4e73a --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgs.java @@ -0,0 +1,100 @@ +package eu.dnetlib.dhp.oa.dedup; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.common.EntityType; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.OafEntity; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.dom4j.DocumentException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Optional; + +public class SparkCopyOpenorgs extends AbstractSparkAction{ + private static final Logger log = LoggerFactory.getLogger(SparkCopyRels.class); + + public SparkCopyOpenorgs(ArgumentApplicationParser parser, SparkSession spark) { + super(parser, spark); + } + + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/copyOpenorgs_parameters.json"))); + parser.parseArgument(args); + + SparkConf conf = new SparkConf(); + new SparkCopyOpenorgs(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } + + @Override + public void run(ISLookUpService isLookUpService) + throws DocumentException, IOException, ISLookUpException { + + // read oozie parameters + final String graphBasePath = parser.get("graphBasePath"); + final String actionSetId = parser.get("actionSetId"); + final String workingPath = parser.get("workingPath"); + final int numPartitions = Optional + .ofNullable(parser.get("numPartitions")) + .map(Integer::valueOf) + .orElse(NUM_PARTITIONS); + + log.info("numPartitions: '{}'", numPartitions); + log.info("graphBasePath: '{}'", graphBasePath); + log.info("actionSetId: '{}'", actionSetId); + log.info("workingPath: '{}'", workingPath); + + String subEntity = "organization"; + log.info("Copying openorgs to the working dir"); + + final String outputPath = DedupUtility.createDedupRecordPath(workingPath, actionSetId, subEntity); + removeOutputDir(spark, outputPath); + + final String entityPath = DedupUtility.createEntityPath(graphBasePath, subEntity); + + final Class clazz = ModelSupport.entityTypes.get(EntityType.valueOf(subEntity)); + + filterEntities(spark, entityPath, clazz) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath); + + } + + public static Dataset filterEntities( + final SparkSession spark, + final String entitiesInputPath, + final Class clazz) { + + // + Dataset entities = spark + .read() + .textFile(entitiesInputPath) + .map( + (MapFunction) it -> { + T entity = OBJECT_MAPPER.readValue(it, clazz); + return entity; + }, + Encoders.kryo(clazz)); + + return entities.filter(entities.col("id").contains("openorgs____")); + } + +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopySimRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRels.java similarity index 70% rename from dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopySimRels.java rename to dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRels.java index 4409aaee7..802085ab9 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopySimRels.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRels.java @@ -6,12 +6,10 @@ import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -import eu.dnetlib.pace.config.DedupConfig; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; 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.dom4j.DocumentException; @@ -22,10 +20,10 @@ import java.io.IOException; import java.util.Optional; //copy simrels (verified) from relation to the workdir in order to make them available for the deduplication -public class SparkCopySimRels extends AbstractSparkAction{ - private static final Logger log = LoggerFactory.getLogger(SparkCopySimRels.class); +public class SparkCopyRels extends AbstractSparkAction{ + private static final Logger log = LoggerFactory.getLogger(SparkCopyRels.class); - public SparkCopySimRels(ArgumentApplicationParser parser, SparkSession spark) { + public SparkCopyRels(ArgumentApplicationParser parser, SparkSession spark) { super(parser, spark); } @@ -33,13 +31,13 @@ public class SparkCopySimRels extends AbstractSparkAction{ ArgumentApplicationParser parser = new ArgumentApplicationParser( IOUtils .toString( - SparkCreateSimRels.class + SparkCopyRels.class .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json"))); + "/eu/dnetlib/dhp/oa/dedup/copyRels_parameters.json"))); parser.parseArgument(args); SparkConf conf = new SparkConf(); - new SparkCreateSimRels(parser, getSparkSession(conf)) + new SparkCopyRels(parser, getSparkSession(conf)) .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); } @@ -49,9 +47,10 @@ public class SparkCopySimRels extends AbstractSparkAction{ // read oozie parameters final String graphBasePath = parser.get("graphBasePath"); - final String isLookUpUrl = parser.get("isLookUpUrl"); final String actionSetId = parser.get("actionSetId"); final String workingPath = parser.get("workingPath"); + final String destination = parser.get("destination"); + final String entity = parser.get("entityType"); final int numPartitions = Optional .ofNullable(parser.get("numPartitions")) .map(Integer::valueOf) @@ -59,26 +58,32 @@ public class SparkCopySimRels extends AbstractSparkAction{ log.info("numPartitions: '{}'", numPartitions); log.info("graphBasePath: '{}'", graphBasePath); - log.info("isLookUpUrl: '{}'", isLookUpUrl); log.info("actionSetId: '{}'", actionSetId); log.info("workingPath: '{}'", workingPath); + log.info("entity: '{}'", entity); - // for each dedup configuration - for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) { + log.info("Copying " + destination + " for: '{}'", entity); - final String entity = dedupConf.getWf().getEntityType(); - final String subEntity = dedupConf.getWf().getSubEntityValue(); - log.info("Copying simrels for: '{}'", subEntity); - - final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity); - removeOutputDir(spark, outputPath); - - final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation"); - - JavaRDD simRels = spark.read().textFile(relationPath).map(patchRelFn(), Encoders.bean(Relation.class)).toJavaRDD().filter(r -> filterRels(r, entity)); - - simRels.saveAsTextFile(outputPath); + final String outputPath; + if (destination.contains("mergerel")) { + outputPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, entity); } + else { + outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, entity); + } + + removeOutputDir(spark, outputPath); + + final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation"); + + JavaRDD simRels = + spark.read() + .textFile(relationPath) + .map(patchRelFn(), Encoders.bean(Relation.class)) + .toJavaRDD() + .filter(r -> filterRels(r, entity)); + + simRels.saveAsTextFile(outputPath); } private static MapFunction patchRelFn() { diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/copyOpenorgs_parameters.json b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/copyOpenorgs_parameters.json new file mode 100644 index 000000000..e45efca01 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/copyOpenorgs_parameters.json @@ -0,0 +1,26 @@ +[ + { + "paramName": "asi", + "paramLongName": "actionSetId", + "paramDescription": "action set identifier (name of the orchestrator)", + "paramRequired": true + }, + { + "paramName": "i", + "paramLongName": "graphBasePath", + "paramDescription": "the base path of the raw graph", + "paramRequired": true + }, + { + "paramName": "w", + "paramLongName": "workingPath", + "paramDescription": "path of the working directory", + "paramRequired": true + }, + { + "paramName": "np", + "paramLongName": "numPartitions", + "paramDescription": "number of partitions for the similarity relations intermediate phases", + "paramRequired": false + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/copyRels_parameters.json b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/copyRels_parameters.json new file mode 100644 index 000000000..715b0e74e --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/copyRels_parameters.json @@ -0,0 +1,32 @@ +[ + { + "paramName": "asi", + "paramLongName": "actionSetId", + "paramDescription": "action set identifier (name of the orchestrator)", + "paramRequired": true + }, + { + "paramName": "i", + "paramLongName": "graphBasePath", + "paramDescription": "the base path of the raw graph", + "paramRequired": true + }, + { + "paramName": "w", + "paramLongName": "workingPath", + "paramDescription": "path of the working directory", + "paramRequired": true + }, + { + "paramName": "e", + "paramLongName": "entityType", + "paramDescription": "type of the entity for the merge relations", + "paramRequired": true + }, + { + "paramName": "np", + "paramLongName": "numPartitions", + "paramDescription": "number of partitions for the similarity relations intermediate phases", + "paramRequired": false + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml index 092a5c30e..2f961face 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml @@ -93,13 +93,12 @@ - - + yarn cluster - Copy Similarity Relations - eu.dnetlib.dhp.oa.dedup.SparkCopySimRels + Copy Merge Relations + eu.dnetlib.dhp.oa.dedup.SparkCopyRels dhp-dedup-openaire-${projectVersion}.jar --executor-memory=${sparkExecutorMemory} @@ -113,65 +112,11 @@ --graphBasePath${graphBasePath} --workingPath${workingPath} - --isLookUpUrl${isLookUpUrl} --actionSetId${actionSetId} + --entityTypeorganization + --destinationsimrel --numPartitions8000 - - - - - - - - - - - yarn - cluster - Create Dedup Record - eu.dnetlib.dhp.oa.dedup.SparkCreateDedupRecord - dhp-dedup-openaire-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 - - --graphBasePath${graphBasePath} - --workingPath${workingPath} - --isLookUpUrl${isLookUpUrl} - --actionSetId${actionSetId} - - - - - - - - yarn - cluster - Update Entity - eu.dnetlib.dhp.oa.dedup.SparkUpdateEntity - dhp-dedup-openaire-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 - - --graphBasePath${graphBasePath} - --workingPath${workingPath} - --dedupGraphPath${dedupGraphPath} - diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml index c42ce1263..d22f05ca8 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml @@ -169,6 +169,61 @@ --isLookUpUrl${isLookUpUrl} --actionSetId${actionSetId} + + + + + + + + yarn + cluster + Copy Merge Relations + eu.dnetlib.dhp.oa.dedup.SparkCopyRels + dhp-dedup-openaire-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.shuffle.partitions=3840 + + --graphBasePath${graphBasePath} + --workingPath${workingPath} + --actionSetId${actionSetId} + --entityTypeorganization + --destinationmergerel + --numPartitions8000 + + + + + + + + yarn + cluster + Copy Entities + eu.dnetlib.dhp.oa.dedup.SparkCopyOpenorgs + dhp-dedup-openaire-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.shuffle.partitions=3840 + + --graphBasePath${graphBasePath} + --workingPath${workingPath} + --isLookUpUrl${isLookUpUrl} + --actionSetId${actionSetId} + From c7e2d5a59addfcbf71c4c5e4319823bd34136cd6 Mon Sep 17 00:00:00 2001 From: miconis Date: Mon, 25 Jan 2021 12:40:45 +0100 Subject: [PATCH 03/10] minor changes --- .../eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml index 2f961face..a6b313cad 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml @@ -85,6 +85,9 @@ Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + From 4b2124a18e611fad6db3de5f3f5947d406f2b88b Mon Sep 17 00:00:00 2001 From: miconis Date: Wed, 10 Feb 2021 11:51:50 +0100 Subject: [PATCH 04/10] implementation of the openorgs wfs, implementation of the raw_all wf to migrate openorgs db entities --- .../dhp/oa/dedup/SparkCopyOpenorgs.java | 140 +++---- .../oa/dedup/SparkCopyOpenorgsMergeRels.java | 181 +++++++++ ...els.java => SparkCopyOpenorgsSimRels.java} | 97 ++--- .../dedup/SparkCopyRelationsNoOpenorgs.java | 110 ++++++ .../dhp/oa/dedup/SparkRemoveDiffRels.java | 363 +++++++++--------- ... => copyOpenorgsMergeRels_parameters.json} | 6 +- .../oa/dedup/openorgs/oozie_app/workflow.xml | 58 ++- .../dhp/oa/dedup/scan/oozie_app/workflow.xml | 49 ++- .../raw/MigrateDbEntitiesApplication.java | 103 +++-- .../oa/graph/raw/common/MigrateAction.java | 9 + .../oa/graph/raw_all/oozie_app/workflow.xml | 38 +- .../sql/queryOrganizationsFromOpenOrgsDB.sql | 37 +- .../sql/querySimilarityFromOpenOrgsDB.sql | 52 ++- 13 files changed, 858 insertions(+), 385 deletions(-) create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java rename dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/{SparkCopyRels.java => SparkCopyOpenorgsSimRels.java} (61%) create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRelationsNoOpenorgs.java rename dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/{copyRels_parameters.json => copyOpenorgsMergeRels_parameters.json} (84%) create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MigrateAction.java diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgs.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgs.java index 12ae4e73a..aa7a131e7 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgs.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgs.java @@ -1,12 +1,9 @@ + package eu.dnetlib.dhp.oa.dedup; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.common.EntityType; -import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.OafEntity; -import eu.dnetlib.dhp.utils.ISLookupClientFactory; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import java.io.IOException; +import java.util.Optional; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.MapFunction; @@ -18,83 +15,88 @@ import org.dom4j.DocumentException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.Optional; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.common.EntityType; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.OafEntity; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -public class SparkCopyOpenorgs extends AbstractSparkAction{ - private static final Logger log = LoggerFactory.getLogger(SparkCopyRels.class); +public class SparkCopyOpenorgs extends AbstractSparkAction { + private static final Logger log = LoggerFactory.getLogger(SparkCopyOpenorgs.class); - public SparkCopyOpenorgs(ArgumentApplicationParser parser, SparkSession spark) { - super(parser, spark); - } + public SparkCopyOpenorgs(ArgumentApplicationParser parser, SparkSession spark) { + super(parser, spark); + } - public static void main(String[] args) throws Exception { - ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkCreateSimRels.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/copyOpenorgs_parameters.json"))); - parser.parseArgument(args); + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/copyOpenorgs_parameters.json"))); + parser.parseArgument(args); - SparkConf conf = new SparkConf(); - new SparkCopyOpenorgs(parser, getSparkSession(conf)) - .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); - } + SparkConf conf = new SparkConf(); + new SparkCopyOpenorgs(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } - @Override - public void run(ISLookUpService isLookUpService) - throws DocumentException, IOException, ISLookUpException { + @Override + public void run(ISLookUpService isLookUpService) + throws DocumentException, IOException, ISLookUpException { - // read oozie parameters - final String graphBasePath = parser.get("graphBasePath"); - final String actionSetId = parser.get("actionSetId"); - final String workingPath = parser.get("workingPath"); - final int numPartitions = Optional - .ofNullable(parser.get("numPartitions")) - .map(Integer::valueOf) - .orElse(NUM_PARTITIONS); + // read oozie parameters + final String graphBasePath = parser.get("graphBasePath"); + final String actionSetId = parser.get("actionSetId"); + final String workingPath = parser.get("workingPath"); + final int numPartitions = Optional + .ofNullable(parser.get("numPartitions")) + .map(Integer::valueOf) + .orElse(NUM_PARTITIONS); - log.info("numPartitions: '{}'", numPartitions); - log.info("graphBasePath: '{}'", graphBasePath); - log.info("actionSetId: '{}'", actionSetId); - log.info("workingPath: '{}'", workingPath); + log.info("numPartitions: '{}'", numPartitions); + log.info("graphBasePath: '{}'", graphBasePath); + log.info("actionSetId: '{}'", actionSetId); + log.info("workingPath: '{}'", workingPath); - String subEntity = "organization"; - log.info("Copying openorgs to the working dir"); + String subEntity = "organization"; + log.info("Copying openorgs to the working dir"); - final String outputPath = DedupUtility.createDedupRecordPath(workingPath, actionSetId, subEntity); - removeOutputDir(spark, outputPath); + final String outputPath = DedupUtility.createDedupRecordPath(workingPath, actionSetId, subEntity); + removeOutputDir(spark, outputPath); - final String entityPath = DedupUtility.createEntityPath(graphBasePath, subEntity); + final String entityPath = DedupUtility.createEntityPath(graphBasePath, subEntity); - final Class clazz = ModelSupport.entityTypes.get(EntityType.valueOf(subEntity)); + final Class clazz = ModelSupport.entityTypes.get(EntityType.valueOf(subEntity)); - filterEntities(spark, entityPath, clazz) - .write() - .mode(SaveMode.Overwrite) - .option("compression", "gzip") - .json(outputPath); + filterEntities(spark, entityPath, clazz) + .write() + .mode(SaveMode.Overwrite) + .option("compression", "gzip") + .json(outputPath); - } + } - public static Dataset filterEntities( - final SparkSession spark, - final String entitiesInputPath, - final Class clazz) { + public static Dataset filterEntities( + final SparkSession spark, + final String entitiesInputPath, + final Class clazz) { - // - Dataset entities = spark - .read() - .textFile(entitiesInputPath) - .map( - (MapFunction) it -> { - T entity = OBJECT_MAPPER.readValue(it, clazz); - return entity; - }, - Encoders.kryo(clazz)); + // + Dataset entities = spark + .read() + .textFile(entitiesInputPath) + .map( + (MapFunction) it -> { + T entity = OBJECT_MAPPER.readValue(it, clazz); + return entity; + }, + Encoders.kryo(clazz)); - return entities.filter(entities.col("id").contains("openorgs____")); - } + return entities.filter(entities.col("id").contains("openorgs____")); + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java new file mode 100644 index 000000000..d705fca6b --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java @@ -0,0 +1,181 @@ + +package eu.dnetlib.dhp.oa.dedup; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +import eu.dnetlib.dhp.schema.oaf.KeyValue; +import eu.dnetlib.dhp.schema.oaf.Qualifier; +import eu.dnetlib.pace.config.DedupConfig; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SparkSession; +import org.dom4j.DocumentException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.DataInfo; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; + +//copy simrels (verified) from relation to the workdir in order to make them available for the deduplication +public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction { + private static final Logger log = LoggerFactory.getLogger(SparkCopyOpenorgsMergeRels.class); + public static final String PROVENANCE_ACTION_CLASS = "sysimport:dedup"; + public static final String DNET_PROVENANCE_ACTIONS = "dnet:provenanceActions"; + + public SparkCopyOpenorgsMergeRels(ArgumentApplicationParser parser, SparkSession spark) { + super(parser, spark); + } + + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCopyOpenorgsMergeRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/copyOpenorgsMergeRels_parameters.json"))); + parser.parseArgument(args); + + SparkConf conf = new SparkConf(); + new SparkCopyOpenorgsMergeRels(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } + + @Override + public void run(ISLookUpService isLookUpService) + throws DocumentException, IOException, ISLookUpException { + + // read oozie parameters + final String graphBasePath = parser.get("graphBasePath"); + final String actionSetId = parser.get("actionSetId"); + final String workingPath = parser.get("workingPath"); + final int numPartitions = Optional + .ofNullable(parser.get("numPartitions")) + .map(Integer::valueOf) + .orElse(NUM_PARTITIONS); + + log.info("numPartitions: '{}'", numPartitions); + log.info("graphBasePath: '{}'", graphBasePath); + log.info("actionSetId: '{}'", actionSetId); + log.info("workingPath: '{}'", workingPath); + + log.info("Copying OpenOrgs Merge Rels"); + + final String outputPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, "organization"); + + removeOutputDir(spark, outputPath); + + final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation"); + + DedupConfig dedupConf = getConfigurations(isLookUpService, actionSetId).get(0); + + JavaRDD rawRels = spark + .read() + .textFile(relationPath) + .map(patchRelFn(), Encoders.bean(Relation.class)) + .toJavaRDD() + .filter(this::isOpenorgs) //takes only relations coming from openorgs + .filter(this::filterOpenorgsRels) //takes only isSimilarTo relations between organizations from openorgs + .filter(this::excludeOpenorgsMesh) //excludes relations between an organization and an openorgsmesh + .filter(this::excludeNonOpenorgs); //excludes relations with no openorgs id involved + + //turn openorgs isSimilarTo relations into mergerels + JavaRDD mergeRels = rawRels.flatMap(rel -> { + List mergerels = new ArrayList<>(); + + String openorgsId = rel.getSource().contains("openorgs____")? rel.getSource() : rel.getTarget(); + String mergedId = rel.getSource().contains("openorgs____")? rel.getTarget() : rel.getSource(); + + mergerels.add(rel(openorgsId, mergedId, "merges", dedupConf)); + mergerels.add(rel(mergedId, openorgsId, "isMergedIn", dedupConf)); + + return mergerels.iterator(); + }); + + mergeRels.saveAsTextFile(outputPath); + } + + private static MapFunction patchRelFn() { + return value -> { + final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class); + if (rel.getDataInfo() == null) { + rel.setDataInfo(new DataInfo()); + } + return rel; + }; + } + + private boolean filterOpenorgsRels(Relation rel) { + + if (rel.getRelClass().equals("isSimilarTo") && rel.getRelType().equals("organizationOrganization") && rel.getSubRelType().equals("dedup")) + return true; + return false; + } + + private boolean isOpenorgs(Relation rel) { + + if (rel.getCollectedfrom() != null) { + for (KeyValue k: rel.getCollectedfrom()) { + if (k.getValue().equals("OpenOrgs Database")) { + return true; + } + } + } + return false; + } + + private boolean excludeOpenorgsMesh(Relation rel) { + + if (rel.getSource().equals("openorgsmesh") || rel.getTarget().equals("openorgsmesh")) { + return false; + } + return true; + } + + private boolean excludeNonOpenorgs(Relation rel) { + + if (rel.getSource().equals("openorgs____") || rel.getTarget().equals("openorgs____")) { + return true; + } + return false; + } + + private Relation rel(String source, String target, String relClass, DedupConfig dedupConf) { + + String entityType = dedupConf.getWf().getEntityType(); + + Relation r = new Relation(); + r.setSource(source); + r.setTarget(target); + r.setRelClass(relClass); + r.setRelType(entityType + entityType.substring(0, 1).toUpperCase() + entityType.substring(1)); + r.setSubRelType("dedup"); + + DataInfo info = new DataInfo(); + info.setDeletedbyinference(false); + info.setInferred(true); + info.setInvisible(false); + info.setInferenceprovenance(dedupConf.getWf().getConfigurationId()); + Qualifier provenanceAction = new Qualifier(); + provenanceAction.setClassid(PROVENANCE_ACTION_CLASS); + provenanceAction.setClassname(PROVENANCE_ACTION_CLASS); + provenanceAction.setSchemeid(DNET_PROVENANCE_ACTIONS); + provenanceAction.setSchemename(DNET_PROVENANCE_ACTIONS); + info.setProvenanceaction(provenanceAction); + + // TODO calculate the trust value based on the similarity score of the elements in the CC + // info.setTrust(); + + r.setDataInfo(info); + return r; + } +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsSimRels.java similarity index 61% rename from dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRels.java rename to dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsSimRels.java index 802085ab9..3ce676f84 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRels.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsSimRels.java @@ -1,29 +1,37 @@ + package eu.dnetlib.dhp.oa.dedup; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +import eu.dnetlib.dhp.schema.oaf.KeyValue; +import eu.dnetlib.dhp.schema.oaf.Qualifier; +import eu.dnetlib.pace.config.DedupConfig; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.dom4j.DocumentException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.DataInfo; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -import org.apache.commons.io.IOUtils; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.function.MapFunction; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.SparkSession; -import org.dom4j.DocumentException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.Optional; //copy simrels (verified) from relation to the workdir in order to make them available for the deduplication -public class SparkCopyRels extends AbstractSparkAction{ - private static final Logger log = LoggerFactory.getLogger(SparkCopyRels.class); +public class SparkCopyOpenorgsSimRels extends AbstractSparkAction { + private static final Logger log = LoggerFactory.getLogger(SparkCopyOpenorgsMergeRels.class); - public SparkCopyRels(ArgumentApplicationParser parser, SparkSession spark) { + public SparkCopyOpenorgsSimRels(ArgumentApplicationParser parser, SparkSession spark) { super(parser, spark); } @@ -31,13 +39,13 @@ public class SparkCopyRels extends AbstractSparkAction{ ArgumentApplicationParser parser = new ArgumentApplicationParser( IOUtils .toString( - SparkCopyRels.class + SparkCopyOpenorgsSimRels.class .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/copyRels_parameters.json"))); + "/eu/dnetlib/dhp/oa/dedup/copyOpenorgsMergeRels_parameters.json"))); parser.parseArgument(args); SparkConf conf = new SparkConf(); - new SparkCopyRels(parser, getSparkSession(conf)) + new SparkCopyOpenorgsSimRels(parser, getSparkSession(conf)) .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); } @@ -49,8 +57,6 @@ public class SparkCopyRels extends AbstractSparkAction{ final String graphBasePath = parser.get("graphBasePath"); final String actionSetId = parser.get("actionSetId"); final String workingPath = parser.get("workingPath"); - final String destination = parser.get("destination"); - final String entity = parser.get("entityType"); final int numPartitions = Optional .ofNullable(parser.get("numPartitions")) .map(Integer::valueOf) @@ -60,30 +66,24 @@ public class SparkCopyRels extends AbstractSparkAction{ log.info("graphBasePath: '{}'", graphBasePath); log.info("actionSetId: '{}'", actionSetId); log.info("workingPath: '{}'", workingPath); - log.info("entity: '{}'", entity); - log.info("Copying " + destination + " for: '{}'", entity); + log.info("Copying OpenOrgs SimRels"); - final String outputPath; - if (destination.contains("mergerel")) { - outputPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, entity); - } - else { - outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, entity); - } + final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, "organization"); removeOutputDir(spark, outputPath); final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation"); - JavaRDD simRels = - spark.read() - .textFile(relationPath) - .map(patchRelFn(), Encoders.bean(Relation.class)) - .toJavaRDD() - .filter(r -> filterRels(r, entity)); + JavaRDD rawRels = spark + .read() + .textFile(relationPath) + .map(patchRelFn(), Encoders.bean(Relation.class)) + .toJavaRDD() + .filter(this::isOpenorgs) + .filter(this::filterOpenorgsRels); - simRels.saveAsTextFile(outputPath); + save(spark.createDataset(rawRels.rdd(),Encoders.bean(Relation.class)), outputPath, SaveMode.Append); } private static MapFunction patchRelFn() { @@ -96,20 +96,23 @@ public class SparkCopyRels extends AbstractSparkAction{ }; } - private boolean filterRels(Relation rel, String entityType) { + private boolean filterOpenorgsRels(Relation rel) { - switch(entityType) { - case "result": - if (rel.getRelClass().equals("isSimilarTo") && rel.getRelType().equals("resultResult") && rel.getSubRelType().equals("dedup")) + if (rel.getRelClass().equals("isSimilarTo") && rel.getRelType().equals("organizationOrganization") && rel.getSubRelType().equals("dedup")) + return true; + return false; + } + + private boolean isOpenorgs(Relation rel) { + + if (rel.getCollectedfrom() != null) { + for (KeyValue k: rel.getCollectedfrom()) { + if (k.getValue().equals("OpenOrgs Database")) { return true; - break; - case "organization": - if (rel.getRelClass().equals("isSimilarTo") && rel.getRelType().equals("organizationOrganization") && rel.getSubRelType().equals("dedup")) - return true; - break; - default: - return false; + } + } } return false; } } + diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRelationsNoOpenorgs.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRelationsNoOpenorgs.java new file mode 100644 index 000000000..319c40d8d --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRelationsNoOpenorgs.java @@ -0,0 +1,110 @@ +package eu.dnetlib.dhp.oa.dedup; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import eu.dnetlib.pace.util.MapDocumentUtil; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.dom4j.DocumentException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Tuple2; + +import java.io.IOException; +import java.util.Optional; + +public class SparkCopyRelationsNoOpenorgs extends AbstractSparkAction { + + private static final Logger log = LoggerFactory.getLogger(SparkUpdateEntity.class); + + private static final String IDJSONPATH = "$.id"; + + public SparkCopyRelationsNoOpenorgs(ArgumentApplicationParser parser, SparkSession spark) { + super(parser, spark); + } + + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCopyRelationsNoOpenorgs.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json"))); + parser.parseArgument(args); + + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); + + new SparkUpdateEntity(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } + + public void run(ISLookUpService isLookUpService) throws IOException { + + final String graphBasePath = parser.get("graphBasePath"); + final String workingPath = parser.get("workingPath"); + final String dedupGraphPath = parser.get("dedupGraphPath"); + + log.info("graphBasePath: '{}'", graphBasePath); + log.info("workingPath: '{}'", workingPath); + log.info("dedupGraphPath: '{}'", dedupGraphPath); + + final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation"); + final String outputPath = DedupUtility.createEntityPath(dedupGraphPath, "relation"); + + removeOutputDir(spark, outputPath); + + JavaRDD simRels = spark + .read() + .textFile(relationPath) + .map(patchRelFn(), Encoders.bean(Relation.class)) + .toJavaRDD() + .filter(this::excludeOpenorgsRels); + + simRels.saveAsTextFile(outputPath); + + } + + private static MapFunction patchRelFn() { + return value -> { + final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class); + if (rel.getDataInfo() == null) { + rel.setDataInfo(new DataInfo()); + } + return rel; + }; + } + + private boolean excludeOpenorgsRels(Relation rel) { + + if (rel.getCollectedfrom() != null) { + for (KeyValue k: rel.getCollectedfrom()) { + if (k.getValue().equals("OpenOrgs Database")) { + return false; + } + } + } + return true; + } +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkRemoveDiffRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkRemoveDiffRels.java index 030f3b783..6f012e00a 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkRemoveDiffRels.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkRemoveDiffRels.java @@ -1,18 +1,15 @@ + package eu.dnetlib.dhp.oa.dedup; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.oa.dedup.graph.ConnectedComponent; -import eu.dnetlib.dhp.oa.dedup.graph.GraphProcessor; -import eu.dnetlib.dhp.oa.dedup.model.Block; -import eu.dnetlib.dhp.schema.oaf.DataInfo; -import eu.dnetlib.dhp.schema.oaf.Qualifier; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.utils.ISLookupClientFactory; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -import eu.dnetlib.pace.config.DedupConfig; -import eu.dnetlib.pace.model.MapDocument; -import eu.dnetlib.pace.util.MapDocumentUtil; +import static eu.dnetlib.dhp.oa.dedup.SparkCreateMergeRels.DNET_PROVENANCE_ACTIONS; +import static eu.dnetlib.dhp.oa.dedup.SparkCreateMergeRels.PROVENANCE_ACTION_CLASS; +import static eu.dnetlib.dhp.oa.dedup.SparkCreateMergeRels.hash; + +import java.io.IOException; +import java.util.*; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; @@ -29,205 +26,215 @@ import org.apache.spark.sql.SparkSession; import org.dom4j.DocumentException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.oa.dedup.graph.ConnectedComponent; +import eu.dnetlib.dhp.oa.dedup.graph.GraphProcessor; +import eu.dnetlib.dhp.oa.dedup.model.Block; +import eu.dnetlib.dhp.schema.oaf.DataInfo; +import eu.dnetlib.dhp.schema.oaf.Qualifier; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import eu.dnetlib.pace.config.DedupConfig; +import eu.dnetlib.pace.model.MapDocument; +import eu.dnetlib.pace.util.MapDocumentUtil; import scala.Tuple2; -import java.io.IOException; -import java.util.*; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; - -import static eu.dnetlib.dhp.oa.dedup.SparkCreateMergeRels.DNET_PROVENANCE_ACTIONS; -import static eu.dnetlib.dhp.oa.dedup.SparkCreateMergeRels.PROVENANCE_ACTION_CLASS; -import static eu.dnetlib.dhp.oa.dedup.SparkCreateMergeRels.hash; - public class SparkRemoveDiffRels extends AbstractSparkAction { - private static final Logger log = LoggerFactory.getLogger(SparkRemoveDiffRels.class); + private static final Logger log = LoggerFactory.getLogger(SparkRemoveDiffRels.class); - public SparkRemoveDiffRels(ArgumentApplicationParser parser, SparkSession spark) { - super(parser, spark); - } + public SparkRemoveDiffRels(ArgumentApplicationParser parser, SparkSession spark) { + super(parser, spark); + } - public static void main(String[] args) throws Exception { - ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkCreateSimRels.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json"))); - parser.parseArgument(args); + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json"))); + parser.parseArgument(args); - SparkConf conf = new SparkConf(); - new SparkCreateSimRels(parser, getSparkSession(conf)) - .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); - } + SparkConf conf = new SparkConf(); + new SparkCreateSimRels(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } - @Override - public void run(ISLookUpService isLookUpService) - throws DocumentException, IOException, ISLookUpException { + @Override + public void run(ISLookUpService isLookUpService) + throws DocumentException, IOException, ISLookUpException { - // read oozie parameters - final String graphBasePath = parser.get("graphBasePath"); - final String isLookUpUrl = parser.get("isLookUpUrl"); - final String actionSetId = parser.get("actionSetId"); - final String workingPath = parser.get("workingPath"); - final int numPartitions = Optional - .ofNullable(parser.get("numPartitions")) - .map(Integer::valueOf) - .orElse(NUM_PARTITIONS); + // read oozie parameters + final String graphBasePath = parser.get("graphBasePath"); + final String isLookUpUrl = parser.get("isLookUpUrl"); + final String actionSetId = parser.get("actionSetId"); + final String workingPath = parser.get("workingPath"); + final int numPartitions = Optional + .ofNullable(parser.get("numPartitions")) + .map(Integer::valueOf) + .orElse(NUM_PARTITIONS); - log.info("numPartitions: '{}'", numPartitions); - log.info("graphBasePath: '{}'", graphBasePath); - log.info("isLookUpUrl: '{}'", isLookUpUrl); - log.info("actionSetId: '{}'", actionSetId); - log.info("workingPath: '{}'", workingPath); + log.info("numPartitions: '{}'", numPartitions); + log.info("graphBasePath: '{}'", graphBasePath); + log.info("isLookUpUrl: '{}'", isLookUpUrl); + log.info("actionSetId: '{}'", actionSetId); + log.info("workingPath: '{}'", workingPath); - // for each dedup configuration - for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) { + // for each dedup configuration + for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) { - final String entity = dedupConf.getWf().getEntityType(); - final String subEntity = dedupConf.getWf().getSubEntityValue(); - log.info("Removing diffrels for: '{}'", subEntity); + final String entity = dedupConf.getWf().getEntityType(); + final String subEntity = dedupConf.getWf().getSubEntityValue(); + log.info("Removing diffrels for: '{}'", subEntity); - final String mergeRelsPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, subEntity); + final String mergeRelsPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, subEntity); - final String relationPath = DedupUtility.createEntityPath(graphBasePath, subEntity); + final String relationPath = DedupUtility.createEntityPath(graphBasePath, subEntity); - final int maxIterations = dedupConf.getWf().getMaxIterations(); - log.info("Max iterations {}", maxIterations); + final int maxIterations = dedupConf.getWf().getMaxIterations(); + log.info("Max iterations {}", maxIterations); - JavaRDD mergeRelsRDD = spark - .read() - .load(mergeRelsPath) - .as(Encoders.bean(Relation.class)) - .where("relClass == 'merges'") - .toJavaRDD(); + JavaRDD mergeRelsRDD = spark + .read() + .load(mergeRelsPath) + .as(Encoders.bean(Relation.class)) + .where("relClass == 'merges'") + .toJavaRDD(); - JavaRDD, String>> diffRelsRDD = spark - .read() - .textFile(relationPath) - .map(patchRelFn(), Encoders.bean(Relation.class)) - .toJavaRDD().filter(r -> filterRels(r, entity)) - .map(rel -> { - if (rel.getSource().compareTo(rel.getTarget()) < 0) - return new Tuple2<>(new Tuple2<>(rel.getSource(), rel.getTarget()), "diffRel"); - else - return new Tuple2<>(new Tuple2<>(rel.getTarget(), rel.getSource()), "diffRel"); - }); + JavaRDD, String>> diffRelsRDD = spark + .read() + .textFile(relationPath) + .map(patchRelFn(), Encoders.bean(Relation.class)) + .toJavaRDD() + .filter(r -> filterRels(r, entity)) + .map(rel -> { + if (rel.getSource().compareTo(rel.getTarget()) < 0) + return new Tuple2<>(new Tuple2<>(rel.getSource(), rel.getTarget()), "diffRel"); + else + return new Tuple2<>(new Tuple2<>(rel.getTarget(), rel.getSource()), "diffRel"); + }); - JavaRDD, String>> flatMergeRels = mergeRelsRDD - .mapToPair(rel -> new Tuple2<>(rel.getSource(), rel.getTarget())) - .groupByKey() - .flatMap(g -> { - List, String>> rels = new ArrayList<>(); + JavaRDD, String>> flatMergeRels = mergeRelsRDD + .mapToPair(rel -> new Tuple2<>(rel.getSource(), rel.getTarget())) + .groupByKey() + .flatMap(g -> { + List, String>> rels = new ArrayList<>(); - List ids = StreamSupport - .stream(g._2().spliterator(), false) - .collect(Collectors.toList()); + List ids = StreamSupport + .stream(g._2().spliterator(), false) + .collect(Collectors.toList()); - for (int i = 0; i < ids.size(); i++){ - for (int j = i+1; j < ids.size(); j++){ - if (ids.get(i).compareTo(ids.get(j)) < 0) - rels.add(new Tuple2<>(new Tuple2<>(ids.get(i), ids.get(j)), g._1())); - else - rels.add(new Tuple2<>(new Tuple2<>(ids.get(j), ids.get(i)), g._1())); - } - } - return rels.iterator(); + for (int i = 0; i < ids.size(); i++) { + for (int j = i + 1; j < ids.size(); j++) { + if (ids.get(i).compareTo(ids.get(j)) < 0) + rels.add(new Tuple2<>(new Tuple2<>(ids.get(i), ids.get(j)), g._1())); + else + rels.add(new Tuple2<>(new Tuple2<>(ids.get(j), ids.get(i)), g._1())); + } + } + return rels.iterator(); - }); + }); - JavaRDD purgedMergeRels = flatMergeRels.union(diffRelsRDD) - .mapToPair(rel -> new Tuple2<>(rel._1(), Arrays.asList(rel._2()))) - .reduceByKey((a, b) -> { - List list = new ArrayList(); - list.addAll(a); - list.addAll(b); - return list; - }) - .filter(rel -> rel._2().size() == 1) - .mapToPair(rel -> new Tuple2<>(rel._2().get(0), rel._1())) - .flatMap(rel -> { - List> rels = new ArrayList<>(); - String source = rel._1(); - rels.add(new Tuple2<>(source, rel._2()._1())); - rels.add(new Tuple2<>(source, rel._2()._2())); - return rels.iterator(); - }) - .distinct() - .flatMap(rel -> tupleToMergeRel(rel, dedupConf)); + JavaRDD purgedMergeRels = flatMergeRels + .union(diffRelsRDD) + .mapToPair(rel -> new Tuple2<>(rel._1(), Arrays.asList(rel._2()))) + .reduceByKey((a, b) -> { + List list = new ArrayList(); + list.addAll(a); + list.addAll(b); + return list; + }) + .filter(rel -> rel._2().size() == 1) + .mapToPair(rel -> new Tuple2<>(rel._2().get(0), rel._1())) + .flatMap(rel -> { + List> rels = new ArrayList<>(); + String source = rel._1(); + rels.add(new Tuple2<>(source, rel._2()._1())); + rels.add(new Tuple2<>(source, rel._2()._2())); + return rels.iterator(); + }) + .distinct() + .flatMap(rel -> tupleToMergeRel(rel, dedupConf)); - spark - .createDataset(purgedMergeRels.rdd(), Encoders.bean(Relation.class)) - .write() - .mode(SaveMode.Overwrite).parquet(mergeRelsPath); - } - } + spark + .createDataset(purgedMergeRels.rdd(), Encoders.bean(Relation.class)) + .write() + .mode(SaveMode.Overwrite) + .json(mergeRelsPath); + } + } - private static MapFunction patchRelFn() { - return value -> { - final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class); - if (rel.getDataInfo() == null) { - rel.setDataInfo(new DataInfo()); - } - return rel; - }; - } + private static MapFunction patchRelFn() { + return value -> { + final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class); + if (rel.getDataInfo() == null) { + rel.setDataInfo(new DataInfo()); + } + return rel; + }; + } - private boolean filterRels(Relation rel, String entityType) { + private boolean filterRels(Relation rel, String entityType) { - switch(entityType) { - case "result": - if (rel.getRelClass().equals("isDifferentFrom") && rel.getRelType().equals("resultResult") && rel.getSubRelType().equals("dedup")) - return true; - break; - case "organization": - if (rel.getRelClass().equals("isDifferentFrom") && rel.getRelType().equals("organizationOrganization") && rel.getSubRelType().equals("dedup")) - return true; - break; - default: - return false; - } - return false; - } + switch (entityType) { + case "result": + if (rel.getRelClass().equals("isDifferentFrom") && rel.getRelType().equals("resultResult") + && rel.getSubRelType().equals("dedup")) + return true; + break; + case "organization": + if (rel.getRelClass().equals("isDifferentFrom") && rel.getRelType().equals("organizationOrganization") + && rel.getSubRelType().equals("dedup")) + return true; + break; + default: + return false; + } + return false; + } - public Iterator tupleToMergeRel(Tuple2 rel, DedupConfig dedupConf) { + public Iterator tupleToMergeRel(Tuple2 rel, DedupConfig dedupConf) { - List rels = new ArrayList<>(); + List rels = new ArrayList<>(); - rels.add(rel(rel._1(), rel._2(), "merges", dedupConf)); - rels.add(rel(rel._2(), rel._1(), "isMergedIn", dedupConf)); + rels.add(rel(rel._1(), rel._2(), "merges", dedupConf)); + rels.add(rel(rel._2(), rel._1(), "isMergedIn", dedupConf)); - return rels.iterator(); - } + return rels.iterator(); + } - private Relation rel(String source, String target, String relClass, DedupConfig dedupConf) { + private Relation rel(String source, String target, String relClass, DedupConfig dedupConf) { - String entityType = dedupConf.getWf().getEntityType(); + String entityType = dedupConf.getWf().getEntityType(); - Relation r = new Relation(); - r.setSource(source); - r.setTarget(target); - r.setRelClass(relClass); - r.setRelType(entityType + entityType.substring(0, 1).toUpperCase() + entityType.substring(1)); - r.setSubRelType("dedup"); + Relation r = new Relation(); + r.setSource(source); + r.setTarget(target); + r.setRelClass(relClass); + r.setRelType(entityType + entityType.substring(0, 1).toUpperCase() + entityType.substring(1)); + r.setSubRelType("dedup"); - DataInfo info = new DataInfo(); - info.setDeletedbyinference(false); - info.setInferred(true); - info.setInvisible(false); - info.setInferenceprovenance(dedupConf.getWf().getConfigurationId()); - Qualifier provenanceAction = new Qualifier(); - provenanceAction.setClassid(PROVENANCE_ACTION_CLASS); - provenanceAction.setClassname(PROVENANCE_ACTION_CLASS); - provenanceAction.setSchemeid(DNET_PROVENANCE_ACTIONS); - provenanceAction.setSchemename(DNET_PROVENANCE_ACTIONS); - info.setProvenanceaction(provenanceAction); + DataInfo info = new DataInfo(); + info.setDeletedbyinference(false); + info.setInferred(true); + info.setInvisible(false); + info.setInferenceprovenance(dedupConf.getWf().getConfigurationId()); + Qualifier provenanceAction = new Qualifier(); + provenanceAction.setClassid(PROVENANCE_ACTION_CLASS); + provenanceAction.setClassname(PROVENANCE_ACTION_CLASS); + provenanceAction.setSchemeid(DNET_PROVENANCE_ACTIONS); + provenanceAction.setSchemename(DNET_PROVENANCE_ACTIONS); + info.setProvenanceaction(provenanceAction); - // TODO calculate the trust value based on the similarity score of the elements in the CC - // info.setTrust(); + // TODO calculate the trust value based on the similarity score of the elements in the CC + // info.setTrust(); - r.setDataInfo(info); - return r; - } + r.setDataInfo(info); + return r; + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/copyRels_parameters.json b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/copyOpenorgsMergeRels_parameters.json similarity index 84% rename from dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/copyRels_parameters.json rename to dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/copyOpenorgsMergeRels_parameters.json index 715b0e74e..75054637f 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/copyRels_parameters.json +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/copyOpenorgsMergeRels_parameters.json @@ -18,9 +18,9 @@ "paramRequired": true }, { - "paramName": "e", - "paramLongName": "entityType", - "paramDescription": "type of the entity for the merge relations", + "paramName": "la", + "paramLongName": "isLookUpUrl", + "paramDescription": "the url for the lookup service", "paramRequired": true }, { diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml index a6b313cad..4c5505eb5 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml @@ -85,9 +85,6 @@ Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - @@ -96,34 +93,6 @@ - - - yarn - cluster - Copy Merge Relations - eu.dnetlib.dhp.oa.dedup.SparkCopyRels - dhp-dedup-openaire-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 - - --graphBasePath${graphBasePath} - --workingPath${workingPath} - --actionSetId${actionSetId} - --entityTypeorganization - --destinationsimrel - --numPartitions8000 - - - - - yarn @@ -147,6 +116,33 @@ --workingPath${workingPath} --numPartitions8000 + + + + + + + + yarn + cluster + Copy OpenOrgs Sim Rels + eu.dnetlib.dhp.oa.dedup.SparkCopyOpenorgsSimRels + dhp-dedup-openaire-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.shuffle.partitions=3840 + + --graphBasePath${graphBasePath} + --workingPath${workingPath} + --actionSetId${actionSetId} + --numPartitions8000 + diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml index d22f05ca8..998f3ac21 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml @@ -12,6 +12,10 @@ actionSetId id of the actionSet + + actionSetIdOpenorgs + id of the actionSet for OpenOrgs dedup + workingPath path for the working directory @@ -169,17 +173,17 @@ --isLookUpUrl${isLookUpUrl} --actionSetId${actionSetId} - + - - + + yarn cluster Copy Merge Relations - eu.dnetlib.dhp.oa.dedup.SparkCopyRels + eu.dnetlib.dhp.oa.dedup.SparkCopyOpenorgsMergeRels dhp-dedup-openaire-${projectVersion}.jar --executor-memory=${sparkExecutorMemory} @@ -193,15 +197,15 @@ --graphBasePath${graphBasePath} --workingPath${workingPath} - --actionSetId${actionSetId} - --entityTypeorganization - --destinationmergerel + --isLookUpUrl${isLookUpUrl} + --actionSetId${actionSetIdOpenorgs} --numPartitions8000 + yarn @@ -222,7 +226,7 @@ --graphBasePath${graphBasePath} --workingPath${workingPath} --isLookUpUrl${isLookUpUrl} - --actionSetId${actionSetId} + --actionSetId${actionSetIdOpenorgs} @@ -253,15 +257,28 @@ + - - - - - -pb - ${graphBasePath}/relation - ${dedupGraphPath}/relation - + + yarn + cluster + Update Entity + eu.dnetlib.dhp.oa.dedup.SparkCopyRelationsNoOpenorgs + dhp-dedup-openaire-${projectVersion}.jar + + --executor-memory=${sparkExecutorMemory} + --executor-cores=${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.shuffle.partitions=3840 + + --graphBasePath${graphBasePath} + --workingPath${workingPath} + --dedupGraphPath${dedupGraphPath} + diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java index b6210013c..532bb43b2 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java @@ -48,6 +48,7 @@ import org.slf4j.LoggerFactory; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.common.DbClient; import eu.dnetlib.dhp.oa.graph.raw.common.AbstractMigrationApplication; +import eu.dnetlib.dhp.oa.graph.raw.common.MigrateAction; import eu.dnetlib.dhp.oa.graph.raw.common.VerifyNsPrefixPredicate; import eu.dnetlib.dhp.oa.graph.raw.common.VocabularyGroup; import eu.dnetlib.dhp.schema.oaf.Context; @@ -76,6 +77,9 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i public static final String SOURCE_TYPE = "source_type"; public static final String TARGET_TYPE = "target_type"; + private static final String ORG_ORG_RELTYPE = "organizationOrganization"; + private static final String ORG_ORG_SUBRELTYPE = "dedup"; + private final DbClient dbClient; private final long lastUpdateTimestamp; @@ -114,35 +118,53 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i final Predicate verifyNamespacePrefix = new VerifyNsPrefixPredicate(nsPrefixBlacklist); - final boolean processClaims = parser.get("action") != null && parser.get("action").equalsIgnoreCase("claims"); - log.info("processClaims: {}", processClaims); + final MigrateAction process = parser.get("action") != null ? MigrateAction.valueOf(parser.get("action")) + : MigrateAction.openaire; + log.info("migrateAction: {}", process); try (final MigrateDbEntitiesApplication smdbe = new MigrateDbEntitiesApplication(hdfsPath, dbUrl, dbUser, dbPassword, isLookupUrl)) { - if (processClaims) { - log.info("Processing claims..."); - smdbe.execute("queryClaims.sql", smdbe::processClaims); - } else { - log.info("Processing datasources..."); - smdbe.execute("queryDatasources.sql", smdbe::processDatasource, verifyNamespacePrefix); - log.info("Processing projects..."); - if (dbSchema.equalsIgnoreCase("beta")) { - smdbe.execute("queryProjects.sql", smdbe::processProject, verifyNamespacePrefix); - } else { - smdbe.execute("queryProjects_production.sql", smdbe::processProject, verifyNamespacePrefix); - } + switch (process) { + case claims: + log.info("Processing claims..."); + smdbe.execute("queryClaims.sql", smdbe::processClaims); + break; + case openaire: + log.info("Processing datasources..."); + smdbe.execute("queryDatasources.sql", smdbe::processDatasource, verifyNamespacePrefix); - log.info("Processing orgs..."); - smdbe.execute("queryOrganizations.sql", smdbe::processOrganization, verifyNamespacePrefix); + log.info("Processing projects..."); + if (dbSchema.equalsIgnoreCase("beta")) { + smdbe.execute("queryProjects.sql", smdbe::processProject, verifyNamespacePrefix); + } else { + smdbe.execute("queryProjects_production.sql", smdbe::processProject, verifyNamespacePrefix); + } - log.info("Processing relationsNoRemoval ds <-> orgs ..."); - smdbe - .execute( - "queryDatasourceOrganization.sql", smdbe::processDatasourceOrganization, verifyNamespacePrefix); + log.info("Processing Organizations..."); + smdbe.execute("queryOrganizations.sql", smdbe::processOrganization, verifyNamespacePrefix); - log.info("Processing projects <-> orgs ..."); - smdbe.execute("queryProjectOrganization.sql", smdbe::processProjectOrganization, verifyNamespacePrefix); + log.info("Processing relationsNoRemoval ds <-> orgs ..."); + smdbe + .execute( + "queryDatasourceOrganization.sql", smdbe::processDatasourceOrganization, + verifyNamespacePrefix); + + log.info("Processing projects <-> orgs ..."); + smdbe + .execute( + "queryProjectOrganization.sql", smdbe::processProjectOrganization, verifyNamespacePrefix); + break; + case openorgs: + log.info("Processing Openorgs..."); + smdbe + .execute( + "queryOrganizationsFromOpenOrgsDB.sql", smdbe::processOrganization, verifyNamespacePrefix); + + log.info("Processing Openorgs Merge Rels..."); + smdbe.execute("querySimilarityFromOpenOrgsDB.sql", smdbe::processOrgOrgSimRels); + + break; } log.info("All done."); } @@ -585,6 +607,43 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i } } + public List processOrgOrgSimRels(final ResultSet rs) { + try { + final DataInfo info = prepareDataInfo(rs); // TODO + + final String orgId1 = createOpenaireId(20, rs.getString("id1"), true); + final String orgId2 = createOpenaireId(40, rs.getString("id2"), true); + final String relClass = rs.getString("relclass"); + + final List collectedFrom = listKeyValues( + createOpenaireId(10, rs.getString("collectedfromid"), true), rs.getString("collectedfromname")); + + final Relation r1 = new Relation(); + r1.setRelType(ORG_ORG_RELTYPE); + r1.setSubRelType(ORG_ORG_SUBRELTYPE); + r1.setRelClass(relClass); + r1.setSource(orgId1); + r1.setTarget(orgId2); + r1.setCollectedfrom(collectedFrom); + r1.setDataInfo(info); + r1.setLastupdatetimestamp(lastUpdateTimestamp); + + final Relation r2 = new Relation(); + r2.setRelType(ORG_ORG_RELTYPE); + r2.setSubRelType(ORG_ORG_SUBRELTYPE); + r2.setRelClass(relClass); + r2.setSource(orgId2); + r2.setTarget(orgId1); + r2.setCollectedfrom(collectedFrom); + r2.setDataInfo(info); + r2.setLastupdatetimestamp(lastUpdateTimestamp); + + return Arrays.asList(r1, r2); + } catch (final Exception e) { + throw new RuntimeException(e); + } + } + @Override public void close() throws IOException { super.close(); diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MigrateAction.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MigrateAction.java new file mode 100644 index 000000000..d9ee9bb6a --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MigrateAction.java @@ -0,0 +1,9 @@ + +package eu.dnetlib.dhp.oa.graph.raw.common; + +//enum to specify the different actions available for the MigrateDbEntitiesApplication job +public enum MigrateAction { + claims, // migrate claims to the raw graph + openorgs, // migrate organizations from openorgs to the raw graph + openaire // migrate openaire entities to the raw graph +} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all/oozie_app/workflow.xml index d8146d9a2..adaee65d3 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_all/oozie_app/workflow.xml @@ -25,6 +25,18 @@ postgresPassword the password postgres + + + postgresOpenOrgsURL + the postgres URL to access to the OpenOrgs database + + + postgresOpenOrgsUser + the user of OpenOrgs database + + + postgresOpenOrgsPassword + the password of OpenOrgs database dbSchema @@ -178,14 +190,34 @@ - + eu.dnetlib.dhp.oa.graph.raw.MigrateDbEntitiesApplication - --hdfsPath${contentPath}/db_records + --hdfsPath${contentPath}/db_openaire --postgresUrl${postgresURL} --postgresUser${postgresUser} --postgresPassword${postgresPassword} --isLookupUrl${isLookupUrl} + --actionopenaire + --dbschema${dbSchema} + --nsPrefixBlacklist${nsPrefixBlacklist} + + + + + + + + + + + eu.dnetlib.dhp.oa.graph.raw.MigrateDbEntitiesApplication + --hdfsPath${contentPath}/db_openorgs + --postgresUrl${postgresOpenOrgsURL} + --postgresUser${postgresOpenOrgsUser} + --postgresPassword${postgresOpenOrgsPassword} + --isLookupUrl${isLookupUrl} + --actionopenorgs --dbschema${dbSchema} --nsPrefixBlacklist${nsPrefixBlacklist} @@ -314,7 +346,7 @@ --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --sourcePaths${contentPath}/db_records,${contentPath}/oaf_records,${contentPath}/odf_records + --sourcePaths${contentPath}/db_openaire,${contentPath}/db_openorgs,${contentPath}/oaf_records,${contentPath}/odf_records --targetPath${workingDir}/entities --isLookupUrl${isLookupUrl} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizationsFromOpenOrgsDB.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizationsFromOpenOrgsDB.sql index 3396f365c..82ece5a1c 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizationsFromOpenOrgsDB.sql +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizationsFromOpenOrgsDB.sql @@ -4,6 +4,8 @@ SELECT o.name AS legalname, array_agg(DISTINCT n.name) AS "alternativeNames", (array_agg(u.url))[1] AS websiteurl, + '' AS logourl, + o.creation_date AS dateofcollection, o.modification_date AS dateoftransformation, false AS inferred, false AS deletedbyinference, @@ -13,7 +15,17 @@ SELECT 'OpenOrgs Database' AS collectedfromname, o.country || '@@@dnet:countries' AS country, 'sysimport:crosswalk:entityregistry@@@dnet:provenance_actions' AS provenanceaction, - array_agg(DISTINCT i.otherid || '###' || i.type || '@@@dnet:pid_types') AS pid + array_agg(DISTINCT i.otherid || '###' || i.type || '@@@dnet:pid_types') AS pid, + null AS eclegalbody, + null AS eclegalperson, + null AS ecnonprofit, + null AS ecresearchorganization, + null AS echighereducation, + null AS ecinternationalorganizationeurinterests, + null AS ecinternationalorganization, + null AS ecenterprise, + null AS ecsmevalidated, + null AS ecnutscode FROM organizations o LEFT OUTER JOIN acronyms a ON (a.id = o.id) LEFT OUTER JOIN urls u ON (u.id = o.id) @@ -22,6 +34,7 @@ FROM organizations o GROUP BY o.id, o.name, + o.creation_date, o.modification_date, o.country @@ -33,6 +46,8 @@ SELECT n.name AS legalname, ARRAY[]::text[] AS "alternativeNames", (array_agg(u.url))[1] AS websiteurl, + '' AS logourl, + o.creation_date AS dateofcollection, o.modification_date AS dateoftransformation, false AS inferred, false AS deletedbyinference, @@ -42,12 +57,24 @@ SELECT 'OpenOrgs Database' AS collectedfromname, o.country || '@@@dnet:countries' AS country, 'sysimport:crosswalk:entityregistry@@@dnet:provenance_actions' AS provenanceaction, - array_agg(DISTINCT i.otherid || '###' || i.type || '@@@dnet:pid_types') AS pid + array_agg(DISTINCT i.otherid || '###' || i.type || '@@@dnet:pid_types') AS pid, + null AS eclegalbody, + null AS eclegalperson, + null AS ecnonprofit, + null AS ecresearchorganization, + null AS echighereducation, + null AS ecinternationalorganizationeurinterests, + null AS ecinternationalorganization, + null AS ecenterprise, + null AS ecsmevalidated, + null AS ecnutscode FROM other_names n LEFT OUTER JOIN organizations o ON (n.id = o.id) LEFT OUTER JOIN urls u ON (u.id = o.id) LEFT OUTER JOIN other_ids i ON (i.id = o.id) GROUP BY - o.id, o.modification_date, o.country, n.name - - + o.id, + o.creation_date, + o.modification_date, + o.country, + n.name; \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/querySimilarityFromOpenOrgsDB.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/querySimilarityFromOpenOrgsDB.sql index 4407559c6..138bf6a96 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/querySimilarityFromOpenOrgsDB.sql +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/querySimilarityFromOpenOrgsDB.sql @@ -1,17 +1,47 @@ -SELECT local_id AS id1, oa_original_id AS id2 FROM openaire_simrels WHERE reltype = 'is_similar' +-- relations approved by the user +SELECT + local_id AS id1, + oa_original_id AS id2, + 'openaire____::openorgs' AS collectedfromid, + 'OpenOrgs Database' AS collectedfromname, + false AS inferred, + false AS deletedbyinference, + 0.99 AS trust, + '' AS inferenceprovenance, + 'isSimilarTo' AS relclass +FROM oa_duplicates WHERE reltype = 'is_similar' UNION ALL +-- relations between openorgs and mesh (alternative names) SELECT - o.id AS id1, - 'openorgsmesh'||substring(o.id, 13)||'-'||md5(a.acronym) AS id2 -FROM acronyms a - LEFT OUTER JOIN organizations o ON (a.id = o.id) - -UNION ALL - -SELECT - o.id AS id1, - 'openorgsmesh'||substring(o.id, 13)||'-'||md5(n.name) AS id2 + o.id AS id1, + 'openorgsmesh'||substring(o.id, 13)||'-'||md5(n.name) AS id2, + 'openaire____::openorgs' AS collectedfromid, + 'OpenOrgs Database' AS collectedfromname, + false AS inferred, + false AS deletedbyinference, + 0.99 AS trust, + '' AS inferenceprovenance, + 'isSimilarTo' AS relclass FROM other_names n LEFT OUTER JOIN organizations o ON (n.id = o.id) + +UNION ALL + +-- diff relations approved by the user +SELECT + local_id AS id1, + oa_original_id AS id2, + 'openaire____::openorgs' AS collectedfromid, + 'OpenOrgs Database' AS collectedfromname, + false AS inferred, + false AS deletedbyinference, + 0.99 AS trust, + '' AS inferenceprovenance, + 'isDifferentFrom' AS relclass +FROM oa_duplicates WHERE reltype = 'is_different' + + +--TODO ??? +--Creare relazioni isDifferentFrom anche tra i suggerimenti: (A is_similar B) and (A is_different C) => (B is_different C) \ No newline at end of file From 1a85020572db5f9e8084a4ff84b11df59643ed5d Mon Sep 17 00:00:00 2001 From: miconis Date: Fri, 26 Feb 2021 10:19:28 +0100 Subject: [PATCH 05/10] bug fix in graph-mapper, changes in the implementation of the openorgs wf to create relations and populate openorgs db --- dhp-workflows/dhp-dedup-openaire/pom.xml | 5 +- .../dhp/oa/dedup/AbstractSparkAction.java | 1 + .../eu/dnetlib/dhp/oa/dedup/DedupUtility.java | 5 + .../dhp/oa/dedup/SparkCopyOpenorgs.java | 31 +- .../oa/dedup/SparkCopyOpenorgsMergeRels.java | 37 +- .../oa/dedup/SparkCopyOpenorgsSimRels.java | 145 ++++---- .../dedup/SparkCopyRelationsNoOpenorgs.java | 144 ++++---- .../dhp/oa/dedup/SparkCreateSimRels.java | 22 +- .../dhp/oa/dedup/SparkPrepareNewOrgs.java | 249 +++++++++++++ .../dhp/oa/dedup/SparkPrepareOrgRels.java | 341 ++++++++++++++++++ .../dhp/oa/dedup/SparkRemoveDiffRels.java | 131 ++++--- .../dnetlib/dhp/oa/dedup/model/OrgSimRel.java | 108 ++++++ .../oa/dedup/openorgs/oozie_app/workflow.xml | 35 +- .../oa/dedup/prepareNewOrgs_parameters.json | 62 ++++ .../oa/dedup/prepareOrgRels_parameters.json | 56 +++ .../dhp/oa/dedup/SparkOpenorgsTest.java | 296 +++++++++++++++ .../profiles/mock_orchestrator_openorgs.xml | 24 ++ .../raw/MigrateDbEntitiesApplication.java | 2 +- pom.xml | 3 - 19 files changed, 1436 insertions(+), 261 deletions(-) create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPrepareNewOrgs.java create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPrepareOrgRels.java create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/OrgSimRel.java create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/prepareNewOrgs_parameters.json create mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/prepareOrgRels_parameters.json create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsTest.java create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/profiles/mock_orchestrator_openorgs.xml diff --git a/dhp-workflows/dhp-dedup-openaire/pom.xml b/dhp-workflows/dhp-dedup-openaire/pom.xml index 03ddbcf4c..04e158542 100644 --- a/dhp-workflows/dhp-dedup-openaire/pom.xml +++ b/dhp-workflows/dhp-dedup-openaire/pom.xml @@ -90,7 +90,10 @@ com.fasterxml.jackson.core jackson-core - + + org.apache.httpcomponents + httpclient + diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/AbstractSparkAction.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/AbstractSparkAction.java index 74cecb7b6..9a1127764 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/AbstractSparkAction.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/AbstractSparkAction.java @@ -29,6 +29,7 @@ import eu.dnetlib.pace.config.DedupConfig; abstract class AbstractSparkAction implements Serializable { protected static final int NUM_PARTITIONS = 1000; + protected static final int NUM_CONNECTIONS = 20; protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper() .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupUtility.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupUtility.java index 01065510a..88873086d 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupUtility.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/DedupUtility.java @@ -95,6 +95,11 @@ public class DedupUtility { return String.format("%s/%s/%s_simrel", basePath, actionSetId, entityType); } + public static String createOpenorgsMergeRelsPath( + final String basePath, final String actionSetId, final String entityType) { + return String.format("%s/%s/%s_openorgs_mergerels", basePath, actionSetId, entityType); + } + public static String createMergeRelPath( final String basePath, final String actionSetId, final String entityType) { return String.format("%s/%s/%s_mergerel", basePath, actionSetId, entityType); diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgs.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgs.java index aa7a131e7..ff7aca627 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgs.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgs.java @@ -6,6 +6,7 @@ import java.util.Optional; 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.MapFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; @@ -19,6 +20,7 @@ import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.common.EntityType; import eu.dnetlib.dhp.schema.common.ModelSupport; import eu.dnetlib.dhp.schema.oaf.OafEntity; +import eu.dnetlib.dhp.schema.oaf.Organization; import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; @@ -34,7 +36,7 @@ public class SparkCopyOpenorgs extends AbstractSparkAction { ArgumentApplicationParser parser = new ArgumentApplicationParser( IOUtils .toString( - SparkCreateSimRels.class + SparkCopyOpenorgs.class .getResourceAsStream( "/eu/dnetlib/dhp/oa/dedup/copyOpenorgs_parameters.json"))); parser.parseArgument(args); @@ -72,7 +74,7 @@ public class SparkCopyOpenorgs extends AbstractSparkAction { final Class clazz = ModelSupport.entityTypes.get(EntityType.valueOf(subEntity)); - filterEntities(spark, entityPath, clazz) + filterOpenorgs(spark, entityPath) .write() .mode(SaveMode.Overwrite) .option("compression", "gzip") @@ -80,21 +82,20 @@ public class SparkCopyOpenorgs extends AbstractSparkAction { } - public static Dataset filterEntities( + public static Dataset filterOpenorgs( final SparkSession spark, - final String entitiesInputPath, - final Class clazz) { + final String entitiesInputPath) { - // - Dataset entities = spark - .read() - .textFile(entitiesInputPath) - .map( - (MapFunction) it -> { - T entity = OBJECT_MAPPER.readValue(it, clazz); - return entity; - }, - Encoders.kryo(clazz)); + JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + Dataset entities = spark + .createDataset( + sc + .textFile(entitiesInputPath) + .map(it -> OBJECT_MAPPER.readValue(it, Organization.class)) + .rdd(), + Encoders.bean(Organization.class)); + + entities.show(); return entities.filter(entities.col("id").contains("openorgs____")); } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java index d705fca6b..4bb46222e 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java @@ -6,14 +6,13 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; -import eu.dnetlib.dhp.schema.oaf.KeyValue; -import eu.dnetlib.dhp.schema.oaf.Qualifier; -import eu.dnetlib.pace.config.DedupConfig; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.dom4j.DocumentException; import org.slf4j.Logger; @@ -21,10 +20,13 @@ import org.slf4j.LoggerFactory; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.DataInfo; +import eu.dnetlib.dhp.schema.oaf.KeyValue; +import eu.dnetlib.dhp.schema.oaf.Qualifier; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import eu.dnetlib.pace.config.DedupConfig; //copy simrels (verified) from relation to the workdir in order to make them available for the deduplication public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction { @@ -83,17 +85,17 @@ public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction { .textFile(relationPath) .map(patchRelFn(), Encoders.bean(Relation.class)) .toJavaRDD() - .filter(this::isOpenorgs) //takes only relations coming from openorgs - .filter(this::filterOpenorgsRels) //takes only isSimilarTo relations between organizations from openorgs - .filter(this::excludeOpenorgsMesh) //excludes relations between an organization and an openorgsmesh - .filter(this::excludeNonOpenorgs); //excludes relations with no openorgs id involved + .filter(this::isOpenorgs) // takes only relations coming from openorgs + .filter(this::filterOpenorgsRels) // takes only isSimilarTo relations between organizations from openorgs + .filter(this::excludeOpenorgsMesh) // excludes relations between an organization and an openorgsmesh + .filter(this::excludeNonOpenorgs); // excludes relations with no openorgs id involved - //turn openorgs isSimilarTo relations into mergerels - JavaRDD mergeRels = rawRels.flatMap(rel -> { + // turn openorgs isSimilarTo relations into mergerels + JavaRDD mergeRelsRDD = rawRels.flatMap(rel -> { List mergerels = new ArrayList<>(); - String openorgsId = rel.getSource().contains("openorgs____")? rel.getSource() : rel.getTarget(); - String mergedId = rel.getSource().contains("openorgs____")? rel.getTarget() : rel.getSource(); + String openorgsId = rel.getSource().contains("openorgs____") ? rel.getSource() : rel.getTarget(); + String mergedId = rel.getSource().contains("openorgs____") ? rel.getTarget() : rel.getSource(); mergerels.add(rel(openorgsId, mergedId, "merges", dedupConf)); mergerels.add(rel(mergedId, openorgsId, "isMergedIn", dedupConf)); @@ -101,7 +103,13 @@ public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction { return mergerels.iterator(); }); - mergeRels.saveAsTextFile(outputPath); + spark + .createDataset( + mergeRelsRDD.rdd(), + Encoders.bean(Relation.class)) + .write() + .mode(SaveMode.Append) + .parquet(outputPath); } private static MapFunction patchRelFn() { @@ -116,7 +124,8 @@ public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction { private boolean filterOpenorgsRels(Relation rel) { - if (rel.getRelClass().equals("isSimilarTo") && rel.getRelType().equals("organizationOrganization") && rel.getSubRelType().equals("dedup")) + if (rel.getRelClass().equals("isSimilarTo") && rel.getRelType().equals("organizationOrganization") + && rel.getSubRelType().equals("dedup")) return true; return false; } @@ -124,7 +133,7 @@ public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction { private boolean isOpenorgs(Relation rel) { if (rel.getCollectedfrom() != null) { - for (KeyValue k: rel.getCollectedfrom()) { + for (KeyValue k : rel.getCollectedfrom()) { if (k.getValue().equals("OpenOrgs Database")) { return true; } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsSimRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsSimRels.java index 3ce676f84..b7f88a5f6 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsSimRels.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsSimRels.java @@ -6,13 +6,13 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; -import eu.dnetlib.dhp.schema.oaf.KeyValue; -import eu.dnetlib.dhp.schema.oaf.Qualifier; -import eu.dnetlib.pace.config.DedupConfig; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.graphx.Edge; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; @@ -22,97 +22,98 @@ import org.slf4j.LoggerFactory; import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.schema.oaf.DataInfo; +import eu.dnetlib.dhp.schema.oaf.KeyValue; +import eu.dnetlib.dhp.schema.oaf.Qualifier; import eu.dnetlib.dhp.schema.oaf.Relation; import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import eu.dnetlib.pace.config.DedupConfig; //copy simrels (verified) from relation to the workdir in order to make them available for the deduplication public class SparkCopyOpenorgsSimRels extends AbstractSparkAction { - private static final Logger log = LoggerFactory.getLogger(SparkCopyOpenorgsMergeRels.class); + private static final Logger log = LoggerFactory.getLogger(SparkCopyOpenorgsSimRels.class); - public SparkCopyOpenorgsSimRels(ArgumentApplicationParser parser, SparkSession spark) { - super(parser, spark); - } + public SparkCopyOpenorgsSimRels(ArgumentApplicationParser parser, SparkSession spark) { + super(parser, spark); + } - public static void main(String[] args) throws Exception { - ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkCopyOpenorgsSimRels.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/copyOpenorgsMergeRels_parameters.json"))); - parser.parseArgument(args); + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCopyOpenorgsSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/copyOpenorgsMergeRels_parameters.json"))); + parser.parseArgument(args); - SparkConf conf = new SparkConf(); - new SparkCopyOpenorgsSimRels(parser, getSparkSession(conf)) - .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); - } + SparkConf conf = new SparkConf(); + new SparkCopyOpenorgsSimRels(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } - @Override - public void run(ISLookUpService isLookUpService) - throws DocumentException, IOException, ISLookUpException { + @Override + public void run(ISLookUpService isLookUpService) + throws DocumentException, IOException, ISLookUpException { - // read oozie parameters - final String graphBasePath = parser.get("graphBasePath"); - final String actionSetId = parser.get("actionSetId"); - final String workingPath = parser.get("workingPath"); - final int numPartitions = Optional - .ofNullable(parser.get("numPartitions")) - .map(Integer::valueOf) - .orElse(NUM_PARTITIONS); + // read oozie parameters + final String graphBasePath = parser.get("graphBasePath"); + final String actionSetId = parser.get("actionSetId"); + final String workingPath = parser.get("workingPath"); + final int numPartitions = Optional + .ofNullable(parser.get("numPartitions")) + .map(Integer::valueOf) + .orElse(NUM_PARTITIONS); - log.info("numPartitions: '{}'", numPartitions); - log.info("graphBasePath: '{}'", graphBasePath); - log.info("actionSetId: '{}'", actionSetId); - log.info("workingPath: '{}'", workingPath); + log.info("numPartitions: '{}'", numPartitions); + log.info("graphBasePath: '{}'", graphBasePath); + log.info("actionSetId: '{}'", actionSetId); + log.info("workingPath: '{}'", workingPath); - log.info("Copying OpenOrgs SimRels"); + log.info("Copying OpenOrgs SimRels"); - final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, "organization"); + final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, "organization"); - removeOutputDir(spark, outputPath); + final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation"); - final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation"); + Dataset rawRels = spark + .read() + .textFile(relationPath) + .map(patchRelFn(), Encoders.bean(Relation.class)) + .filter(this::filterOpenorgsRels); - JavaRDD rawRels = spark - .read() - .textFile(relationPath) - .map(patchRelFn(), Encoders.bean(Relation.class)) - .toJavaRDD() - .filter(this::isOpenorgs) - .filter(this::filterOpenorgsRels); + save(rawRels, outputPath, SaveMode.Append); - save(spark.createDataset(rawRels.rdd(),Encoders.bean(Relation.class)), outputPath, SaveMode.Append); - } + log.info("Copied " + rawRels.count() + " Similarity Relations"); + } - private static MapFunction patchRelFn() { - return value -> { - final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class); - if (rel.getDataInfo() == null) { - rel.setDataInfo(new DataInfo()); - } - return rel; - }; - } + private static MapFunction patchRelFn() { + return value -> { + final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class); + if (rel.getDataInfo() == null) { + rel.setDataInfo(new DataInfo()); + } + return rel; + }; + } - private boolean filterOpenorgsRels(Relation rel) { + private boolean filterOpenorgsRels(Relation rel) { - if (rel.getRelClass().equals("isSimilarTo") && rel.getRelType().equals("organizationOrganization") && rel.getSubRelType().equals("dedup")) - return true; - return false; - } + if (rel.getRelClass().equals("isSimilarTo") && rel.getRelType().equals("organizationOrganization") + && rel.getSubRelType().equals("dedup") && isOpenorgs(rel)) + return true; + return false; + } - private boolean isOpenorgs(Relation rel) { + private boolean isOpenorgs(Relation rel) { - if (rel.getCollectedfrom() != null) { - for (KeyValue k: rel.getCollectedfrom()) { - if (k.getValue().equals("OpenOrgs Database")) { - return true; - } - } - } - return false; - } + if (rel.getCollectedfrom() != null) { + for (KeyValue k : rel.getCollectedfrom()) { + if (k.getValue() != null && k.getValue().equals("OpenOrgs Database")) { + return true; + } + } + } + return false; + } } - diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRelationsNoOpenorgs.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRelationsNoOpenorgs.java index 319c40d8d..64a110892 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRelationsNoOpenorgs.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRelationsNoOpenorgs.java @@ -1,12 +1,9 @@ + package eu.dnetlib.dhp.oa.dedup; -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.*; -import eu.dnetlib.dhp.utils.ISLookupClientFactory; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -import eu.dnetlib.pace.util.MapDocumentUtil; +import java.io.IOException; +import java.util.Optional; + import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -19,92 +16,95 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.*; import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; import org.dom4j.DocumentException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.Tuple2; -import java.io.IOException; -import java.util.Optional; +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import eu.dnetlib.pace.util.MapDocumentUtil; +import scala.Tuple2; public class SparkCopyRelationsNoOpenorgs extends AbstractSparkAction { - private static final Logger log = LoggerFactory.getLogger(SparkUpdateEntity.class); + private static final Logger log = LoggerFactory.getLogger(SparkUpdateEntity.class); - private static final String IDJSONPATH = "$.id"; + public SparkCopyRelationsNoOpenorgs(ArgumentApplicationParser parser, SparkSession spark) { + super(parser, spark); + } - public SparkCopyRelationsNoOpenorgs(ArgumentApplicationParser parser, SparkSession spark) { - super(parser, spark); - } + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCopyRelationsNoOpenorgs.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json"))); + parser.parseArgument(args); - public static void main(String[] args) throws Exception { - ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkCopyRelationsNoOpenorgs.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json"))); - parser.parseArgument(args); + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); - SparkConf conf = new SparkConf(); - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); - conf.registerKryoClasses(ModelSupport.getOafModelClasses()); + new SparkUpdateEntity(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } - new SparkUpdateEntity(parser, getSparkSession(conf)) - .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); - } + public void run(ISLookUpService isLookUpService) throws IOException { - public void run(ISLookUpService isLookUpService) throws IOException { + final String graphBasePath = parser.get("graphBasePath"); + final String workingPath = parser.get("workingPath"); + final String dedupGraphPath = parser.get("dedupGraphPath"); - final String graphBasePath = parser.get("graphBasePath"); - final String workingPath = parser.get("workingPath"); - final String dedupGraphPath = parser.get("dedupGraphPath"); + log.info("graphBasePath: '{}'", graphBasePath); + log.info("workingPath: '{}'", workingPath); + log.info("dedupGraphPath: '{}'", dedupGraphPath); - log.info("graphBasePath: '{}'", graphBasePath); - log.info("workingPath: '{}'", workingPath); - log.info("dedupGraphPath: '{}'", dedupGraphPath); + final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation"); + final String outputPath = DedupUtility.createEntityPath(dedupGraphPath, "relation"); - final JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + removeOutputDir(spark, outputPath); - final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation"); - final String outputPath = DedupUtility.createEntityPath(dedupGraphPath, "relation"); + JavaRDD simRels = spark + .read() + .textFile(relationPath) + .map(patchRelFn(), Encoders.bean(Relation.class)) + .toJavaRDD() + .filter(this::excludeOpenorgsRels); - removeOutputDir(spark, outputPath); + spark + .createDataset(simRels.rdd(), Encoders.bean(Relation.class)) + .write() + .mode(SaveMode.Overwrite) + .json(outputPath); - JavaRDD simRels = spark - .read() - .textFile(relationPath) - .map(patchRelFn(), Encoders.bean(Relation.class)) - .toJavaRDD() - .filter(this::excludeOpenorgsRels); + } - simRels.saveAsTextFile(outputPath); + private static MapFunction patchRelFn() { + return value -> { + final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class); + if (rel.getDataInfo() == null) { + rel.setDataInfo(new DataInfo()); + } + return rel; + }; + } - } + private boolean excludeOpenorgsRels(Relation rel) { - private static MapFunction patchRelFn() { - return value -> { - final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class); - if (rel.getDataInfo() == null) { - rel.setDataInfo(new DataInfo()); - } - return rel; - }; - } - - private boolean excludeOpenorgsRels(Relation rel) { - - if (rel.getCollectedfrom() != null) { - for (KeyValue k: rel.getCollectedfrom()) { - if (k.getValue().equals("OpenOrgs Database")) { - return false; - } - } - } - return true; - } + if (rel.getCollectedfrom() != null) { + for (KeyValue k : rel.getCollectedfrom()) { + if (k.getValue().equals("OpenOrgs Database")) { + return false; + } + } + } + return true; + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateSimRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateSimRels.java index b3ee47bfc..a7566f2e2 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateSimRels.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateSimRels.java @@ -10,6 +10,7 @@ import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; @@ -81,7 +82,6 @@ public class SparkCreateSimRels extends AbstractSparkAction { log.info("Creating simrels for: '{}'", subEntity); final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity); - removeOutputDir(spark, outputPath); JavaSparkContext sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); @@ -99,13 +99,19 @@ public class SparkCreateSimRels extends AbstractSparkAction { .createSortedBlocks(mapDocuments, dedupConf) .repartition(numPartitions); - // create relations by comparing only elements in the same group - Deduper - .computeRelations(sc, blocks, dedupConf) - .map(t -> createSimRel(t._1(), t._2(), entity)) - .repartition(numPartitions) - .map(r -> OBJECT_MAPPER.writeValueAsString(r)) - .saveAsTextFile(outputPath); + Dataset simRels = spark + .createDataset( + Deduper + .computeRelations(sc, blocks, dedupConf) + .map(t -> createSimRel(t._1(), t._2(), entity)) + .repartition(numPartitions) + .rdd(), + Encoders.bean(Relation.class)); + + save(simRels, outputPath, SaveMode.Append); + + log.info("Generated " + simRels.count() + " Similarity Relations"); + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPrepareNewOrgs.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPrepareNewOrgs.java new file mode 100644 index 000000000..3b29e1e17 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPrepareNewOrgs.java @@ -0,0 +1,249 @@ + +package eu.dnetlib.dhp.oa.dedup; + +import java.io.IOException; +import java.util.Optional; +import java.util.Properties; + +import org.apache.commons.io.IOUtils; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.FilterFunction; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.oa.dedup.model.OrgSimRel; +import eu.dnetlib.dhp.schema.common.ModelSupport; +import eu.dnetlib.dhp.schema.oaf.DataInfo; +import eu.dnetlib.dhp.schema.oaf.Organization; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import scala.Tuple2; + +public class SparkPrepareNewOrgs extends AbstractSparkAction { + + private static final Logger log = LoggerFactory.getLogger(SparkPrepareNewOrgs.class); + + public SparkPrepareNewOrgs(ArgumentApplicationParser parser, SparkSession spark) { + super(parser, spark); + } + + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkPrepareNewOrgs.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/prepareNewOrgs_parameters.json"))); + parser.parseArgument(args); + + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); + + new SparkPrepareNewOrgs(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } + + @Override + public void run(ISLookUpService isLookUpService) throws IOException { + + final String graphBasePath = parser.get("graphBasePath"); + final String isLookUpUrl = parser.get("isLookUpUrl"); + final String actionSetId = parser.get("actionSetId"); + final String workingPath = parser.get("workingPath"); + final int numConnections = Optional + .ofNullable(parser.get("numConnections")) + .map(Integer::valueOf) + .orElse(NUM_CONNECTIONS); + + final String apiUrl = Optional + .ofNullable(parser.get("apiUrl")) + .orElse(""); + + final String dbUrl = parser.get("dbUrl"); + final String dbTable = parser.get("dbTable"); + final String dbUser = parser.get("dbUser"); + final String dbPwd = parser.get("dbPwd"); + + log.info("graphBasePath: '{}'", graphBasePath); + log.info("isLookUpUrl: '{}'", isLookUpUrl); + log.info("actionSetId: '{}'", actionSetId); + log.info("workingPath: '{}'", workingPath); + log.info("numPartitions: '{}'", numConnections); + log.info("apiUrl: '{}'", apiUrl); + log.info("dbUrl: '{}'", dbUrl); + log.info("dbUser: '{}'", dbUser); + log.info("table: '{}'", dbTable); + log.info("dbPwd: '{}'", "xxx"); + + final String entityPath = DedupUtility.createEntityPath(graphBasePath, "organization"); + final String mergeRelPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, "organization"); + final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation"); + + Dataset newOrgs = createNewOrgs(spark, mergeRelPath, relationPath, entityPath); + + final Properties connectionProperties = new Properties(); + connectionProperties.put("user", dbUser); + connectionProperties.put("password", dbPwd); + + log.info("Number of New Organization created: '{}'", newOrgs.count()); + + newOrgs + .repartition(numConnections) + .write() + .mode(SaveMode.Append) + .jdbc(dbUrl, dbTable, connectionProperties); + + if (!apiUrl.isEmpty()) + updateSimRels(apiUrl); + + } + + public static Dataset createNewOrgs( + final SparkSession spark, + final String mergeRelsPath, + final String relationPath, + final String entitiesPath) { + + // collect diffrels from the raw graph relations: + JavaPairRDD diffRels = spark + .read() + .textFile(relationPath) + .map(patchRelFn(), Encoders.bean(Relation.class)) + .toJavaRDD() + .filter(r -> filterRels(r, "organization")) + // take the worst id of the diffrel: + .mapToPair(rel -> { + if (compareIds(rel.getSource(), rel.getTarget()) > 0) + return new Tuple2<>(rel.getSource(), "diffRel"); + else + return new Tuple2<>(rel.getTarget(), "diffRel"); + }) + .distinct(); + log.info("Number of DiffRels collected: '{}'", diffRels.count()); + + // collect entities: + Dataset> entities = spark + .read() + .textFile(entitiesPath) + .map( + (MapFunction>) it -> { + Organization entity = OBJECT_MAPPER.readValue(it, Organization.class); + return new Tuple2<>(entity.getId(), entity); + }, + Encoders.tuple(Encoders.STRING(), Encoders.kryo(Organization.class))); + + // collect mergerels and remove ids in the diffrels + Dataset> openorgsRels = spark + .createDataset( + spark + .read() + .load(mergeRelsPath) + .as(Encoders.bean(Relation.class)) + .where("relClass == 'isMergedIn'") + .toJavaRDD() + .mapToPair(r -> new Tuple2<>(r.getSource(), r.getTarget())) // + .leftOuterJoin(diffRels) // + .filter(rel -> !rel._2()._2().isPresent()) + .mapToPair(rel -> new Tuple2<>(rel._1(), rel._2()._1())) + .rdd(), + Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + log.info("Number of Openorgs Relations loaded: '{}'", openorgsRels.count()); + + return entities + .joinWith(openorgsRels, entities.col("_1").equalTo(openorgsRels.col("_1")), "left") + .filter((FilterFunction, Tuple2>>) t -> t._2() == null) + // take entities not in mergerels (they are single entities, therefore are new orgs) + .filter( + (FilterFunction, Tuple2>>) t -> !t + ._1() + ._1() + .contains("openorgs")) + // exclude openorgs, don't need to propose them as new orgs + .map( + (MapFunction, Tuple2>, OrgSimRel>) r -> new OrgSimRel( + "", + r._1()._2().getOriginalId().get(0), + r._1()._2().getLegalname() != null ? r._1()._2().getLegalname().getValue() : "", + r._1()._2().getLegalshortname() != null ? r._1()._2().getLegalshortname().getValue() : "", + r._1()._2().getCountry() != null ? r._1()._2().getCountry().getClassid() : "", + r._1()._2().getWebsiteurl() != null ? r._1()._2().getWebsiteurl().getValue() : "", + r._1()._2().getCollectedfrom().get(0).getValue(), ""), + Encoders.bean(OrgSimRel.class)); + + } + + private static String updateSimRels(final String apiUrl) throws IOException { + + log.info("Updating simrels on the portal"); + + final HttpGet req = new HttpGet(apiUrl); + try (final CloseableHttpClient client = HttpClients.createDefault()) { + try (final CloseableHttpResponse response = client.execute(req)) { + return IOUtils.toString(response.getEntity().getContent()); + } + } + } + + private static MapFunction patchRelFn() { + return value -> { + final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class); + if (rel.getDataInfo() == null) { + rel.setDataInfo(new DataInfo()); + } + return rel; + }; + } + + public static int compareIds(String o1, String o2) { + if (o1.contains("openorgs____") && o2.contains("openorgs____")) + return o1.compareTo(o2); + if (o1.contains("corda") && o2.contains("corda")) + return o1.compareTo(o2); + + if (o1.contains("openorgs____")) + return -1; + if (o2.contains("openorgs____")) + return 1; + + if (o1.contains("corda")) + return -1; + if (o2.contains("corda")) + return 1; + + return o1.compareTo(o2); + } + + private static boolean filterRels(Relation rel, String entityType) { + + switch (entityType) { + case "result": + if (rel.getRelClass().equals("isDifferentFrom") && rel.getRelType().equals("resultResult") + && rel.getSubRelType().equals("dedup")) + return true; + break; + case "organization": + if (rel.getRelClass().equals("isDifferentFrom") && rel.getRelType().equals("organizationOrganization") + && rel.getSubRelType().equals("dedup")) + return true; + break; + default: + return false; + } + return false; + } + +} 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 new file mode 100644 index 000000000..cbca0b326 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPrepareOrgRels.java @@ -0,0 +1,341 @@ + +package eu.dnetlib.dhp.oa.dedup; + +import java.io.IOException; +import java.util.*; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import 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.DataInfo; +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; + +public class SparkPrepareOrgRels extends AbstractSparkAction { + + private static final Logger log = LoggerFactory.getLogger(SparkPrepareOrgRels.class); + + public SparkPrepareOrgRels(ArgumentApplicationParser parser, SparkSession spark) { + super(parser, spark); + } + + public static void main(String[] args) throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/prepareOrgRels_parameters.json"))); + parser.parseArgument(args); + + SparkConf conf = new SparkConf(); + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + conf.registerKryoClasses(ModelSupport.getOafModelClasses()); + + new SparkPrepareOrgRels(parser, getSparkSession(conf)) + .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); + } + + @Override + public void run(ISLookUpService isLookUpService) throws IOException { + + final String graphBasePath = parser.get("graphBasePath"); + final String isLookUpUrl = parser.get("isLookUpUrl"); + final String actionSetId = parser.get("actionSetId"); + final String workingPath = parser.get("workingPath"); + final int numConnections = Optional + .ofNullable(parser.get("numConnections")) + .map(Integer::valueOf) + .orElse(NUM_CONNECTIONS); + + final String dbUrl = parser.get("dbUrl"); + final String dbTable = parser.get("dbTable"); + final String dbUser = parser.get("dbUser"); + final String dbPwd = parser.get("dbPwd"); + + log.info("graphBasePath: '{}'", graphBasePath); + log.info("isLookUpUrl: '{}'", isLookUpUrl); + log.info("actionSetId: '{}'", actionSetId); + log.info("workingPath: '{}'", workingPath); + log.info("numPartitions: '{}'", numConnections); + log.info("dbUrl: '{}'", dbUrl); + log.info("dbUser: '{}'", dbUser); + log.info("table: '{}'", dbTable); + log.info("dbPwd: '{}'", "xxx"); + + final String mergeRelPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, "organization"); + final String entityPath = DedupUtility.createEntityPath(graphBasePath, "organization"); + final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation"); + + Dataset relations = createRelations(spark, mergeRelPath, relationPath, entityPath); + + final Properties connectionProperties = new Properties(); + connectionProperties.put("user", dbUser); + connectionProperties.put("password", dbPwd); + + relations + .repartition(numConnections) + .write() + .mode(SaveMode.Overwrite) + .jdbc(dbUrl, dbTable, connectionProperties); + + } + + private static boolean filterRels(Relation rel, String entityType) { + + switch (entityType) { + case "result": + if (rel.getRelClass().equals("isDifferentFrom") && rel.getRelType().equals("resultResult") + && rel.getSubRelType().equals("dedup")) + return true; + break; + case "organization": + if (rel.getRelClass().equals("isDifferentFrom") && rel.getRelType().equals("organizationOrganization") + && rel.getSubRelType().equals("dedup")) + return true; + break; + default: + return false; + } + return false; + } + + // create openorgs simrels starting from mergerels, remove the diffrels + public static Dataset createRelations( + final SparkSession spark, + final String mergeRelsPath, + final String relationPath, + final String entitiesPath) { + + // collect diffrels from the raw graph relations: <, "diffRel"> + JavaRDD, String>> diffRels = spark + .read() + .textFile(relationPath) + .map(patchRelFn(), Encoders.bean(Relation.class)) + .toJavaRDD() + .filter(r -> filterRels(r, "organization")) + // put the best id as source of the diffrel: + .map(rel -> { + if (compareIds(rel.getSource(), rel.getTarget()) < 0) + return new Tuple2<>(new Tuple2<>(rel.getSource(), rel.getTarget()), "diffRel"); + else + return new Tuple2<>(new Tuple2<>(rel.getTarget(), rel.getSource()), "diffRel"); + }) + .distinct(); + log.info("Number of DiffRels collected: {}", diffRels.count()); + + // collect all the organizations + Dataset> entities = spark + .read() + .textFile(entitiesPath) + .map( + (MapFunction>) it -> { + Organization entity = OBJECT_MAPPER.readValue(it, Organization.class); + return new Tuple2<>(entity.getId(), entity); + }, + Encoders.tuple(Encoders.STRING(), Encoders.kryo(Organization.class))); + + // relations with their group (connected component id) + JavaRDD, String>> rawOpenorgsRels = spark + .read() + .load(mergeRelsPath) + .as(Encoders.bean(Relation.class)) + .where("relClass == 'merges'") + .toJavaRDD() + .mapToPair(r -> new Tuple2<>(r.getSource(), r.getTarget())) + .filter(t -> !t._2().contains("openorgsmesh")) // remove openorgsmesh: they are only for dedup + .groupByKey() + .map(g -> Lists.newArrayList(g._2())) + .filter(l -> l.size() > 1) + .flatMap(l -> { + String groupId = "group::" + UUID.randomUUID(); + List ids = sortIds(l); // sort IDs by type + List, String>> rels = new ArrayList<>(); + String source = ids.get(0); + for (String target : ids) { + rels.add(new Tuple2<>(new Tuple2<>(source, target), groupId)); + } + + return rels.iterator(); + }); + log.info("Number of Raw Openorgs Relations created: {}", rawOpenorgsRels.count()); + + // filter out diffRels + JavaRDD> openorgsRels = rawOpenorgsRels + .union(diffRels) + // concatenation of source and target: or + .mapToPair(t -> new Tuple2<>(t._1()._1() + "@@@" + t._1()._2(), t._2())) + .groupByKey() + .map( + g -> new Tuple2<>(g._1(), StreamSupport + .stream(g._2().spliterator(), false) + .collect(Collectors.toList()))) + // : take only relations with only the group_id, it + // means they are correct. If the diffRel is present the relation has to be removed + .filter(g -> g._2().size() == 1 && g._2().get(0).contains("group::")) + .map( + t -> new Tuple3<>( + t._1().split("@@@")[0], + t._1().split("@@@")[1], + t._2().get(0))); + log.info("Number of Openorgs Relations created: '{}'", openorgsRels.count()); + + // + Dataset> relations = spark + .createDataset( + openorgsRels.rdd(), + Encoders.tuple(Encoders.STRING(), Encoders.STRING(), Encoders.STRING())); + + // create orgsimrels + Dataset> relations2 = relations + .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(), + r._1()._3()), + Encoders.bean(OrgSimRel.class)) + .map( + (MapFunction>) o -> new Tuple2<>(o.getLocal_id(), o), + Encoders.tuple(Encoders.STRING(), Encoders.bean(OrgSimRel.class))); + + return relations2 + .joinWith(entities, relations2.col("_1").equalTo(entities.col("_1")), "inner") + .map( + (MapFunction, Tuple2>, OrgSimRel>) r -> { + OrgSimRel orgSimRel = r._1()._2(); + orgSimRel.setLocal_id(r._2()._2().getOriginalId().get(0)); + return orgSimRel; + }, + Encoders.bean(OrgSimRel.class)); + + } + + public static int compareIds(String o1, String o2) { + if (o1.contains("openorgs____") && o2.contains("openorgs____")) + return o1.compareTo(o2); + if (o1.contains("corda") && o2.contains("corda")) + return o1.compareTo(o2); + + if (o1.contains("openorgs____")) + return -1; + if (o2.contains("openorgs____")) + return 1; + + if (o1.contains("corda")) + return -1; + if (o2.contains("corda")) + return 1; + + return o1.compareTo(o2); + } + + // Sort IDs basing on the type. Priority: 1) openorgs, 2)corda, 3)alphabetic + public static List sortIds(List ids) { + ids.sort((o1, o2) -> compareIds(o1, o2)); + return ids; + } + + public static Dataset createRelationsFromScratch( + final SparkSession spark, + final String mergeRelsPath, + final String entitiesPath) { + + // + Dataset> entities = spark + .read() + .textFile(entitiesPath) + .map( + (MapFunction>) it -> { + Organization entity = OBJECT_MAPPER.readValue(it, Organization.class); + return new Tuple2<>(entity.getId(), entity); + }, + Encoders.tuple(Encoders.STRING(), Encoders.kryo(Organization.class))); + + Dataset> relations = spark + .createDataset( + spark + .read() + .load(mergeRelsPath) + .as(Encoders.bean(Relation.class)) + .where("relClass == 'merges'") + .toJavaRDD() + .mapToPair(r -> new Tuple2<>(r.getSource(), r.getTarget())) + .groupByKey() + .flatMap(g -> { + List> rels = new ArrayList<>(); + for (String id1 : g._2()) { + for (String id2 : g._2()) { + if (!id1.equals(id2)) + if (id1.contains("openorgs____") && !id2.contains("openorgsmesh")) + rels.add(new Tuple2<>(id1, id2)); + } + } + return rels.iterator(); + }) + .rdd(), + Encoders.tuple(Encoders.STRING(), Encoders.STRING())); + + Dataset> relations2 = relations // + .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()), + Encoders.bean(OrgSimRel.class)) + .map( + (MapFunction>) o -> new Tuple2<>(o.getLocal_id(), o), + Encoders.tuple(Encoders.STRING(), Encoders.bean(OrgSimRel.class))); + + return relations2 + .joinWith(entities, relations2.col("_1").equalTo(entities.col("_1")), "inner") + .map( + (MapFunction, Tuple2>, OrgSimRel>) r -> { + OrgSimRel orgSimRel = r._1()._2(); + orgSimRel.setLocal_id(r._2()._2().getOriginalId().get(0)); + return orgSimRel; + }, + Encoders.bean(OrgSimRel.class)); + + } + + private static MapFunction patchRelFn() { + return value -> { + final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class); + if (rel.getDataInfo() == null) { + rel.setDataInfo(new DataInfo()); + } + return rel; + }; + } +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkRemoveDiffRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkRemoveDiffRels.java index 6f012e00a..4c0bfadf0 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkRemoveDiffRels.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkRemoveDiffRels.java @@ -27,6 +27,8 @@ import org.dom4j.DocumentException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.collect.Iterables; + import eu.dnetlib.dhp.application.ArgumentApplicationParser; import eu.dnetlib.dhp.oa.dedup.graph.ConnectedComponent; import eu.dnetlib.dhp.oa.dedup.graph.GraphProcessor; @@ -95,6 +97,9 @@ public class SparkRemoveDiffRels extends AbstractSparkAction { final String relationPath = DedupUtility.createEntityPath(graphBasePath, subEntity); + final String openorgsMergeRelsPath = DedupUtility + .createOpenorgsMergeRelsPath(workingPath, actionSetId, subEntity); + final int maxIterations = dedupConf.getWf().getMaxIterations(); log.info("Max iterations {}", maxIterations); @@ -105,67 +110,103 @@ public class SparkRemoveDiffRels extends AbstractSparkAction { .where("relClass == 'merges'") .toJavaRDD(); + System.out.println("mergeRelsRDD = " + mergeRelsRDD.count()); + +// JavaRDD, String>> diffRelsRDD = spark +// .read() +// .textFile(relationPath) +// .map(patchRelFn(), Encoders.bean(Relation.class)) +// .toJavaRDD() +// .filter(r -> filterRels(r, entity)) +// .map(rel -> { +// if (rel.getSource().compareTo(rel.getTarget()) < 0) +// return new Tuple2<>(new Tuple2<>(rel.getSource(), rel.getTarget()), "diffRel"); +// else +// return new Tuple2<>(new Tuple2<>(rel.getTarget(), rel.getSource()), "diffRel"); +// }); + // THIS IS FOR TESTING PURPOSE JavaRDD, String>> diffRelsRDD = spark .read() - .textFile(relationPath) - .map(patchRelFn(), Encoders.bean(Relation.class)) + .load(mergeRelsPath) + .as(Encoders.bean(Relation.class)) .toJavaRDD() - .filter(r -> filterRels(r, entity)) .map(rel -> { if (rel.getSource().compareTo(rel.getTarget()) < 0) return new Tuple2<>(new Tuple2<>(rel.getSource(), rel.getTarget()), "diffRel"); else return new Tuple2<>(new Tuple2<>(rel.getTarget(), rel.getSource()), "diffRel"); + }) + .distinct(); + + System.out.println("diffRelsRDD = " + diffRelsRDD.count()); + +// JavaRDD, String>> flatMergeRels = mergeRelsRDD +// .mapToPair(rel -> new Tuple2<>(rel.getSource(), rel.getTarget())) +// .groupByKey() +// .flatMap(g -> { +// List, String>> rels = new ArrayList<>(); +// +// List ids = StreamSupport +// .stream(g._2().spliterator(), false) +// .collect(Collectors.toList()); +// +// for (int i = 0; i < ids.size(); i++) { +// for (int j = i + 1; j < ids.size(); j++) { +// if (ids.get(i).compareTo(ids.get(j)) < 0) +// rels.add(new Tuple2<>(new Tuple2<>(ids.get(i), ids.get(j)), g._1())); +// else +// rels.add(new Tuple2<>(new Tuple2<>(ids.get(j), ids.get(i)), g._1())); +// } +// } +// return rels.iterator(); +// +// }); + JavaRDD, String>> mergeRels = mergeRelsRDD + .map(rel -> { + if (rel.getSource().compareTo(rel.getTarget()) < 0) + return new Tuple2<>(new Tuple2<>(rel.getSource(), rel.getTarget()), "mergeRel"); + else + return new Tuple2<>(new Tuple2<>(rel.getTarget(), rel.getSource()), "mergeRel"); }); + System.out.println("mergeRelsProcessed = " + mergeRels.count()); - JavaRDD, String>> flatMergeRels = mergeRelsRDD - .mapToPair(rel -> new Tuple2<>(rel.getSource(), rel.getTarget())) - .groupByKey() - .flatMap(g -> { - List, String>> rels = new ArrayList<>(); - - List ids = StreamSupport - .stream(g._2().spliterator(), false) - .collect(Collectors.toList()); - - for (int i = 0; i < ids.size(); i++) { - for (int j = i + 1; j < ids.size(); j++) { - if (ids.get(i).compareTo(ids.get(j)) < 0) - rels.add(new Tuple2<>(new Tuple2<>(ids.get(i), ids.get(j)), g._1())); - else - rels.add(new Tuple2<>(new Tuple2<>(ids.get(j), ids.get(i)), g._1())); - } - } - return rels.iterator(); - - }); - - JavaRDD purgedMergeRels = flatMergeRels +// JavaRDD purgedMergeRels = flatMergeRels +// .union(diffRelsRDD) +// .mapToPair(rel -> new Tuple2<>(rel._1(), Arrays.asList(rel._2()))) +// .reduceByKey((a, b) -> { +// List list = new ArrayList(); +// list.addAll(a); +// list.addAll(b); +// return list; +// }) +// .filter(rel -> rel._2().size() == 1) +// .mapToPair(rel -> new Tuple2<>(rel._2().get(0), rel._1())) +// .flatMap(rel -> { +// List> rels = new ArrayList<>(); +// String source = rel._1(); +// rels.add(new Tuple2<>(source, rel._2()._1())); +// rels.add(new Tuple2<>(source, rel._2()._2())); +// return rels.iterator(); +// }) +// .distinct() +// .flatMap(rel -> tupleToMergeRel(rel, dedupConf)); + JavaRDD purgedMergeRels = mergeRels .union(diffRelsRDD) - .mapToPair(rel -> new Tuple2<>(rel._1(), Arrays.asList(rel._2()))) - .reduceByKey((a, b) -> { - List list = new ArrayList(); - list.addAll(a); - list.addAll(b); - return list; - }) - .filter(rel -> rel._2().size() == 1) - .mapToPair(rel -> new Tuple2<>(rel._2().get(0), rel._1())) - .flatMap(rel -> { - List> rels = new ArrayList<>(); - String source = rel._1(); - rels.add(new Tuple2<>(source, rel._2()._1())); - rels.add(new Tuple2<>(source, rel._2()._2())); - return rels.iterator(); - }) - .distinct() - .flatMap(rel -> tupleToMergeRel(rel, dedupConf)); + .mapToPair(t -> new Tuple2<>(t._1()._1() + "|||" + t._1()._2(), t._2())) + .groupByKey() + .filter(g -> Iterables.size(g._2()) == 1) + .flatMap( + t -> tupleToMergeRel( + new Tuple2<>(t._1().split("\\|\\|\\|")[0], t._1().split("\\|\\|\\|")[1]), + dedupConf)); + + System.out.println("purgedMergeRels = " + purgedMergeRels.count()); spark .createDataset(purgedMergeRels.rdd(), Encoders.bean(Relation.class)) .write() .mode(SaveMode.Overwrite) - .json(mergeRelsPath); + .json(openorgsMergeRelsPath); } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/OrgSimRel.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/OrgSimRel.java new file mode 100644 index 000000000..65f383500 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/OrgSimRel.java @@ -0,0 +1,108 @@ + +package eu.dnetlib.dhp.oa.dedup.model; + +import java.io.Serializable; + +public class OrgSimRel implements Serializable { + + String local_id; + String oa_original_id; + String oa_name; + String oa_acronym; + String oa_country; + String oa_url; + String oa_collectedfrom; + String group_id; + + public OrgSimRel() { + } + + public OrgSimRel(String local_id, String oa_original_id, String oa_name, String oa_acronym, String oa_country, + String oa_url, String oa_collectedfrom, String group_id) { + this.local_id = local_id; + this.oa_original_id = oa_original_id; + this.oa_name = oa_name; + this.oa_acronym = oa_acronym; + this.oa_country = oa_country; + this.oa_url = oa_url; + this.oa_collectedfrom = oa_collectedfrom; + this.group_id = group_id; + } + + public String getLocal_id() { + return local_id; + } + + public void setLocal_id(String local_id) { + this.local_id = local_id; + } + + public String getOa_original_id() { + return oa_original_id; + } + + public void setOa_original_id(String oa_original_id) { + this.oa_original_id = oa_original_id; + } + + public String getOa_name() { + return oa_name; + } + + public void setOa_name(String oa_name) { + this.oa_name = oa_name; + } + + public String getOa_acronym() { + return oa_acronym; + } + + public void setOa_acronym(String oa_acronym) { + this.oa_acronym = oa_acronym; + } + + public String getOa_country() { + return oa_country; + } + + public void setOa_country(String oa_country) { + this.oa_country = oa_country; + } + + public String getOa_url() { + return oa_url; + } + + public void setOa_url(String oa_url) { + this.oa_url = oa_url; + } + + public String getOa_collectedfrom() { + return oa_collectedfrom; + } + + public void setOa_collectedfrom(String oa_collectedfrom) { + this.oa_collectedfrom = oa_collectedfrom; + } + + public String getGroup_id() { + return group_id; + } + + public void setGroup_id(String group_id) { + this.group_id = group_id; + } + + @Override + public String toString() { + return "OrgSimRel{" + + "local_id='" + local_id + '\'' + + ", oa_original_id='" + oa_original_id + '\'' + + ", oa_name='" + oa_name + '\'' + + ", oa_acronym='" + oa_acronym + '\'' + + ", oa_country='" + oa_country + '\'' + + ", oa_url='" + oa_url + '\'' + + ", oa_collectedfrom='" + oa_collectedfrom + '\'' + + '}'; + } +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml index 4c5505eb5..339e99084 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml @@ -79,7 +79,7 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] @@ -88,8 +88,9 @@ + - + @@ -120,7 +121,7 @@ - + yarn @@ -139,6 +140,7 @@ --conf spark.sql.shuffle.partitions=3840 --graphBasePath${graphBasePath} + --isLookUpUrl${isLookUpUrl} --workingPath${workingPath} --actionSetId${actionSetId} --numPartitions8000 @@ -170,33 +172,6 @@ --actionSetId${actionSetId} --cutConnectedComponent${cutConnectedComponent} - - - - - - - yarn - cluster - Create Merge Relations - eu.dnetlib.dhp.oa.dedup.SparkRemoveDiffRels - dhp-dedup-openaire-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 - - --graphBasePath${graphBasePath} - --isLookUpUrl${isLookUpUrl} - --actionSetId${actionSetId} - --workingPath${workingPath} - --numPartitions8000 - diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/prepareNewOrgs_parameters.json b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/prepareNewOrgs_parameters.json new file mode 100644 index 000000000..b70d1af28 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/prepareNewOrgs_parameters.json @@ -0,0 +1,62 @@ +[ + { + "paramName": "i", + "paramLongName": "graphBasePath", + "paramDescription": "the base path of raw graph", + "paramRequired": true + }, + { + "paramName": "w", + "paramLongName": "workingPath", + "paramDescription": "the working directory path", + "paramRequired": true + }, + { + "paramName": "la", + "paramLongName": "isLookUpUrl", + "paramDescription": "the url of the lookup service", + "paramRequired": true + }, + { + "paramName": "asi", + "paramLongName": "actionSetId", + "paramDescription": "the id of the actionset (orchestrator)", + "paramRequired": true + }, + { + "paramName": "nc", + "paramLongName": "numConnections", + "paramDescription": "number of connections to the postgres db (for the write operation)", + "paramRequired": false + }, + { + "paramName": "au", + "paramLongName": "apiUrl", + "paramDescription": "the url for the APIs of the openorgs service", + "paramRequired": false + }, + { + "paramName": "du", + "paramLongName": "dbUrl", + "paramDescription": "the url of the database", + "paramRequired": true + }, + { + "paramName": "dusr", + "paramLongName": "dbUser", + "paramDescription": "the user of the database", + "paramRequired": true + }, + { + "paramName": "t", + "paramLongName": "dbTable", + "paramDescription": "the name of the table in the database", + "paramRequired": true + }, + { + "paramName": "dpwd", + "paramLongName": "dbPwd", + "paramDescription": "the password for the user of the database", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/prepareOrgRels_parameters.json b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/prepareOrgRels_parameters.json new file mode 100644 index 000000000..2119cbc3a --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/prepareOrgRels_parameters.json @@ -0,0 +1,56 @@ +[ + { + "paramName": "i", + "paramLongName": "graphBasePath", + "paramDescription": "the base path of raw graph", + "paramRequired": true + }, + { + "paramName": "w", + "paramLongName": "workingPath", + "paramDescription": "the working directory path", + "paramRequired": true + }, + { + "paramName": "la", + "paramLongName": "isLookUpUrl", + "paramDescription": "the url of the lookup service", + "paramRequired": true + }, + { + "paramName": "asi", + "paramLongName": "actionSetId", + "paramDescription": "the id of the actionset (orchestrator)", + "paramRequired": true + }, + { + "paramName": "nc", + "paramLongName": "numConnections", + "paramDescription": "number of connections to the postgres db (for the write operation)", + "paramRequired": false + }, + { + "paramName": "du", + "paramLongName": "dbUrl", + "paramDescription": "the url of the database", + "paramRequired": true + }, + { + "paramName": "dusr", + "paramLongName": "dbUser", + "paramDescription": "the user of the database", + "paramRequired": true + }, + { + "paramName": "t", + "paramLongName": "dbTable", + "paramDescription": "the name of the table in the database", + "paramRequired": true + }, + { + "paramName": "dpwd", + "paramLongName": "dbPwd", + "paramDescription": "the password for the user of the database", + "paramRequired": true + } +] \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsTest.java new file mode 100644 index 000000000..f8627d023 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsTest.java @@ -0,0 +1,296 @@ + +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.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 org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +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 eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.DataInfo; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.utils.ISLookupClientFactory; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import jdk.nashorn.internal.ir.annotations.Ignore; + +@ExtendWith(MockitoExtension.class) +public class SparkOpenorgsTest 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"; + + protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + + @BeforeAll + public static void cleanUp() throws IOException, URISyntaxException { + + testGraphBasePath = Paths + .get(SparkOpenorgsTest.class.getResource("/eu/dnetlib/dhp/dedup/entities").toURI()) + .toFile() + .getAbsolutePath(); + testOutputBasePath = createTempDirectory(SparkOpenorgsTest.class.getSimpleName() + "-") + .toAbsolutePath() + .toString(); + testDedupGraphBasePath = createTempDirectory(SparkOpenorgsTest.class.getSimpleName() + "-") + .toAbsolutePath() + .toString(); + +// FileUtils.deleteDirectory(new File(testOutputBasePath)); + FileUtils.deleteDirectory(new File(testDedupGraphBasePath)); + FileUtils.deleteDirectory(new File("/tmp/test-orchestrator/organization_openorgs_mergerels")); + + final SparkConf conf = new SparkConf(); + conf.set("spark.sql.shuffle.partitions", "200"); + spark = SparkSession + .builder() + .appName(SparkDedupTest.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( + IOUtils + .toString( + SparkDedupTest.class + .getResourceAsStream( + "/eu/dnetlib/dhp/dedup/profiles/mock_orchestrator_openorgs.xml"))); + + lenient() + .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("organization"))) + .thenReturn( + IOUtils + .toString( + SparkDedupTest.class + .getResourceAsStream( + "/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json"))); + } + + @Test + public void copyOpenorgsTest() throws Exception { + + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCopyOpenorgs.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/copyOpenorgs_parameters.json"))); + parser + .parseArgument( + new String[] { + "-i", testGraphBasePath, + "-asi", testActionSetId, + "-w", testOutputBasePath, + "-np", "50" + }); + + new SparkCopyOpenorgs(parser, spark).run(isLookUpService); + + long orgs_deduprecord = jsc + .textFile(testOutputBasePath + "/" + testActionSetId + "/organization_deduprecord") + .count(); + + assertEquals(0, orgs_deduprecord); + } + + @Test + public void copyOpenorgsMergeRels() throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCopyOpenorgsMergeRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/copyOpenorgsMergeRels_parameters.json"))); + parser + .parseArgument( + new String[] { + "-i", testGraphBasePath, + "-asi", testActionSetId, + "-w", testOutputBasePath, + "-la", "lookupurl", + "-np", "50" + }); + + new SparkCopyOpenorgsMergeRels(parser, spark).run(isLookUpService); + + long orgs_mergerel = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel") + .count(); + + assertEquals(0, orgs_mergerel); + + } + + @Test + public void copyOpenorgsSimRels() throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCopyOpenorgsSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/copyOpenorgsMergeRels_parameters.json"))); + parser + .parseArgument( + new String[] { + "-i", testGraphBasePath, + "-asi", testActionSetId, + "-w", testOutputBasePath, + "-la", "lookupurl", + "-np", "50" + }); + + new SparkCopyOpenorgsSimRels(parser, spark).run(isLookUpService); + + long orgs_simrel = spark + .read() + .textFile(testOutputBasePath + "/" + testActionSetId + "/organization_simrel") + .count(); + + System.out.println("orgs_simrel = " + orgs_simrel); + } + + @Test + public void createSimRelsTest() throws Exception { + + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json"))); + parser + .parseArgument( + new String[] { + "-i", testGraphBasePath, + "-asi", testActionSetId, + "-la", "lookupurl", + "-w", "/tmp", + "-np", "50" + }); + + new SparkCreateSimRels(parser, spark).run(isLookUpService); + + long orgs_simrel = spark + .read() + .textFile("/tmp/" + testActionSetId + "/organization_simrel") + .count(); + + assertEquals(3082, orgs_simrel); + } + + @Test + public void createMergeRelsTest() throws Exception { + + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateMergeRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json"))); + parser + .parseArgument( + new String[] { + "-i", + testGraphBasePath, + "-asi", + testActionSetId, + "-la", + "lookupurl", + "-w", + "/tmp" + }); + + new SparkCreateMergeRels(parser, spark).run(isLookUpService); + + long orgs_mergerel = spark + .read() + .load("/tmp/" + testActionSetId + "/organization_mergerel") + .count(); + assertEquals(1272, orgs_mergerel); + } + + @Test + public void copyRelationsNoOpenorgsTest() throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCopyRelationsNoOpenorgs.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/updateEntity_parameters.json"))); + parser + .parseArgument( + new String[] { + "-i", testGraphBasePath, + "-w", testOutputBasePath, + "-o", testDedupGraphBasePath + }); + + new SparkCopyRelationsNoOpenorgs(parser, spark).run(isLookUpService); + + long relations = jsc.textFile(testDedupGraphBasePath + "/relation").count(); + +// Dataset relsRDD = spark.read().textFile(testDedupGraphBasePath + "/relation").map(patchRelFn(), Encoders.bean(Relation.class)); + + assertEquals(500, relations); + } + + @AfterAll + public static void finalCleanUp() throws IOException { + FileUtils.deleteDirectory(new File(testOutputBasePath)); + FileUtils.deleteDirectory(new File(testDedupGraphBasePath)); + } + + private static MapFunction patchRelFn() { + return value -> { + final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class); + if (rel.getDataInfo() == null) { + rel.setDataInfo(new DataInfo()); + } + return rel; + }; + } +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/profiles/mock_orchestrator_openorgs.xml b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/profiles/mock_orchestrator_openorgs.xml new file mode 100644 index 000000000..59b6179ed --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/profiles/mock_orchestrator_openorgs.xml @@ -0,0 +1,24 @@ + +
+ + + + + +
+ + + + + + + + + + + + + + SECURITY_PARAMETERS + +
\ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java index 532bb43b2..3e5030eaa 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java @@ -612,7 +612,7 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i final DataInfo info = prepareDataInfo(rs); // TODO final String orgId1 = createOpenaireId(20, rs.getString("id1"), true); - final String orgId2 = createOpenaireId(40, rs.getString("id2"), true); + final String orgId2 = createOpenaireId(20, rs.getString("id2"), true); final String relClass = rs.getString("relclass"); final List collectedFrom = listKeyValues( diff --git a/pom.xml b/pom.xml index a2e2587b3..25a52064a 100644 --- a/pom.xml +++ b/pom.xml @@ -114,9 +114,6 @@ test - - - From 98854b01245ba086cb7aeaf7f6fc12656db7400d Mon Sep 17 00:00:00 2001 From: miconis Date: Fri, 19 Mar 2021 16:57:40 +0100 Subject: [PATCH 06/10] minor changes --- .../dhp/oa/dedup/SparkCopyOpenorgs.java | 11 +- .../oa/dedup/SparkCopyOpenorgsMergeRels.java | 18 +- .../dedup/SparkCopyRelationsNoOpenorgs.java | 18 +- .../dhp/oa/dedup/SparkRemoveDiffRels.java | 281 ------------------ .../oa/dedup/openorgs/oozie_app/workflow.xml | 14 +- .../dhp/oa/dedup/scan/oozie_app/workflow.xml | 21 +- .../dhp/oa/dedup/SparkOpenorgsTest.java | 77 +---- .../openorgs/organization/organization.gz | Bin 0 -> 71922 bytes .../dhp/dedup/openorgs/relation/relation.gz | Bin 0 -> 22489 bytes 9 files changed, 56 insertions(+), 384 deletions(-) delete mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkRemoveDiffRels.java create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/openorgs/organization/organization.gz create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/openorgs/relation/relation.gz diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgs.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgs.java index ff7aca627..7984f0104 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgs.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgs.java @@ -68,12 +68,9 @@ public class SparkCopyOpenorgs extends AbstractSparkAction { log.info("Copying openorgs to the working dir"); final String outputPath = DedupUtility.createDedupRecordPath(workingPath, actionSetId, subEntity); - removeOutputDir(spark, outputPath); final String entityPath = DedupUtility.createEntityPath(graphBasePath, subEntity); - final Class clazz = ModelSupport.entityTypes.get(EntityType.valueOf(subEntity)); - filterOpenorgs(spark, entityPath) .write() .mode(SaveMode.Overwrite) @@ -95,9 +92,13 @@ public class SparkCopyOpenorgs extends AbstractSparkAction { .rdd(), Encoders.bean(Organization.class)); - entities.show(); + log.info("Number of organization entities processed: {}", entities.count()); - return entities.filter(entities.col("id").contains("openorgs____")); + entities = entities.filter(entities.col("id").contains("openorgs____")); + + log.info("Number of Openorgs organization entities: {}", entities.count()); + + return entities; } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java index 4bb46222e..201043a08 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java @@ -74,8 +74,6 @@ public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction { final String outputPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, "organization"); - removeOutputDir(spark, outputPath); - final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation"); DedupConfig dedupConf = getConfigurations(isLookUpService, actionSetId).get(0); @@ -85,11 +83,13 @@ public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction { .textFile(relationPath) .map(patchRelFn(), Encoders.bean(Relation.class)) .toJavaRDD() - .filter(this::isOpenorgs) // takes only relations coming from openorgs - .filter(this::filterOpenorgsRels) // takes only isSimilarTo relations between organizations from openorgs - .filter(this::excludeOpenorgsMesh) // excludes relations between an organization and an openorgsmesh + .filter(this::isOpenorgs) + .filter(this::filterOpenorgsRels) + .filter(this::excludeOpenorgsMesh) .filter(this::excludeNonOpenorgs); // excludes relations with no openorgs id involved + log.info("Number of raw Openorgs Relations collected: {}", rawRels.count()); + // turn openorgs isSimilarTo relations into mergerels JavaRDD mergeRelsRDD = rawRels.flatMap(rel -> { List mergerels = new ArrayList<>(); @@ -103,6 +103,8 @@ public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction { return mergerels.iterator(); }); + log.info("Number of Openorgs Merge Relations created: {}", mergeRelsRDD.count()); + spark .createDataset( mergeRelsRDD.rdd(), @@ -134,7 +136,7 @@ public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction { if (rel.getCollectedfrom() != null) { for (KeyValue k : rel.getCollectedfrom()) { - if (k.getValue().equals("OpenOrgs Database")) { + if (k.getValue() != null && k.getValue().equals("OpenOrgs Database")) { return true; } } @@ -144,7 +146,7 @@ public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction { private boolean excludeOpenorgsMesh(Relation rel) { - if (rel.getSource().equals("openorgsmesh") || rel.getTarget().equals("openorgsmesh")) { + if (rel.getSource().contains("openorgsmesh") || rel.getTarget().contains("openorgsmesh")) { return false; } return true; @@ -152,7 +154,7 @@ public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction { private boolean excludeNonOpenorgs(Relation rel) { - if (rel.getSource().equals("openorgs____") || rel.getTarget().equals("openorgs____")) { + if (rel.getSource().contains("openorgs____") || rel.getTarget().contains("openorgs____")) { return true; } return false; diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRelationsNoOpenorgs.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRelationsNoOpenorgs.java index 64a110892..71bab79d0 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRelationsNoOpenorgs.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyRelationsNoOpenorgs.java @@ -33,7 +33,7 @@ import scala.Tuple2; public class SparkCopyRelationsNoOpenorgs extends AbstractSparkAction { - private static final Logger log = LoggerFactory.getLogger(SparkUpdateEntity.class); + private static final Logger log = LoggerFactory.getLogger(SparkCopyRelationsNoOpenorgs.class); public SparkCopyRelationsNoOpenorgs(ArgumentApplicationParser parser, SparkSession spark) { super(parser, spark); @@ -52,7 +52,7 @@ public class SparkCopyRelationsNoOpenorgs extends AbstractSparkAction { conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); conf.registerKryoClasses(ModelSupport.getOafModelClasses()); - new SparkUpdateEntity(parser, getSparkSession(conf)) + new SparkCopyRelationsNoOpenorgs(parser, getSparkSession(conf)) .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); } @@ -69,14 +69,14 @@ public class SparkCopyRelationsNoOpenorgs extends AbstractSparkAction { final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation"); final String outputPath = DedupUtility.createEntityPath(dedupGraphPath, "relation"); - removeOutputDir(spark, outputPath); - JavaRDD simRels = spark .read() .textFile(relationPath) .map(patchRelFn(), Encoders.bean(Relation.class)) .toJavaRDD() - .filter(this::excludeOpenorgsRels); + .filter(x -> !isOpenorgs(x)); + + log.info("Number of non-Openorgs relations collected: {}", simRels.count()); spark .createDataset(simRels.rdd(), Encoders.bean(Relation.class)) @@ -96,15 +96,15 @@ public class SparkCopyRelationsNoOpenorgs extends AbstractSparkAction { }; } - private boolean excludeOpenorgsRels(Relation rel) { + private boolean isOpenorgs(Relation rel) { if (rel.getCollectedfrom() != null) { for (KeyValue k : rel.getCollectedfrom()) { - if (k.getValue().equals("OpenOrgs Database")) { - return false; + if (k.getValue() != null && k.getValue().equals("OpenOrgs Database")) { + return true; } } } - return true; + return false; } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkRemoveDiffRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkRemoveDiffRels.java deleted file mode 100644 index 4c0bfadf0..000000000 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkRemoveDiffRels.java +++ /dev/null @@ -1,281 +0,0 @@ - -package eu.dnetlib.dhp.oa.dedup; - -import static eu.dnetlib.dhp.oa.dedup.SparkCreateMergeRels.DNET_PROVENANCE_ACTIONS; -import static eu.dnetlib.dhp.oa.dedup.SparkCreateMergeRels.PROVENANCE_ACTION_CLASS; -import static eu.dnetlib.dhp.oa.dedup.SparkCreateMergeRels.hash; - -import java.io.IOException; -import java.util.*; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; - -import org.apache.commons.io.IOUtils; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.MapFunction; -import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.graphx.Edge; -import org.apache.spark.rdd.RDD; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.SparkSession; -import org.dom4j.DocumentException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.google.common.collect.Iterables; - -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.oa.dedup.graph.ConnectedComponent; -import eu.dnetlib.dhp.oa.dedup.graph.GraphProcessor; -import eu.dnetlib.dhp.oa.dedup.model.Block; -import eu.dnetlib.dhp.schema.oaf.DataInfo; -import eu.dnetlib.dhp.schema.oaf.Qualifier; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.utils.ISLookupClientFactory; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -import eu.dnetlib.pace.config.DedupConfig; -import eu.dnetlib.pace.model.MapDocument; -import eu.dnetlib.pace.util.MapDocumentUtil; -import scala.Tuple2; - -public class SparkRemoveDiffRels extends AbstractSparkAction { - - private static final Logger log = LoggerFactory.getLogger(SparkRemoveDiffRels.class); - - public SparkRemoveDiffRels(ArgumentApplicationParser parser, SparkSession spark) { - super(parser, spark); - } - - public static void main(String[] args) throws Exception { - ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkCreateSimRels.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json"))); - parser.parseArgument(args); - - SparkConf conf = new SparkConf(); - new SparkCreateSimRels(parser, getSparkSession(conf)) - .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); - } - - @Override - public void run(ISLookUpService isLookUpService) - throws DocumentException, IOException, ISLookUpException { - - // read oozie parameters - final String graphBasePath = parser.get("graphBasePath"); - final String isLookUpUrl = parser.get("isLookUpUrl"); - final String actionSetId = parser.get("actionSetId"); - final String workingPath = parser.get("workingPath"); - final int numPartitions = Optional - .ofNullable(parser.get("numPartitions")) - .map(Integer::valueOf) - .orElse(NUM_PARTITIONS); - - log.info("numPartitions: '{}'", numPartitions); - log.info("graphBasePath: '{}'", graphBasePath); - log.info("isLookUpUrl: '{}'", isLookUpUrl); - log.info("actionSetId: '{}'", actionSetId); - log.info("workingPath: '{}'", workingPath); - - // for each dedup configuration - for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) { - - final String entity = dedupConf.getWf().getEntityType(); - final String subEntity = dedupConf.getWf().getSubEntityValue(); - log.info("Removing diffrels for: '{}'", subEntity); - - final String mergeRelsPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, subEntity); - - final String relationPath = DedupUtility.createEntityPath(graphBasePath, subEntity); - - final String openorgsMergeRelsPath = DedupUtility - .createOpenorgsMergeRelsPath(workingPath, actionSetId, subEntity); - - final int maxIterations = dedupConf.getWf().getMaxIterations(); - log.info("Max iterations {}", maxIterations); - - JavaRDD mergeRelsRDD = spark - .read() - .load(mergeRelsPath) - .as(Encoders.bean(Relation.class)) - .where("relClass == 'merges'") - .toJavaRDD(); - - System.out.println("mergeRelsRDD = " + mergeRelsRDD.count()); - -// JavaRDD, String>> diffRelsRDD = spark -// .read() -// .textFile(relationPath) -// .map(patchRelFn(), Encoders.bean(Relation.class)) -// .toJavaRDD() -// .filter(r -> filterRels(r, entity)) -// .map(rel -> { -// if (rel.getSource().compareTo(rel.getTarget()) < 0) -// return new Tuple2<>(new Tuple2<>(rel.getSource(), rel.getTarget()), "diffRel"); -// else -// return new Tuple2<>(new Tuple2<>(rel.getTarget(), rel.getSource()), "diffRel"); -// }); - // THIS IS FOR TESTING PURPOSE - JavaRDD, String>> diffRelsRDD = spark - .read() - .load(mergeRelsPath) - .as(Encoders.bean(Relation.class)) - .toJavaRDD() - .map(rel -> { - if (rel.getSource().compareTo(rel.getTarget()) < 0) - return new Tuple2<>(new Tuple2<>(rel.getSource(), rel.getTarget()), "diffRel"); - else - return new Tuple2<>(new Tuple2<>(rel.getTarget(), rel.getSource()), "diffRel"); - }) - .distinct(); - - System.out.println("diffRelsRDD = " + diffRelsRDD.count()); - -// JavaRDD, String>> flatMergeRels = mergeRelsRDD -// .mapToPair(rel -> new Tuple2<>(rel.getSource(), rel.getTarget())) -// .groupByKey() -// .flatMap(g -> { -// List, String>> rels = new ArrayList<>(); -// -// List ids = StreamSupport -// .stream(g._2().spliterator(), false) -// .collect(Collectors.toList()); -// -// for (int i = 0; i < ids.size(); i++) { -// for (int j = i + 1; j < ids.size(); j++) { -// if (ids.get(i).compareTo(ids.get(j)) < 0) -// rels.add(new Tuple2<>(new Tuple2<>(ids.get(i), ids.get(j)), g._1())); -// else -// rels.add(new Tuple2<>(new Tuple2<>(ids.get(j), ids.get(i)), g._1())); -// } -// } -// return rels.iterator(); -// -// }); - JavaRDD, String>> mergeRels = mergeRelsRDD - .map(rel -> { - if (rel.getSource().compareTo(rel.getTarget()) < 0) - return new Tuple2<>(new Tuple2<>(rel.getSource(), rel.getTarget()), "mergeRel"); - else - return new Tuple2<>(new Tuple2<>(rel.getTarget(), rel.getSource()), "mergeRel"); - }); - System.out.println("mergeRelsProcessed = " + mergeRels.count()); - -// JavaRDD purgedMergeRels = flatMergeRels -// .union(diffRelsRDD) -// .mapToPair(rel -> new Tuple2<>(rel._1(), Arrays.asList(rel._2()))) -// .reduceByKey((a, b) -> { -// List list = new ArrayList(); -// list.addAll(a); -// list.addAll(b); -// return list; -// }) -// .filter(rel -> rel._2().size() == 1) -// .mapToPair(rel -> new Tuple2<>(rel._2().get(0), rel._1())) -// .flatMap(rel -> { -// List> rels = new ArrayList<>(); -// String source = rel._1(); -// rels.add(new Tuple2<>(source, rel._2()._1())); -// rels.add(new Tuple2<>(source, rel._2()._2())); -// return rels.iterator(); -// }) -// .distinct() -// .flatMap(rel -> tupleToMergeRel(rel, dedupConf)); - JavaRDD purgedMergeRels = mergeRels - .union(diffRelsRDD) - .mapToPair(t -> new Tuple2<>(t._1()._1() + "|||" + t._1()._2(), t._2())) - .groupByKey() - .filter(g -> Iterables.size(g._2()) == 1) - .flatMap( - t -> tupleToMergeRel( - new Tuple2<>(t._1().split("\\|\\|\\|")[0], t._1().split("\\|\\|\\|")[1]), - dedupConf)); - - System.out.println("purgedMergeRels = " + purgedMergeRels.count()); - - spark - .createDataset(purgedMergeRels.rdd(), Encoders.bean(Relation.class)) - .write() - .mode(SaveMode.Overwrite) - .json(openorgsMergeRelsPath); - } - } - - private static MapFunction patchRelFn() { - return value -> { - final Relation rel = OBJECT_MAPPER.readValue(value, Relation.class); - if (rel.getDataInfo() == null) { - rel.setDataInfo(new DataInfo()); - } - return rel; - }; - } - - private boolean filterRels(Relation rel, String entityType) { - - switch (entityType) { - case "result": - if (rel.getRelClass().equals("isDifferentFrom") && rel.getRelType().equals("resultResult") - && rel.getSubRelType().equals("dedup")) - return true; - break; - case "organization": - if (rel.getRelClass().equals("isDifferentFrom") && rel.getRelType().equals("organizationOrganization") - && rel.getSubRelType().equals("dedup")) - return true; - break; - default: - return false; - } - return false; - } - - public Iterator tupleToMergeRel(Tuple2 rel, DedupConfig dedupConf) { - - List rels = new ArrayList<>(); - - rels.add(rel(rel._1(), rel._2(), "merges", dedupConf)); - rels.add(rel(rel._2(), rel._1(), "isMergedIn", dedupConf)); - - return rels.iterator(); - } - - private Relation rel(String source, String target, String relClass, DedupConfig dedupConf) { - - String entityType = dedupConf.getWf().getEntityType(); - - Relation r = new Relation(); - r.setSource(source); - r.setTarget(target); - r.setRelClass(relClass); - r.setRelType(entityType + entityType.substring(0, 1).toUpperCase() + entityType.substring(1)); - r.setSubRelType("dedup"); - - DataInfo info = new DataInfo(); - info.setDeletedbyinference(false); - info.setInferred(true); - info.setInvisible(false); - info.setInferenceprovenance(dedupConf.getWf().getConfigurationId()); - Qualifier provenanceAction = new Qualifier(); - provenanceAction.setClassid(PROVENANCE_ACTION_CLASS); - provenanceAction.setClassname(PROVENANCE_ACTION_CLASS); - provenanceAction.setSchemeid(DNET_PROVENANCE_ACTIONS); - provenanceAction.setSchemename(DNET_PROVENANCE_ACTIONS); - info.setProvenanceaction(provenanceAction); - - // TODO calculate the trust value based on the similarity score of the elements in the CC - // info.setTrust(); - - r.setDataInfo(info); - return r; - } -} diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml index 339e99084..dc63d0a79 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/openorgs/oozie_app/workflow.xml @@ -87,8 +87,8 @@ - - + + @@ -113,7 +113,7 @@ --graphBasePath${graphBasePath} --isLookUpUrl${isLookUpUrl} - --actionSetId${actionSetId} + --actionSetId${actionSetIdOpenorgs} --workingPath${workingPath} --numPartitions8000 @@ -142,7 +142,7 @@ --graphBasePath${graphBasePath} --isLookUpUrl${isLookUpUrl} --workingPath${workingPath} - --actionSetId${actionSetId} + --actionSetId${actionSetIdOpenorgs} --numPartitions8000 @@ -169,7 +169,7 @@ --graphBasePath${graphBasePath} --workingPath${workingPath} --isLookUpUrl${isLookUpUrl} - --actionSetId${actionSetId} + --actionSetId${actionSetIdOpenorgs} --cutConnectedComponent${cutConnectedComponent} @@ -196,7 +196,7 @@ --graphBasePath${graphBasePath} --workingPath${workingPath} --isLookUpUrl${isLookUpUrl} - --actionSetId${actionSetId} + --actionSetId${actionSetIdOpenorgs} --dbUrl${dbUrl} --dbTable${dbTable} --dbUser${dbUser} @@ -227,7 +227,7 @@ --graphBasePath${graphBasePath} --workingPath${workingPath} --isLookUpUrl${isLookUpUrl} - --actionSetId${actionSetId} + --actionSetId${actionSetIdOpenorgs} --apiUrl${apiUrl} --dbUrl${dbUrl} --dbTable${dbTable} diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml index 998f3ac21..c28a2a921 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml @@ -92,11 +92,23 @@ + + + + + + + + + + + + yarn @@ -182,7 +194,7 @@ yarn cluster - Copy Merge Relations + Copy Openorgs Merge Relations eu.dnetlib.dhp.oa.dedup.SparkCopyOpenorgsMergeRels dhp-dedup-openaire-${projectVersion}.jar @@ -201,7 +213,7 @@ --actionSetId${actionSetIdOpenorgs} --numPartitions8000 - + @@ -210,7 +222,7 @@ yarn cluster - Copy Entities + Copy Openorgs Entities eu.dnetlib.dhp.oa.dedup.SparkCopyOpenorgs dhp-dedup-openaire-${projectVersion}.jar @@ -225,7 +237,6 @@ --graphBasePath${graphBasePath} --workingPath${workingPath} - --isLookUpUrl${isLookUpUrl} --actionSetId${actionSetIdOpenorgs} @@ -262,7 +273,7 @@ yarn cluster - Update Entity + Copy Non-Openorgs Relations eu.dnetlib.dhp.oa.dedup.SparkCopyRelationsNoOpenorgs dhp-dedup-openaire-${projectVersion}.jar diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsTest.java index f8627d023..6ad2145a9 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsTest.java @@ -19,9 +19,11 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.FilterFunction; +import org.apache.spark.api.java.function.ForeachFunction; 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.Row; import org.apache.spark.sql.SparkSession; import org.junit.jupiter.api.*; import org.junit.jupiter.api.extension.ExtendWith; @@ -61,7 +63,7 @@ public class SparkOpenorgsTest implements Serializable { public static void cleanUp() throws IOException, URISyntaxException { testGraphBasePath = Paths - .get(SparkOpenorgsTest.class.getResource("/eu/dnetlib/dhp/dedup/entities").toURI()) + .get(SparkOpenorgsTest.class.getResource("/eu/dnetlib/dhp/dedup/openorgs").toURI()) .toFile() .getAbsolutePath(); testOutputBasePath = createTempDirectory(SparkOpenorgsTest.class.getSimpleName() + "-") @@ -71,9 +73,8 @@ public class SparkOpenorgsTest implements Serializable { .toAbsolutePath() .toString(); -// FileUtils.deleteDirectory(new File(testOutputBasePath)); + FileUtils.deleteDirectory(new File(testOutputBasePath)); FileUtils.deleteDirectory(new File(testDedupGraphBasePath)); - FileUtils.deleteDirectory(new File("/tmp/test-orchestrator/organization_openorgs_mergerels")); final SparkConf conf = new SparkConf(); conf.set("spark.sql.shuffle.partitions", "200"); @@ -133,7 +134,7 @@ public class SparkOpenorgsTest implements Serializable { .textFile(testOutputBasePath + "/" + testActionSetId + "/organization_deduprecord") .count(); - assertEquals(0, orgs_deduprecord); + assertEquals(100, orgs_deduprecord); } @Test @@ -161,7 +162,7 @@ public class SparkOpenorgsTest implements Serializable { .load(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel") .count(); - assertEquals(0, orgs_mergerel); + assertEquals(6, orgs_mergerel); } @@ -190,67 +191,7 @@ public class SparkOpenorgsTest implements Serializable { .textFile(testOutputBasePath + "/" + testActionSetId + "/organization_simrel") .count(); - System.out.println("orgs_simrel = " + orgs_simrel); - } - - @Test - public void createSimRelsTest() throws Exception { - - ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkCreateSimRels.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json"))); - parser - .parseArgument( - new String[] { - "-i", testGraphBasePath, - "-asi", testActionSetId, - "-la", "lookupurl", - "-w", "/tmp", - "-np", "50" - }); - - new SparkCreateSimRels(parser, spark).run(isLookUpService); - - long orgs_simrel = spark - .read() - .textFile("/tmp/" + testActionSetId + "/organization_simrel") - .count(); - - assertEquals(3082, orgs_simrel); - } - - @Test - public void createMergeRelsTest() throws Exception { - - ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkCreateMergeRels.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json"))); - parser - .parseArgument( - new String[] { - "-i", - testGraphBasePath, - "-asi", - testActionSetId, - "-la", - "lookupurl", - "-w", - "/tmp" - }); - - new SparkCreateMergeRels(parser, spark).run(isLookUpService); - - long orgs_mergerel = spark - .read() - .load("/tmp/" + testActionSetId + "/organization_mergerel") - .count(); - assertEquals(1272, orgs_mergerel); + assertEquals(96, orgs_simrel); } @Test @@ -273,9 +214,7 @@ public class SparkOpenorgsTest implements Serializable { long relations = jsc.textFile(testDedupGraphBasePath + "/relation").count(); -// Dataset relsRDD = spark.read().textFile(testDedupGraphBasePath + "/relation").map(patchRelFn(), Encoders.bean(Relation.class)); - - assertEquals(500, relations); + assertEquals(400, relations); } @AfterAll diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/openorgs/organization/organization.gz b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/openorgs/organization/organization.gz new file mode 100644 index 0000000000000000000000000000000000000000..45b0edeb252685bace9873a5810da3a49169093e GIT binary patch literal 71922 zcmY&gWmuF^w;ehqq)}iH=?KWJYFa`L{GcwHuA)(>Avt5;`2EC>G47Q z>?e=I&&nrLepjl;OMa-f+X^W~P1n&pk$uFyyY=5!U%VE|$JVr+>{LDu>yz{%c$O&i z_pZ`5WkB;PX}6v)*YGi`;b7<5v_b2khjq5h>(GqIFJhq5z<_l%_aGs1S2*{u@;b0< zV9o9^W>9fW!1pG3`1eMJ&?8fKN=Dx8od>sY|L38bj5EuWsbAqMk%xe1xwhn{5Nl97PtWE$EwyMXt~K}#yvQDYG&V=^8=MZc z+(yEQjv7kd=;D94$L>Ki^n>4dY}$P^X7?Z{<}bfH4Qo*kL0SpK{aAUSGFecr;8%NE zFYI%q8}D3M*rA2mSN;6icy)Arv*+>7U|;k7bWPEYb6w-!!`abFwtmzY(w6v(CBw$p znC$i@wX5=T$)>@pL953T?q>DU$M(|rRqV%8zfqK>ZVr9l&I#iL-vaFcvu%^oV!vZ_ zJ7EDU+I?1`p_-rCXO=J3B3*sA+V?MFeIHZ}^}1-%jOtgz&Uo|WH0QV7SXVvCuTf+l zmR{fgDkeIn*e(~o|7hRIKq0G8H7bj|JB9@B65zx@9lg03)?IiEwyP-@z z%}z_4WlPTzBkRS94!oqAPH{rkTbxb?KFnOdHVVHS9zM+9p2#ZpGt~E4DoZ}cxGgyI z_4c4mqnML)&RGrPu^S6EQNCR@wPrOxUTRoXZ`6zbdG^&+)9++6`n#Kai16dV=KUpQ z+L0Yfn_1Bq%Z^Oi=YWD|{#PBshW-UDweb=6D`q<5116r=yB^mxYEj=0&yXK79(P+P z2XVGi#lv{~hEg;9M*Cx(uP)C^q*WVNEv?1$@T0j0wEj5#lIZ;S!dV_dT=$gVlfyL& z!?NRN3CVBl337A<$TV>~59PVOK`KcqqzXB-Dl+tX^p^PZyD@w^j>vg|Dp?wHR!Dh# zRS}}!Gu05J$z_r9da3rdXcZ~^-sTMYHpaHcI%j{1TzxcnJ8QdH*8H+?wf~xUHh=Ne z_WdAnXW=R`F;Y}7=MF9#I&r-IV0i)iz`R-KwM$LZX_-6OeU+v8{(8N;{ew{G^c^l? z{CCM7E@g8QX~|!N)Sv1{4T_l!0s{SZskZs<@gHt454-!ECXf8zKpuMT{#}N%biSk4QUo%ZSopDK>7`S4gBoGk8+La zpycm;u@>k+bj@Zg{(fBT^LOHpZtPM2MZbnfp7i`EX@`TNAv>q?&Vc*{VI)YAIeWH+{SflpFT zjq^HV^KQQHHZjdP#+^nZMkKCOGBYH-m~TD2IcYc`JgU!|OC{gpHt#;?w`r{R-+1ES z-+FRk@}{Eiq!wLATlI})Z;blv=^5ij-WY*tMEi-6N8!YUPTL7qsd0sX)*>V(Y@_Yu z1l7x{zy)Pr(8rlJX0B6Dcyi!nuI2hg;$zy!q5Rk;_~U>p*g;njj=^g^wk8qwUL@fm zdx(s6{c5mgW4Uf|9PO8j8U`O;RDEZPjiPkf$wD0lC;kvGU~u@sFPMN&;|kKy1H zvl@N;@uni9elZL;qPWYDKj&TWJ?YBcrZry~;ZYkL7TVRauRMwCbtb9`siCrdyfXSm zEq7pifUYu$hRhYz8>f2KfbnWMOC3`PmEyU`$&u)l5&65v4~!RV7hzmFi~{JGnBOeN zso0>zYz?Uy`F0f+qtOja3bWVV6XV>Uiptyx_1$Y6Bb{yA^S?H=w?%fYtCt)!o(xeu zXn2bxY1{FlVD_S4*W$!{=G{qVJdt_rCq+mYH76HxyMP+Uo?QyJbn&|;;}=}o)po;i zZd3bW1_BuY&iwti{r#tUGI7~?Wr2k0Gp+ijjFe5zTygS@s4R#?utTFBfI|mjw2_IN z;OL>+lD`uc^tebYm)TFZz3h3p1#gV;V*nJP--N~~up;!s{plD}sY9&gvFB;h@(dHb z;#0G5^0kk4Gb-e60}{O15TX0BsbV2A5$=qZ??}_;{w4@}KPeKU_-Df64RE*O!*h)( zIW+Tf@}j7>1m0j=z};rT3grcFRYZVKs}LxZdk7ajRD1fRwH+!@xzaZArA(2eH-dri zq5+kC*T)~eVr`VsD~zrzP5sO+jacjJ({5VCn4JRXhEWeMqWdl1UH7;A1w1z8borOH zUy_Y=yM1>&S)+Z62^Yl5r#=LJY`)Dp$DolVsgYWAaZnAdM?3Dga`Z%FAYHzK5bSrm zY$z~r$a8+Ea&G^Lqi@Qbm*>Eg?tA#|_!+r)P}8SG%diJ(H!uA2Al;9S8zdAZ88j(im?`VimfLs-Su05?loDep@~( zMi)RN7c2cWg)ZQ1L4bfA5C*D@JE}mO)U>Xy1OMNx>q!2O^WGEp!_mXCO2gQHX@?4> zGAV`?U3oLAwq++me(@wt{`}Is-Qw0MlN{9GIqDz4VG6ZL8Zmi-+#4S-T}^XHD<2qe zV+!^2;%X`s4{KTv<>P>g+&XO3%K1mJwe?thA7@dr87&Vf%BF3uVlfWDK}4+%gqW(Z zqv9aKQq~_T0^(9kR!2s92C6Y%O-gSX>lkm{9PV$oH@|#+%_rxYYdMd9sW6dg(*A)uU$kJ0BL- zNp27P(We>~X7pEdoP=CzDx7chf0~M39|Scut7wr;QekKaXK;5@OckhCEGtPNYuO8L zV)1LURa(e)F*0D%Q<=6uV`c1V%;}e?fNK5A;R;v|GoccK95xcIfaP##`krW8W_}!d zE7XV$gwkKM)yS2%rOjMLflUf8jSyR4*5cL$}50U zYD1lx%cyYa+}MV!7bGU>)DwA&@E~{^@P0ov$p3L9ME$BGO)A|Pm7jnpR+=awxAsnq z?OzQv5(5|Q8;(2(Xh*P-7zpUISqFakzh~EJ-lN<_3xe8Tpb4#kMxq`dJCND}L3X^? z(6pQ-jELRdoP+pzN(2%`_7rO^cx+?{-<1}MScnx;!e7KDZ9A~!KF1q>r8!1LR8%sF zZ6X=sRd{5$8+o4Ex!|#H_|R2TbZ$sT9VcUNvoz`U!=2a4gW=X24wJr1@#bN&E$@*s5zf?`*ScRsu`k1y1mH7^r|7D zQJ+kK0z z*j)voq5mPI8>Q7v*esmCZAz*vZ!AF93Slkc%c!RgA1xz2T`SRYBn%F zqvcMqGrDX!+77?Pd7e(67^glJwyctl1FbAW4hwy?8gC9& zG1n2fdo=98M-Sk@jVO!QP>*H7BcbSuAwL1H(^qLrBXWw<$ z+9P6j(3%~OfnzB;b6)Vn`^{g;?2_F_SJn#}wbPCI%V~3pAzq8Qb4oP9Z7d6dhofZk z6FXaSIRkT;q5Q2`a2*TLAD3-AN>ejO;;UljpXON*6#bEH52C>e#psYMHL6Wg!vs&D z=dkG!gPvn&SwRuJ$&kP@nOQeHFOWkn>XcPCvvIfOI_qqsdRM+WR!*MU5>5Gf+JIko zMpbN=eR9{7(JQLaN7&E2V)M25%v!J$74iG>!~6K9*LK{B3rK&T*h@gpE=*V|@XqwY z)zl|E1LL1*i(te0pZ_9A_)L`HXCfyML4N~~@Uqb^C{=eewPSIkoWp7XQ{5Pk;hP4`5C9k zg}~SdM+6o60ST@@*BCr418fEngPfw?TKPWfEdPHUQ49{e zt-O?hHOCpka$6}^hsq>jf4I~4P?_*e@BUViIu0BUb0H>LcQ(Vr;#L+r?7WOJ+=}is zr7#o`3t<*Aw?G`)c_qTsoQUf}8&8b55|wnvJZcnFxL~E?{c3nrGM!Fvh%C65*f{X0 z8@Lymx{fH+VqH2x=2xP@u=gX^83mX!)d5lMEv26Honb6><3O zS_nM77>FUGza!Xpl9^FC_>*o5Uy4*Lxx8aM>}D_NN*oJ&L`HtToC!gIMTQ8$Q-!o# zOtLYz@nxGvQFb1BWsCtT3Nj)>HVh;n3jmM+*X9pE0wyYQKh+*LEv+G5OXAQ$@R6g= z&`w`peI4szY;XP;UnBNnqUku5J4GW5?Ekzd*CFDAEM*2n(`BPe5~O1$JUg-!ttK%M zAF>XJ_8oW1BA9f(P+9TK*p}j@>RSHl>%+R)STxort0}o_7)853FUV;AJF<2z^r~Lk<@R5d{@vHr`gy%JuD61xihakvvU= zi|tqx116esW0$RXzlWn5G5A5^kvB$%E%RK8T9$vbGRx{dq}MiLo?1oTyLx@MZ)tF4 z=*h*`s-L17XYp(POTsT)M#MHt`5B#fACy^F(o$sujfgh~OW5+Ic6{@>(#!I)V)9UtxeEn zX09dFgS9t%=GxZ{sT8}b!eeq;c>($YSDk{weu%T1-@Rf`zwgxs->(;W{r0(%e(?jE z={vO}>Eq(XM0V?ng-E(u3xVJ7HeFLdWXP#XR%=W7`yEQ~i6g%z#EmH~0Y1QA1QtcO z01!nHD0CPSg8~m10HQcG>wr@2JLEX%E!nBxP%Oy&Tj?I5md9U1IwN zI)~f?O}&GjCsFV>Qb zr*@=PmHLm=uGO;wBKvuvndP&)>+Q6u`#zO4R{ekD-{=27{`t4xlo(h)yeo+dd5<5x z+lS>{y5Sx5#N#KDA+E6eYKXB?o3DC>!Hwsn?FXD^waYkRNP70d`8fRgY^3zLM3f0M z8lw_Y7S!M9f=a$o*^AafKU6nq*tdxn6YJry@c6H<1lqR&GQ!8_hq^GD>zpvMi8bJPcv$3B)ViUL~)Wy6J%8`jTF7!JNot}U?>)=l3b^LDKV zwCIvd%!scDh2oNK(deT{H9_VdbQ8MD(vJb=1VeD3@TTeY@Qm zguQ$F`nYFG`f}hnjd)S7L~~<)EL(x))2fJmCagsH*NZHGtXC`ed1dO*tXJ1r={`0N z%P}Ivelo7jr&(&0rESJ720Kf0X8rQSVo0y(tZd)d<+7PX)Ng(`J6o%+uUBHb!Te;P zRrIP8&&5mQrc$>w$@dl95VnJ~|JJ2RU`xtYkaK>7{0|F;zEX`v4)yP6&qAe-1l-AC zjr!;gG-?6tbPAD^Wsf6oFsSnRovNq$c@k~W?qWR%&NPlBxBeH78S8q;&|YNT`Gxv} z$88`vXboOI#?(gn3#{!u{Zo)CX0YYQy-04-CCQD*Ip*%F1Lbndv zwW>aa(slt6$whHrgBc37;;EmM#l*9|8b{d)f?=xhEr6-c18y*(9);NBv@j4TXB&o| z)>w--D#LHJW$L5IlFOAf9ZtjoORriSJ&<00vf}P_vRkQ%s);lXLS+fniiVm3(&l+_ z`vX`_3lst3Nvk+N=l`;kS0piG^|-sZ;Ja==6axJysZHXOcZNBn_1mR|J zK@j}Ef}#UKhEO%6DyG=_2WQ+K+Hfh*)S)U*Ltj$@p|2I2Rm4 z7$vJtXHud2&elJVXNt5q1;$L3*j9=)nY$n9EfuHi&=`=%yG+$hmhHxAl0FP5S~YdftI|QzqAbCw4kR#BVFyq4V7@_r7vd4@b8~mxSI)aU z*0N~xv~;v_j7)I0pZ~BMj@mr59d5-RhENgO0p{^&x^NjuFx`(pDUjoC=!(gsjLe|R zt*)#HT%nh$`$&Nohr^`(uSEGWdDA6QM2Upl7TUKA(XIuPWR(PLUKH0oZ0+Wpr&Cow zo01DZz(GNga1+iC;+LF~=}dD{(`-i}`-C}SFG_N1YX3YNV?+`*ZfAR05Cr`db4K|h zAVHL!A~nrFyT|W+d3L^#aC8%S(CzXaU$)ItIwV$}BT330`;Z$FJ|1(cfLdp0vdV#&)zej8beY?`bSF5>7Ctcpe-3sU?p)jsT2|I)b_bGV0OlmEkYbR02h! z{hgC7!9RW_DKZvYS_sLEPaSfI$@rnlWzxSHWxlT)wF*o@bh?<}AQy=K3#Aa>MnPRb ziFWU_P-0=QE&2ANyRuW{sN71YZm6Iuc=sSir!&0ZN!$+Sg+C3rR$^o`#3DcdPk^rY zhTWeiHQkaoS8r831xCJ`4b3I&8}`1@NW>es`%1tzo*~=-Wgs#W1_&IMdm5CU>0Kq;Zyga8igUlp5ACp_ciYhWPIuy4 z*>3Uot{e)d5+>^By4;tlGhinE)$R+RKv8j3)Y^UW!T?W{?)PhA^J#0lt}InQpM6B~ z^t6=$o*0f8tiD>Jf~C%@hb`Wcmb7-)luMoPc-?^t?n@{N`oZ8#^u@wpgVVP!gPQR_ z_#Dd-Xy-i#8(*tf(b&6T9Sn-XZjIa4bDhqAf0Brt)bgXaZLQ;8^z{7v8C$*(bh_z1 zCCIbY=X|p7+PHeybdatU(pN0Vt>0zYCS5ro?Y0gvccXU6!nUtrj1q}_j zVlle>oA0v^*H6_)^KnGLl#U5K^p(ePdr>~DAb}^doolAUYg%|_fX?XiMZ^Hg`T=Pj zGzsu@2#U!Gya#9!mt2e3CQIkeXi~&?L=Q0Ja>aTF2J@QzhGC7)zO#3Ut4*nsSQq?8 zw|Pj)q4;mCj*M(CybxN%jCm~*;$KDNRC?Gn8r8kRmmJs=`gGom5wk)`cRzk^a4>dN zNOVJXq5l_5x(5x=+{&nB!qnzV{cpb#*N8Q0FU`WK;4c^Vq?~N0nz&O@_mNH$k_#~z z5lpht%Sw9B+{SA!kuMivWMYVy9VmkT{sgso-Ydw<4%ET*naISjx)%6@^0xa|V0>!7 ze^uK;(~vrpqo}&I;c}d-e|2?D^K4#?b2;UMged+9lBSwS@JXefXr7XFR7zfDwI0i} zMYUeYohN+DfyQ0^2fp-fwuAYO?t%9jZmPpekZ$C^cEX!a8vZI)ocYpM$E7E{QTu)R zSB&XZ^de`Qh+n4gKmO1U&^p5uf9(K&_#om3%HG5JjOO;?-IW6RmMy`~s{2Z>=Tut9 zlI}xQ72yfWHEi@;$36k0r#U;>slLN3*ZzbV7ogo8@Omgj{XL$3dD(8y*=4Xj^Au+z zF>%ZdKDv6yo|ghoLb(IYy?p81`*dKt^5NQ=_hSFK)omybAH@5XU2q!Bn_;-MF<&(C zx5bO#ujzkBwVvf*Lxdm{FgOEg25`n7yA^0FaNZE0tvuM+Oa!Ddi{liWt*2U(_ z-|crduUizi{#TyqXG%DQW-Y%FmjJ#lFcfaiduwXLv5^uw{SDl5*X)W7- z0g~N$c@YtLu@5hxDo-a`4ysg!Dv_|!5Tk>JxPuljxvj}9HF`4^fPcX`o&|S16V2GI ze?euc9%_gx!5d*;<%EiCVzX$95w*mLyc1>gNfzF?C=j`d|ew7oI zg@1)`*8_rcgmM+DKR7g2oJ1&G8|k_oVP5P(I??BhMW--2(>mo|p*y+Tz(eK!Ue~sY zmW#?dYT^0MJVExnz8>8U5wH8Q!w>CeTc=d-?Fy+7Dr+dl0@EsMeECue@jn_c<`B62 z!jG_2=*^VFnhDzb7n>PaY%ZW)AU2gKGaxow!UFsvGiy4;NZ>0lxC#51Ws6gj`v=dA zJWd3cE+hHvFUzfJR)b$ZXAOfcZh`~HPPYo&3@*15IU~+Vn$c_l>1z1@NmsUaKCf9P zF{&}LdFZ)f{j>WaN>6RSks!GKL=x#*Qn~(24>5|iy0uN;Noc0}YgcFje^x_r8k%-S zRL2-!^qv~2{H2?ya#!Bx__aKw`+x@}#$iJslz}JE6aHeKfu3LK7DA!;JJCRST`8pJl7SCS{P+_JHtw^i}82Uk&mt-va1F;|y`~mVbST z38M4*{utT$e(p^bjGblZ&&!J_$xsceG@~AhQERgczUNS++Q&=j0+uGgigeaMp>PQ;dT6;&(V*77a$h)HhR?i^cofTXmRaJrF7 z?1b~-8cGy(83n1@Ey(3)Okax5Kyywe)fC!qjC^1bY`X#Vxk5;52}JO`?FJ~2x?V0Q zJ88|$cz-q{4^U3E>WvXL;Z9)`Q$NBc00KE@buJC54?dF8T!$JzP#=8-lKN+_H>q5I zkgQ5-fqB(MGgvFN!8zXR%=j}wV_k2?OEvpXC<-tVUVF#wd?W^g31wF|tamQQMjmCB zsz#`v+@r2~1`c%{_?Pfc|7HG5U@?IE=q~{> zUmEcxuo&onswKN{Ix%z;oAK8E58<$X*zI5Yi%=0h7|s-2Pr#@3vTU{d8kkdeATM9t zJP25QpFAkucN58MEh^?S+V&ubq=cq#-&-+B$Nf}3B+o1=?4zk%VAZ-68=AID z-(&{1W-WmzSnnGPb4k{z-p-)(C2lx4@$;&=lNx&nV_Wrm^(ZPDitwBl+HwEQaPLJ2 z$0u*rB+9Say;1CO8o~5-NbHT-P@wq2f(z{c!3Byhn4d*Mp&cN&pM2)X0J2v?yKYYjL04QS#3+_rE)`Uuo}=nJ$oQq7~?D!u+011|D~@jB_EnzjoLBj+fu67HCaj#3%gdNDr+0@MT2f%a1_h zjUv(a9v)45qmm?Ch1^e)S;mgfEH-;>e`Y{2E&NU3k%30^ng5Bjj0)dM8YZ8ownKK{ zQG7zKO8?qdRW|52I&vJKoJ80%0p)}c2nPsSJDZ+;|19n;H?f;eq9trBOXimdSa4x5 z`TjXP7RaOP4Nn7ofugy=sa-p50UkdJacd*7rrEYd8-_yEm5{H&ZJ$p#+SF7mucouh z>XMB7yi1HdwZjXLYh`~(PxuzCqa#R2tl zfb3=G@k^hY-4|~cBYxKERsD2T0d_H<>YkS}4s5mzYDN}o4Vk3~W9xmRCPHI;iR+H6 zF6`2j@dRn*reGa^JK6m3AXD5CIK|AZhqw)NE7VbGn zu2mG8$qjpjj-!%P1Z$?4I&=*BMsb#u#5FptpN*vcRjALf3Pm+N0xA?4hzaWL-PPXt z_0+e&BV+vR0y+aUwho4K=%luz>YSOVN^yW{7QxK~^L+-BG3I`wdyLtD%Z-IYqn2yu z?to@%U}Y6l1~s0La2%kZj==S>nzx0y{z&KVMHiTn)w!j{b}o$gF81AB;Odt}H zG0m2a9I;@O5YS@TJc)f=dHqJOm&KSy9}V*p=Li)ww$0`rnQDXX#|S&hNz z$~55`bXI0!iX>yMdf;ZG3!Wba)#bY5SbY{#BzW6xis~~Y=BNFsud`!X-pqv?X$;#z z&wu;UB)cVcJ6^TdgPO!#Oes%xNNgi;BSC4U?aas%2IgDO0GMYJ{N@Y5{C3+J&}cU| zV$0d@a~f>2r7$jkq%TB|;AN47jCA}x1xc6_He#N5>an{^tVc)gmUwW;D8?{5IIpuy z!JTz(`MSxMY3qi2)SO_d^HiNr>Izsa8sXNkza7iG%T+Zccg=NP>P=5 zppDy+M}W(LN+JQn1RU;w(!dl$2AJSI4#zjC>8)rA;YZRZLP9OE?bmt5DYsumOEERS zi2gAG6d*L{OB*0xoqS(8g;E}|rieV!^?6$NBU;n0rzXL$~kN%Z-b)`gns5E7+X`V}R zU!jEkSLsrJ;Z_P*bp3@vbeFM@%ppApe*VHBx>jj*WHe`+WoQdRzAmS1O}0{4OLL)7 zY!#H#NaZnRlnz>=s&I~#amVZafGyDZppKGI%cZ*9k5%ALc=*iq)HW}1(#sGoET98ca^S)MtZaP;(mw_T3n2`^ zO1G*_H=g63q2Y}TSOOOh4{p-~#s7_Zmn;IBH@eR=>f#@Wd1 zPxpbK!v^ktfpA%KfBM>C!N}$kCk)}rSD5Ci;VPZo{Kc_6rvWKZ5?4$fz03{JPgcHW zEPIWkduPaWwq7pZg8tKtva|(wjT9oopVwnsx}Mir&9uo|UB>&bErh0k|HNb18<-|O zpU>6PlM6)mS?cuzW=uG69Yx_pgXm3ci>N=xPu=3W%{70(7S~7^8jgI-f;6Z5mpAx) z*>ZFub;00PJp6h7W)OFg`_Bm6;as$Khr%Ieo^n@lM8c%BzK59QaQtb2rHmYq{Qe2M z?n|I~FQOa*VfRSL-H`m<;a=APq;*SpL1rV8wTS|mOO_rBAfZh#8^P2>maQoa%{+Qt zI)%xx2DJ2TUTgT5dP!n<9~b;ib>gj_jLfU3z2cnSki`%JE|U;_9hbyDH^hcL^IAI1 zXM-LC-@z$dtKX=2urvg5Oye&gJ!^Hmn7r$_~(=aX|u6|x$3 z(LYqK21Df&K|3G=3(3_0D%b6Nk1{3DBh?S`WAS%+4NYMf=v*Zu za7eS!R$Xc6YV1V}FTSnJcXQn9B9_Lp_dnex#z#f}XGO(_Sy7P!;ei#E4hJ7tQQgi~ zvE%}(;`&7bGDTo4z<-=#@hNV{^*q`Xcjb}h+mg8mO@?*^GdFxJD!sd#PeJk8&R35;*lDhIj-9m9K zNyf6rH6#y9tG4yuegFC!od^2-pLK`_W*x#pMFD`vUW5l&hg$eNTpwOtdpY^ihRiWT zh7z~uy%dLb)^xO5DVAAbXi>No47L<40`v;AXW5fBE30kSx5UOl)JAoV*m!YC?8L0P z2e^mhD5m%f^T6uo^NWbKAN4ISUAbYeQ{uxPKR&7Uvrs-RBhOoQ%4tha%36!s(p0k- zAKIl~=Dp}G@WbtX=gB4rE2T!BFf$RkJYbzVK<@&jH2%#l)&{?yH*I1{c|FBqj*1ri zBElhzDF8kZz!a<%&%QNQ6}Z}4L5BosNFL>yrdtzR%A}jf>gz<@sf^v~O8~hE*y?AM+A3%oS z31sH5@L!Obe%06w9gpck^|Ae|EjQ`Um2!KipW|SC1N0&YF5sx6&GW3vBF*=UB0IG) zQH|zIETW#^kahuSo1OuCa$dx?N?oQey3xBN+!wT()OM>pwahgM3jx^ zF(OI+>n06i9C9Tb&#~Qi<=`50ksI|g*b7oPH#hYsnWGo;V>d@i(a5idXx~~aS9D=; z0kILYflH|}GcLwJSwCTr?E^jr6N8Qe5i?<=D1tJhg%J%w5V9Bos2N;1q@ZiJR1)SenTdqRo46mT8fbLF`dGK`U1-+$=^+1LbiAL^0}`PNr%W_G(($2u;wbW3B3F`E8bgS zGWos~bu&mdV5=b`zIxmT$S=q&)H~^@CM2GT3w+Ox%NvgJM1aEM{Cj;+*f1OpfephV zh*0qQ%%juE((mZr(vfo_U-tkz);h`wAm+B34g7M-!uwWT&c^c7s$-p>(uv*NzJH}# zM#$abv3PR>kXg3=;$1Ogdif@Ii{AL5M$Og}X=PuP_umqXt)Y^2vbZ_&22UfABn4%^ zrc_}h8hoTEPrljtqdyUwZ__maZ@~3?1&$T43IqBExu{d=yzQSxQ)haW$(p#^|3G=( z9ldb>M=Bh_NQFjJ15jdDB#&UE!lZs9*}>Vmt<`Fe@u`y?_MR8(E8|oQf4o+D0C^#_ z{;FWYloJ+7jdx)G7`YnO*=yLZf77-@^SUz#ED1a$?+ORT+`J4EoJ;wF-(|0DF~IwW zo>O)_P8maX55!mWqW}w^-}j@z{k-;jp&AA4w`l1)QZQRh<$(OYPh!4A+9cqXpN5hMl`dZVT>Zxg1tUAZZk{F0aQFi?!#-d=vm`V6C z7@CO1qX2@bsabK;I>L@V{Do=PyywfNg=&`@)!Pu! zycJoa!p3IgXC*m>{&%577R7jvj&EEPh7d^F6b@tM<4mcmmQ^$Y5)>r0M`L)&vQYsX z_{VV{!8mSU6apOgiq8>je}LEH0^?Vrak%*|iodWBEh=r(4(Q zPmh}V>Ai9E&S%S+{;T`l@i!KbYV9ww~9sJ-!#4kVjkWhnkbz+b|Y-z0(cN%5ma0&vDtB*ATYOaI_{%R z^>d{V4!q&W6Y8TbFu_3WO}JT%sOfW?_Yd-1&BT8YAQK1Eg3{g`@Ax}79DoYC)1d1X zIH9N-j?Zc@6vocXmQ^HJio#9xA(Z3jP(XzIms8Ddw}))JEIm|vZwKvJ@2eeLYt+{O zmUREHGObHq{a`YdO!RQCM0fRFB5G!&HcZqeONQClB;!JSE| zk(`4IIzB91Hm=`r@Wivx7~T48?}<<6BW10MbAB~DQp+5|YiJlGQo@Hq#Q(wgP{17c zQ?|m4HkfdsfZH?|w+%zO2iDM2x_mslpSyW}c1YQA=rHgC@>vN=rvx~{a=(Y^yGF6| z&DZeOaNp<=WnPfrnZTj;5(NHn9+eKWS@m3?DQn=3NN{P0@{K564dr8l07v#fEuKGS z)*EHqAkLBaKtxT4GJWc|U)13JA^&mx_=Cd$Fb6vf1kNAO7*-18D$d zbu7U0;V^l%c2~#H4*AS8XpCa_j7pz~b>T)K%oOEKZ`{`?g1{(hVtUOtt557$;@VEb zZuHF8=R2v>`MJhdAT+Z;&`FbjPessH-|KF`(5H8!NGbMZ@W|-ZH&pq5kn$T0QqsWV zfy>N>`wc+K=x9H051{9KdTZ_q=1*E6LIB7HbI`xea_z*fW>k&QKi+ZmCQ=-o0fu$E zTE%Fg|23=&-8(A$5b7=%WfJ>wz(iQ^F0FM_ci(;gF@myWXsh!k^s^YL6-H522==-1 zBki`$C4+{W!3`0`qsFU*;8B;f??&jI1DGz=Z;95X1uDTtO5f(!L<(W=D-mj?nx*Q1 z^P52Wc>`8;VMFQw&oDk|k)~UA#d*ml7CKVP&gb+^s8umK!^CPO;RT_bO00$dc0800 z0WuxO*a3dqOamo5K*e=E++v{$`84NgrZ%tKLnlgBE zY;Nun?+WFkGZMdbK&$Hu?UjD=oq@V3UqY(Ok}iiai4MVJq$AYmU$(TtvIPw9;DUxd zX#?4E>-KBYm3X8_pGwD%LQ}wN;OsZjJ z&3YkDXr

$p)H|Z>W9Cnu4MbUWh=|rt|f1y=2 z+EM-DgFIAlMs{c-eoZxyHt@U_YT;y4FkS%917bLr~?xa zT3ayCL90?Gi4K|=4Oooh^WBR>QV8slZvY-7-SxYf2x{^&AXW|}=zVZ+Y5Sm#la zn3CGu{_f$q-ziS*oZNCw-^BFw@thp=0Zf=Ue!`o#{J2Y6Wlda^q-I4N8X8K)?#>KK zv~^cUq3+P5sd6%C?Q@whT=#vx;2%pzMXxLs7rJD<)G0k@6XT>?uNf|3K;f=h-O(6@rr>dBFj!&swQD^0x8!md<|~2fx@X> zfApC&rb2=QipJB;yeH#0l;KB>CGu$7Az*m{Sdq5{f!^v#*#v;*D8}{x)t*#AbaYEs zov=`IY3drZnX~6>wcNZk88oo=?}Dh8-Zl!9Q_Og9Kcx$vV=) zhZym!_0KMQZ!nsVwvD26hd91;gyP`*(=DJd-2xR65$G1c@(FZ{M>(>3{kbc^>P`1l z+3FoH|L;QW*%MaH4W@faoAd5lhnG}Qnf5PvU<*vk()M71>B0Gi{^7a4>fQS0{s%Zl z(jz_*6)jQcERr_lm^$sO>@E(0t)HMmMG_bc+X|VI5**~T(+(M?Eu|$c_#Wv_Tn)@H zzGY6IpUrugv@>5*S2ZLXnC)zzc>7#4GVH)&^RvcVL?mp7aR0zd6OP)*Kq?bHrA z83z)l0xV;CCY#B_`+t`CJ2*ILnqIeZ81aJc;&0mO2;;{n^fdpoBTrs!i!0Kb6tmie z7h-z!dtN}^-HS0DD8de}sk_;cc;ru%Pb zdq;u_^go&%b62TdScnhg_Qg!|Ryy95Qq8@n0E?&U+|K5tJJ z=NC+N-p0~gzH5qkPjc9Ek~=BU0k%eAqzc#+1tn#^rq6WYZKUS*ow9&cL!ali*>a=& zNSLhc#596Wr6h%+V;3FV%~_<-c1)`cgT3Cb;*F7>s`QYv0%vT$4|X?q3tz3;N3hRV z^DQR@@2*up3y}pyPj(wwj6kIDt1N`eUb%h1rBx@r?w3<_Y-8wnt7aDD`fakTxKK`DBKqqY7l{k!USW~!I zWSGZ$;X*{DSslGlo8-DS(J8+4x-V{BS)F^uVnWbbGcw1;v_5QOORhL)J26pXdF9D%kPfZV5Hje3K%=wi3J4w*V{UlR{*It zleHAkp-8dcz5-;Mr=#+YUhqrWVZ7Q)lUQV-Smxk1^I?(=vTXgVpSfEGs!jBQF(4j% znjR*}qTHw7l+-f4{FS=JT*|kXvQ*>gHOB2bNtv2BFmqn?*A_C zHB#kdH#)0@7UtzqU#1SrCa9r{KJUI5tffMzZYLuFY>wM|@Kv~%(fl0tQg1+VR!0xD zxT17zpE7IU?daPC9eS^YEUU@Py9YdD1urT18LR!8B!E+bzCL;ak+#25R}~`VCq8Q% z_tyq)uNS%MDa1GDPD@dWR|CstDL%^Fc_XP8dE$XZJum0yF^UoRqfj0$i2E%&#x#<9 zloC<($><*?dXw)pgYLKt`(qBA@JA5$e{lQ=hJXbiVA&aE8SV&l@r;IIl-AqH}^L{mFkS7iG z8v0y52%(q9&dEz><6caLREqI}o=W^T9ox6M1`TF6Us&A1ZVP$+IYM>7nfP-ek(>K( zvxgF%)VT#EnCU9tRg#lE$f~NlENn=?GWx$Z_yx9~m5VY8YlDJczPaWgsZ8G+qkbPN71f7<$7!usX59l00L zs`pQ%+t_*;VdX2GAXQ#F-mPH1^oRXVCjH%7ZwjV{mW|M#4BGL07+#)_7GE8DqTsgj zaQyEFCy)mv?HNLGh*kAGR|)9fs%z&2cfStw$*6PX=9!ALoB}JrZrH=id!-t4&r-*q z<9hw>FX!gj$E~)ozwl!JjBIYFoYUrE*pt&Xv&3}ThrBi9lvqkuOY3G=ac^9ClICUi z=siK?LV{Y*i^5D@+NjZHVM6ATovp<;cBmyLg+*4^FD@l@;I(<@SzW8;11H+zjmo0q zNc`HOXTQ(u94sJh+bn;3N|O*HN!`fN>}X1&y?M3*E(um|A#IHv0PPIdpkx0dd&}8E``G};&Lp2fwkz#q6Y7N%SN-oIoH785g1JJ zl8x(h48~6e`>(?mT?0HK6mCIbUc@d%xuQ z>D?kaZoS?9c(Kr4MqIj3F=biCk^A^9w*0-rUuGkpZEj%eRsRdD#>vwmHttHZe=CuM z$NN*`Ud}@n`wzMc41(}of_$~-?+%o3hkeAr1yFwSyfj?1eFdv1O6t{(r=NA|#cWn8BzH#T^a zW_;W6i2e#Kuf4s~jMLUyDe|)miP^sSb8RmnK^YGFd!cYCM!h=jdPdOk=T0zn`0V0* z<|SHzFHX=}cY%9xd@wU=lKMX2fz-!}RQm2%=HAS>;X%!VnPJOoQfsXPxawxTSqsv* zgBgOxsr5z{xrBUrKdB+7yuiIz66I-T^JIQ{ly265&|X&Hx7GpC?s!zp|J+NHirCEA zO?j<=ayG)hS;r>m!@Yzqx%$+)*cA7fWWns}pc_GhGnavv?t+%_)#m9H@%6F-U$ckH z-*n$oAhyR&<8hk2b^)^;-~xjbf|kGq(rU$jmui;0)kMB}6SP07t$;;5D4_B;GjDRU zG^-j3N+V8_!lVcgw;a@%tF7>?0H0*X!8WB>Rm(zTbFZ(U* zdkCzI#kZWi$rOUNEX(%@V(}?F-D&{5<;tf`w?=`8ukYq2cco*oM$q|}JbbI= zlK+qbfym5KLu7)7yWzvbqM&Av$b960snBRJN|#s{Bh^uCFiM*@+QKSVj>iR-`XWJb z4_XPXrO!K zB0JP0n8*u5kQBOH59CzLw}%bIL-8T~SMwdt_s)un6-egnQg=Jp^9Z^Dzj%Wu@N)CI8pIKw{U6=g z%b({VyXAh5fb(MmIX$91>jJ4eSiq51WPRe>jt-CA`W{{3NjTAWf& zuSo&CO$=ysg0W zvhGqu3oV8kT&=K!ejsKCPCl^I&&TbA-hM3f;>yxKwaqk|V5!_->tm!IMbJYxbI9~t zd72E_)#>+Mw(KMC$O$D7+Yj8|7?Bg_xOU5fQQ(%Fd07dNTgp9O)l04nx;yxcr-O^x zs?;x1&+>k$e8LTlqfdHTpx8DHz6SYaM!TReyCU@q?~9-If5*07zYrF2yIh_i>{{Wg z&};DsBcYJIuZ4iFj>rm><$s1yYBCIydGM*ApNv|1&58(-l@%BSRbjOdpsQCiJp2+0 z%dAHZxZxHL!VYTi)VQcw`$zHMbA#7Z(mr{{nl9G!Xh(AeBODt*`Cvt|B)nuOxuFR$jMLp zfi&l&hMeCw)t4iN;$_2zsNDaJ9;J!*LIf3Bx#RI7m{hTC<;dlVDs>*R)q&ZHEC@xw zRMCjxcF2ag*W`P?@H(3xHG-BYYbL4apAC!=Av89pmTz~qP1I9+^5qIb__uA+?>GG} z&Dsdc3QnJ;4!U%Avx$={2z;#aDv!A*Y3)f|Rvf#e#E7H#(0|-!4sx5utPRL*9G|bI zYv*?0WS5M-&0GFlHOj(N$Q2UG$mV9V*$5483G*GyLGD^s5x~2UExmALO1*1C97>vU zZ{Aw+UsKV0YA8oR6u+~|D!N98e62ncx@Qk0Lqej8mx=$rrDH2$F1RDEim$+PEt1|q zBB>}Se+Gyo)t#sm3-BJOfNVi*Tk$o4&swMu*+<2_k0ikIS}5`!4NQnf+~k?-J988wd=^Sl&dCP zf~INpodeWiR!>4g^+9OLP{@x$mmlS(JH?$i%3*>vvqzi<3u88+9k@Mr8QfYs$|rrv zRrd}tiJsmgp0P&ML!HaO_p_kDvBO(43o_do>)<&XqbjEw96>U*nZkhm@dQPSo|>g} zBv}08I+SOJZ9pgwMGCsKq%*S4n2x;(P3Z}3YcntQbPGT{&j7v1{3lhHJL;d!C-Lj~ z6g=be7xa&IY`kzE@#DR6T0G)7SeW-UnoD?V-pbNg95&gMYhu!x$7-Q`*O!1w(C+$p zFhQ9>{5HX(25>wCb5A}jQW3r-&MPL%>xT7(kNCQn2*kg-@c#9~QdL_jn|-l;G~+0# z=)U3t$A-uKm)*ra2DH&h@(L40ZH8$+uL9!8hbmrJ`tI->1->e}QKR)IeTM4}-so9f z?nI>f;V90{+rA~-KdEmK+*H>c^#5xtt4J@EQ~gAMJf=SO(uI?DtmMHaIrhB+tE-m_ zVncC`B7KFzAF5p0$ui;wyS^hz2DMEk+o+2Yd=L|<-D&}@@q@v+FO!3M>%s1ApB=X zs(Fm05}Us~qH{4}I*!;Ih<)4rb=1Io)52Z7vZkB|Th2cZ)YU6%*tNA5$A2pMDMiR^ zC-L-~Z8tF0ah5Q$;v?0}#)UN~Sj(DH3s5Nsy@*9v7=;~Bcx{*o(?n&sT#{Ync{*{~ zJg0ypzSU5}mz%0BmUW{_B(Lj4f;w1-vcC#yEMxO(=RbhL+y!13vIL3Q!;14WGFHQf;4CktWH8@^^Nu-ARxoP4-S2LNM!%R|d zEv}~29|P-3UZ>ht77gtw3jf-U8SpfthIT?3o3xDX(VNev+4r5{yot+=d%S^UMFHUR zpJ7e6E&XeODW}(V1E}teW?Y4)o=teB^F0W#mB^n0m%60vAL#nkHfNBN_N0v z>d&uLY%qFup}<_R(cy`s^2g@7mUuDPGS7WhpP1_+Yf2N5bla>sgL&X}1b6hL637yG z??_Y7mJJrVN5Q4FoCPunhB$`G?SdGB@Jg)+#jh^Z=*i6b1eaVt!4jho+0m{vn0W&N zge?B?StUEV9sNy{WTkj&z@(l1O;#oY({U0%z5@us|Bi#>SeWmJK$kQVM{|X2fm>O zBv&K@f}Q%x_bR-EjIlTNj%{jMd z1x3hUej*8PvDM3n9hqA~*kxhF58yx9LrOrvD<;3OZrMSi=x7a=5KHR-~OUMs@Qf^-= zFS0MMgnNB%(K~5A-j~n+fZqSIuSy{xqYx7SrvMw7n!-D74* zeTri*u1-`hOX1%jHU~WEqjo}!&8kSE#uHvmuK zwYN0GD>3nS(Vq6=v3CDz6;FMG(5Z_Ni^Im{iV3coWBbX}%JGj_Wp|yo9v%%&n-eDi zjBI4I1eo{l*F&ym?}tU{6e7t;@{Ew(lT0|8HW;*REy{TmnN- z-$&C;M>(=Ty~8f|Y3?CO5qB4@G!<<*mV7`tw>@FlZ@B^J|HcG>^+!qydjoOmFD>&lKi6?8I1G3TN#Q$1s9t8BgWrp1|%24O~rjEx`{<0BVdi^+Ox|g)fzKv zvdbxTAq{U4%D{XmMDca4|KMa!)^=l9VAKL=&YUz+;;yk9lptbWlTIU%l}y($h-o$? zz6}5cvVsz3A+Q3%K102J)gygfDuG6=y@CB@8-v;Y?Ap3ln@`?5>5frozpM&!1!cQ1 zReHL}Nef$R`!=%lnw(0w#`S(#!G1dcT!a0xmZ6aAE~cZqL_%z0!>n@L+xLY|Xoc&k z%sjr1LTmt#99&YyW+7r>!+q~~J-U>)KG#akcusyixqfrIrMKkvqbHpcgy>gG!RYKp z{K}xeSQ@kK!ykNWqiXB9ByDln*fpLd?!^QqzGb>wiLD{7KHGe=)VYl73x&ZBC_t7v zlTW^JDw2Mu@{aZ6*H)v{pWl+qWFJyuSo+io=KQ`fiD5ah=24q?H7=jFLBpXW%I@j- zPbNR*RYdP!57*`v*YlTibE{`p3fQQl`nnJ0Qj*80lq28rSzHSF7p#+iXxb*~+N)1W z{6}h}uvhvHmS(V#6c?fG;J@LU(!g9r+u1ezbVz5n{q4T$HQl1Napb@5zT!59xBs5IeqH*YN1^R@ z?mM9y*J1NGyvPu%Ryf)XFd=z2VMNS?* zG*}-x&95LKXdZ}eVPr*M=DwwHCWJVIedE$eFnREFr6Wp5?@>nhr2`M`A9i-B>dT3R zUKvLtZ&-3nPGo&rw7EAt{ZX2(PI&$<4<7fwYr|LE?X-S3+O6c^6hppbMda(aBXOp) z*)g?fxrx)JCT_IL^AF-M{q3p>iS<}Ecuj2*BAz#)To$6Z9?=S|m2 zAC7q(HvIfWxj;q3MndS|yxwP3YsF!*i$>Tl2+g3IT2HD?qgPJ$j{chEx_YdZyN~L7 zw%|W6dna^r`H1V_(c{4$RtGqfX1YSiplO~_UkF_ujM#@?lnNq=$K`+|mND!BXg)3< zBT6urrh_}imBu-971IW?u=7e8>zLqw#QX-`7}4#I*DT@0|74?cdUg5NvJ?DP^1ELc zXy~YiS?SBOi3ORNh2;crZ26C2roUsjM7cA`q00=Hg3F+NOu=P_OHl|C=kX?iQ$v+T z=@h0A$9_{;xjiun|9uf((|8be$v4m->@0*%ZTOQSOvN2wsWDpya4N!N!XR!KCBb|+ z4U_eZUeThJ(|cZ8@22Fz!Am^tP9py@2++Q3sfqC1Gkq3s8hD0it(#7K=%GfEa2ZgR z#qwQ?F1%yXz!KbqRT-)--<3%Dls0drRN&SaTw~2RD~xH7%D=2*65=Fn|1v-j1&+ep z=Ia>7!*e-DtBT8{MZ!zm9LXR2Qd4AR5BWN$pMRvlCV2(8@aiHvKk-iB$&tQT&fP&~ zO9=v<=NxenRPJSBK@~YDmG&@lPBl%OvAA+v{x$SD zbIxzE@y8bbA2Fn;|BUOKP2KBXh^nDY*clnI{{C(4UZc=b&w}ZpRw8pZ?lWRk@vEhV#m`DV{l47i z@qyqJB5MGr#&2@HTa$axXt!#M3b|eW{=Ob8EAqIYdswm%|7F5;lizXZ7}p%ax|^}9m)i=qXgq03QxiV(Lm9zkU ziJE5u@J^5Du3pl1m~T<4SVtefNZmNG8{O#aypi>eR#-Sj6mj+!c48S|8k)s)XXQu3>+m#TBista8_PKbtXOetMO^IxhaP zJ=SX?XnUfF$KZk@`|ld}w%0u3Wlo3ZwoW2bJYN4!u>WnFOt_a^>Y&3LdFnMk&k!?1 z`R9+vz%d4I7&ited$wU#5}TKi=-BnQzz29}1WiMSV=T&U`QU%Kh7kiOh2q7S1;Ayn z4Xc#V!eJ4?>T7u`mcPkA^_8E+lYQpKcML0tQ7UhI2d+Ivd{@M1LaL)_}$(fMtI?bw;<8yt34$2@p zAsa&Rg|3xUv%b}@_Y3CD)>>+1Iip!szb?69@A5lgMV91pxljo~Z++@Eq2JTT2r1>9O=(uDOv5JRW#vYK2UBabic1DzxE z`706=U+_7kcThei6Km(@OPZIO1=4y^n$KApn*W9#b`mjMjmaNg5}p`3Q`q4f;Wa?H zn~!CSfP)-NlfH^0j#7`TUPX#NF}LI9l#eBpihrpEskjbiIWYQ{TJ{w!WyNiS*sP~a z>&}a6K@5+u?C={P)Lx{p1E?Ljfofhu2hRqp3R`oxxEqgCQwW>Be|>sP~Imy#(2sVJufI}&&ixx;x3p&O?V&S5d zAgpj_N?xYLX-;5CfRvS3wJKCO`DIc7_#A={pkM%eaI(|9mqb~@E?gaJkKC}(>d8H4 zE}^Yh1@GSYjzQ4OufS47JS9oD4Sygrq9imCN6AvSzX-Rx*0cOT&-#vif%GhY87dWt zJ2q~T68cVS`(Ze;zp?kJ`V(#}Ma%*SnNzV90WzC=!!YCCwP5r^aGW_{nISM$9=HBw zKx#-fx(K0Ks93ZZa?=OqgvAJ4LzF!87hwE2XYzih`}q$!JbCA27)nj&Hhw)#NH>dM z!NKTo|%=*#apJ{aL$?o-=(PMK1cqGPq_H(fvom|Z3 z^mxYK&x2Na&0KY96GEfJ_A_a6ZVtVR z6A_l}a6J2a|J+h2A&yh5I4*R3QZfgBLkLEKQ>`PhOXWfUd7IEwZTA<)#YW(5LQ^K` zEFch7q_xb!=cPb@48stb&H`5+T|?%n^)L3)!}$KL+jG~d<^7($m}ooS%P~pg6Tcb@ z#KW*Av-JT@FVEWuqhxaSMy77pZaVy!QG`^zj{4>L+P8NjG42Jc@nh?Z+Qez#>Ph3B6xlDQ3IrGx4 zl=S^`;TNDQ=nP$N9~(L8O4Mn2e!J+MvF|wWhBRq3KbwxyC|9{>|L4`1fQbkeB%U;W z?lpX=0Qh>$69wrmm2Fs&a*^hn0d8yOzfA(nC<&4*BSZM`EfvfnNdM(xs{lVcb8d70 zLz)zazBtD=4vNNvLB#NwTP#Oe{_L@J)(($I=&_`0zrY7P(vw?BSu9bSY~j8Gq7eD_ zmZFT2IiusWf08>nw;EUArlc)KA}$qj7)ibZkm+EUNx*bK+00#`1!GOjE7*x%=xX^y zjhX1R4@m(xdW~Lfa*e{^31jhr6eVV%On%JZoDv857^}>m()ccmGLzN&nb=C#(zXzo z6eOgOv@LWdA5T3H*)k5{%X01HL%*RDL{%N3OS?vHf&3E+BJd@!x!0MJ>A zd%r#R8E4nbm||Dfm`&PGNEz{;;lx4ED{4Of(XeJ5RvE{JMlYqTrAzg^m4D+iDtcW7 z&4j9;fb0SEX4bH)QiS?`ifSc6d(gu49K| zcLabgr&arMd+lrv+@A(@8Q1POTnX~> z7^Y7a=+-ZFX2Vq=_QX(`fWm}Yl67>_4+}Q_Q`w~sYQs%1(Uf(h>NX7dYH4n0nxrz6 zoCBx80TtbAL$L!xfjTc_DE4hwLZ4v@`6pK8Zuy*n{;Gj6WQW$+oX^hS7ZXVMCitR& zIv|}*Gl984#VD^Do$}H1zjr?6;**Cls5-nVXJU46~{5-@J_tNysQi7}KQ7?l&?8n(|TxYb)q@Q$b zzM_RVAWGqO$0sYY`MI5AJyInKB)M){roAm)e=OIEN&ps$w@W*udmO+p` zCMN(I?-z-hvS~|oak&BYVsQi9a$?Fh{?&aL0~n@|IKZd|9?FTC2s{*{n%!Ts>eI}9 z4@Ncq2YEOu8g#<3lCBlczTg=^yY?af0s^C=JLgYO>=uHTmO

h+M&p)GUe{?{P0w zwD(Wt!A$QmNW7!CN}I>je~U@rgUT4$6R-f%1HGC+52T6>0@v)-OjwiWk$NW<@TH1< z1W}GtpN>nPG&%lm2;K`7?jM^)RsVFLJ}B_Nq!X|>mTo@l2^#&SMo9hgbgIOa+tcMs z$(9?r_wH0=N|EPzdC!Y;g*FCPh0zzhsj#Vb%RefPgvd|{FaBo-lnyX`VMQ?!*!Q2@ z{x>!`6*H&{Fq$r0?p?mkL`fuO2IT2DBpD4b2mMkQR%ZOd0C%u*?x)wDSe=9YAnFm$0bzzL%X=ww%?y7$4gQc9M0%TIi1 zh|7PNTBrBjBWL@a($nbr@6@d4>GpAk{2x$O+Na?Y6_N zvRs-Pl-%_(bm4F9MfsetoPg-aq#TB@xXGFP$J4RyyFEn~^9Qv`)8@Yel|`zp96E~m zGqu4z_}GZngFZ)#xnA~c=heBKA_^C#eOaPQa-3x}Zt1G#6jM)FbDx=P$ea?r2VAKl z#4OR)O033a5{>~3fk$?H4U|weJ_m8sk3}1B2ym~$2I}GA1d(k z`dS>R#&a0@`gx`Bomr0apBwv(APYetN6X&lH?mzjo$J2 zd*f-a@7-u96o>@UjPEE)qjZcFX*Xma#t|pQlBf}YWwd`U0864q_E4RH zWE|}V_Gq~cx#fRo;|(8#Oz7a?z@fzDO40r=;VHoUiLG zQJXw>b3Lnk$X@Yj;XKsiq|1FTFmo>U1%A-Qi&)!U4wA8Ok&*espG#YNl|DA7VE$Ci z{cOLuNi*Wl3%7aUT>>LA7pRTm^C2m;v~j;5S1&}4HJ-L=gqhTcjMp1;K8s~hmoEGU zSOC0cw|vZ{AVftZ>6eRB-oMgRBKh?ri~Lby%ofaf$ZK3~w}5l|uULiACj2>#-M?Hk z+_f3Ex%I)zo|j4|?NBQA2^`-aCmEWe@KSJu)tHICT2mNTu^&vvY*;Tc;GXIazN6ru ze{HrgU^X~@GGw-~rc(1!x!A%x$=F&?Kcwe_4rb*2&c^AX!Hpp7k7Bv3T0b&A6jjTP(mLqg)^UNcD@FTP`f1b9P6^!RcmNhcY}QL* z3^@G7pQHCsQ(+)0rROCwP<|uSazSWRzU_(i98JM(<0ifGbLZwPZO4CbTd&KF2eRrk zy)te4CC>Z9Q#+Albq^xnHgi#(WSD0E^X@fpb+0$Sq#$QcK2DfBzv$u#BM>K+ge+y? zq@}Xa8vZ7THC|d^M8$994fh6a7=dJPxq%b7;T-=FaKoHU?3B9wlg13Lqa$+pMfG1@ z^Jv-7>T%1Qf@Hejqzs%m0nt$7o!Pi8afPY@(R>^D=tjHWYzeZ{iH&UIGE)8=4R4rM@pZ-KLB_U+?F#kuY;}=*l(9$h zj7gZaGfEP*GRn+M0e8D_dns41+sAyppx5~tdF=IhcQyND66+ft=55342DRyAqNoJy zY2N*!4EI$MXW=)H&4j%Z$}>sLGP#A_=yErV4z{7Q4@u zos_*~wp$JH&(NY7;Gd&kKbNh&9U@S824LJ28nUS{Ne>*QO5oZq&9Bp7&?I6`Au9|E zaRUkZb!|!u0Dh*)FjSk;nlxdH?OV&#FposZ8jl9Pv8tM<(32g<1BpL$en9?%^V3iN zoCIt1SP=TQmZsmXKhIjRv15I_I=28qMMRnsc42qbz=@tWg$gT5H=Lax9<8$F6k`QF>Z+)Oy9cqpc-cp+wM9IZGh5H$}f=gX>cZ=OUzbu6em~D zX3Y0cRN<$WcPQSa!mz=B==R1O2B2FZH+L>d0#Oq(u1qiv>3N%3uIW?xeM|s& zuwo(scmNq~q=`(%*1p97mn<_;?zfEHG32{1M^$H7;U_~(f=nOziAiy<|agS#?VXA6>< zP?6TTzKn}{!&)Rzx-B;I*m@IZ6ttnDc%`9guK9LV4OdaOfw0;e=Mz!&}bu z>vzokYapN*#Wn{(M7nBK?UkXvYQc3?lqAhZ__1XlJ+QP{v41GY@lsD$e2lX& z6B^yMbcAfh7ZSsUuYfgOoXTi}w%2iSZZTa{jGtz8r zCiipRIo7AhU^(IY<279qoo=X~EKvM>Ht@VBs&q#8NB6#ZGL4H(WnNcShoQ=+ z9j;~HPF6riYan0Ru|2AW#m6Pu%YN4>j2`CMgv#lemcyW1KrJsRrLN{^Kn@ufng~F} z$Ul|Vv$``$R^)gWj$1e^e> z0tqD@^$y#*8Bbm&zdxL=JRvVxGaL~K;dzMfe>VhznM_2CszOo76Pc}8lP4Rr@d_et zlyX7YVj~cqWg-BcgzVD$8yH_E{EFe78ACin!37y99zrOs1ZM>O zL|ba!SC+reZdbA9Ha4Dob|e(+ymK=R(saX`iZR;M;UhDj76W6V8Q7@m1>JuIVhf6^ zXI9HnV5a#(N#L+%!WJuTa&qIRUiw*99ir@8q?(R!`ZsYD>LeIxkfw9O76ksnQBGIM zy6Q49zfUdj=XLoYNUMaXqGL?mL&olzMi1+{Wx=lDt8uI_xl`}kIum&ot+nd{X> zZgb(^p~7As!@PI1o~ELzo!*X}A)f6}%fiCi-FzsQ|9E=yX_z_@|DEg3-YwAC6I*<2 z2MF_4IqD`ui_rk=*&C$dswlC-3}cF{Azbo?um2awkD+lqa9w>9jeEe5!4SDZm_vwPzJgjK0IHpXSsTELQ9yEyk4Z?4+X1cl6MmfUpp^4ZhSiNG1l=s z{R3<1KtVc3`%gHT4&1&rxZJabS}FN2@S7nZRiM+>9&L>@FNL z89Ge_HXNL${_om4uZ%wZjcspUj@gnkQ^TlUzE#?VlZmDuI%O9TmxdQ!wJq9!XFkVM zbd&p7LsMm6kcjB%CZbkke-$E|QI;>*`DL>-TRA3^MbGRq%$=o$xL!EUXfL!CxU*gu)BMCQPy-O@5MuPQY0V zhraeW#IiWE`)s$-4gNj}k=;wmWfjXeOo zSJq$Xj=|>xbPyq!C|cUIk_CO-2L5)c!Jw6n$`dDfb_P-gb^qzY6M|8i$iXu+^o}6GkpRU&>13fh_y%YVM9!>a_W1mk@g`2b;pHS^OFh4Gr`=ZhRz%R!!GWD$&Q6cz zT7>|`hJX|cm0U!9K9YPzT=S)?*)!epcsZ-9vsa-)ua7f7FOVwdH@5fd0E)(>GYlRwn? zIdJfC`NYBw3j-dg3|9$N zh)aAYKwMO6C1{45m%Fgf4d*^pfC(}w^uH0DXgRaItA<$u*}oK94cOn?ic#Uz>&In? ztH%r5KM%LA7FFBQ>TPPDdZ#TcY)CjZb8NR@{LhSCni3mXa?(^cmzGOz{@w9!>vskp zGEphv7hYF)-+<~an74zftZ%kq#dI8b*+JNJDJ!3)Rrj;xUz+xWO=V$#89b=>TYO^z z)D;{YxSA8g6@5*}xW8PBJJyw#TbbgflN9ESDgzp6;gKr14+MbBx&D1t@cZDoaj1IC zx(!2iJtgPXkxG{x9ABeKH0C0wd8jG`wN{~@2h<6EeusI8yHRVv+k$?0-?{lQ8;VC@ zup~tQney(j*f7w|k8YJ+1f#zeg6B%_S#t_ehzW=1lWhs#!uPwro*1|u5&RCcAF=J4 zOs1v4yD zZ|rk}62c%$n!;pUD?fn6<|b;GxDw22|5vw|C7MaiueZL$;vXJ|b|)A*)ENHQT6MvI?MV^{*aP9Xe9_f>JkZwj$%O@5JuqCEV>fIWNt55><4E zolpv1O3G`=>ZRNh&CVz1_?<+T*NKm9(XiFqNug=ef22#fQGW3Tt%{Sh|0vI|x#|+I z$fE6TJqW+cNfYntQahU}<1QI}jXkce0hN3qiFb|u?HvnH_<8%czfExTj#hK^5?4%V zmb-MCY){4mwMRFu)Yk0`g)2PqP9?uA=+McuKJ_=>Z{d}UV^qkOpj%6-dPyty4#9cd zKp}&=)>$3HAUw#l67Ute5RGfh@vmGzN4QZWX33Cj)pU|)aMoynR10*iK+RpKiR^%3 zr?{r0>%`>BNr)mQb}3G1-phk$E7D9=ngaVF?e)A%j9T)Oj8rCi=_ko%w-lIN*h0{< zXW6W@_%YXxBm*1?j7~$2B(q(WYrpMaIDh+rUk<(0Hf?BXaH@numgwkXbEw`1X0n0f z?&Z6FI!aM1RHP{%+af{Dg4qoDd3I95W5ehardxfMFl#SG9b2_aY|8(T0&r!7g7mW% z4)(G8MRwT&exaiykJ;kitW@2~sEaf zxr?gpdkC=s@MHvj4e(_3m5~pfeNJ%8sr+cvr7@f?OvK9${yTD3cdXLndeH)!RUdw`eF^xCI!CGXoQ_s zf&d#s2Vxi9EX>A)_!*|uC)rWNtc<|?U$Y9I+MBiO2tTF%3%s55Zq;W18d>ON4NeT< zQDj&PJ#4zel$$lDgsfS7%pG;jz`cNh|Er0-PKW#CA1jmdQ}V2UQ7!u7(yXeuJD_>& z{Msf;pqw*Ix^WLL=(_jn1v+aPS?n=$CJwID2b}2#vj?A)!gtr(dEoc zc>XouMytj+XXcbFHTHpnY5@oSop)FDoozb2$L&eWXyHaJp>%Yt$5YI9uhgGEYO32! zBmU_+y8KkXw3l^h_xShZx$2xZ6*&V5E1WH?co~QIN?uUsU~+Wq~Z}{6@=aVAxK?|)wJ_= zli%b(z>)gW6HXnqzf|finSWt_b$0K>M%pw-%o~;(eR z&wsk6bc5>RSJf`f>Un6QX>5gIqP5yii|v`X=Q-DY%f*w+^NC=FQQPdJgq{&arh>*d z8nDxBTb?hLi_U9Vn1EQ=@?SivFVS4!K=Prs35{zsSiRj^yn5jkUdcTkuOr%@|9+iS(TG+|vGi`xlQsf*j5 zE)xH&+ejoEh^l4Ysintjugq6c)+Y~c=4C1w{Yw>Z2c*6A@H}Z{`c6vU8Y8A zeGlTVE{&%J-r;#2*-3vi%5F??oWcWi#LHO-A30p43LJBot?q38NDrd+q$j_&?-e_w>JkejYOe$#9XS2OXF@+s_ET8JH4=#L}NZ8f=$nvcg1s)nH zmn7csd_RuEEX+2-*Rv>QP9Lkl`cg;#p8?kD41kYVmIA3TYGE7FecDE^OZKR36uMos zoP&gC;IK9(xUr3j)!@gjW6;Ioo!4bl)abU5cQr?R6_s}1nI-aS@VqXvhr}^>NeI-b z1u@P997SFvMFCplrB=*eeeECm_u!`$FGa>Bd1$b{=U+RmA1CXZ%oV>s|HUG~_pc;; zeDIPs)>R)h_%-ES-$9AyUK2_==o$NK_x@bn!OiRZw@ghp-L7a+ z=J-$Xt7mb^Ez1%n9j&OKnJx&%YYoqx9wLR5+~+l!Cd zPkwLVvl`rC4WdD+Yw7R2ElD({M% zu?fN7lcYzTua21%9%XnEck)-HXm*9`=bq`iwhS>9mnSKy{35_pE@S`6F+X#om_vju zpg_cEAmNXsSq=)t)cF5CsHf@8|7_!;@9)Iw!!za<1gZY7l1j~iNSh00E43+GmF_y6 zT@Xu2yC9YX{+$#2v-EhRi5J7nlK4i5$u_K?PU7dSiLaJm=AGiLFs^>eGv4Z*Swbc^y?&3}$@WZq>s1Ra&E{0otuiv&t(?@6U(@H%|ddADNLFUC&N) z46-Xqb#B@IDe)n>oMrRyot&;n2sN9>CR4BK%nVTfqq_OLv|cflA=>=@oOzxX zP1U5?`<3{+ay03prO(HR%ZJElfE0#o0|xxXH@JC@RgPoiPu zduU0ehf#XGBK$|aLrbu;+OQ)qGYu{v#qJ^9Z$C0eP}z-R`6dwmWibLzFC72nPdsX! zaN^0sz#pRu@u!_X1Xd!A1_e z&!2c&eYmI^@m?B7tv8d}F1uS%`Gt$?8OeVwI$ml0t)4c6D;Ibu`%ek6J6*C)U!fY+ zC5Kzk-Ted-f8Y6YLD8@Hvwbhn-9_LZodY~a2K6#xes(kiYPxi6yF6ae`jyaQk=L*` zdb33TX!=g6*NFF@;J59AFG80+*OMGXGB~Jz^J|jqv7eV9yM(ml|9a1Q<~NnuD@&IwavV*nA4N=&zoJfT=oh|B)BOR z(F0pH4{X*yC&^u%5Ub&8{|-Pt#0Z`E`Xhp(B=DHU>CP|5_4BFAgC7Ww8NCm7g4;tu ztN-)Fz`~2Ia+-yT$|_;u^Bx6(ismasHOA1R$2*A3NvF9kSAi%9sz}{ z%Bos$Xjq@C4nf%+$I!&`(Yq^Wx5M$K$&xMgCmyR4TTQOpo`DKMX_GpLP=%n+RkbV1 zMkd?T#sun~NvUR?9nF8crn|N_XK`WYM-DD9kLQ*zsMF*nH#}i!_zazwad<<6rwkWN zlMQW%=lZ4gx*I3q?VgUUUn$f}?LEVv5xN}dQTd-!m^t=0ga^V*;uv|8a#f`1w02tl zeY0M)Vzrle+b&oA`J3|sMdy8WR(s*xRp06FN(~W}X8#XUUmX`^7j+9FA#IEx2#6vm z-Q5TXA|VXj-QC?NEg{k+0y8jxbeDAJ(4cgKbl)@Y_uc!u|L7U#8GLx=oW0jtd+l@D zNBqR^JtNHcw>#FOR-1-!JZihcE!|BR(|*vxt%~m^U|B@EP%=zCJCuBM{MY(8sX2am zUw@J%<%-1A2XsH*o$vEFcw9xAM@deRCW>zR;QE9~PH8{Q6F=npy_WnU#Vn9Z>~(HU zET>`t*HeRB;PN+k=n5#MmeIpgFv+SrU+(a?-M+)IYw|jJq8c;Qglm+B+^0Gt^$p3J z60gK7s|-*#dh1%9>-%p}ntD<_LI*mi++3S_R|7bm&zRrQvF5hfJo4VQKby7~3~aqb zHMOnk(Gh&8yVmmSXUBa54crbViNZEb2?uQ{|F4oL2C^L7k&`&v*ub?uIV)YQb^300iM3)3xE&Y|0n9)nR#rBbBJkFpvS zfG|2U^P0(KRNUA^?pxfZF$Od)qDAGtxlt4=%3d>x<5tY}nR^lkoL~IHrsyH!)y_!i z9p=j&5exWQiw2IhSwKf?C6H`D842E3d$(&oN+)866ikb=@N>kf^ChS|73o*IHBVo@ z>iB2sG2Z}beharln?l<`lBT2W+_+<0Lt9QeJ@YXproCEg%eiSy-$j0@djqbpMk!)w z;QPVQ#{g;E1GFX7N0(CRXTNf{331lce9(ZU3+BX*b(tOteR8ZHMI4&DvOSAZ=sCXH zw00MMZC3tx>LlXZLC3G{*;T_P+XYJ!r1SuW&id>XkH=A@cNBpnt7t3ozb{JhcS>9_ zbef#J_9s92Q}u6OJIx(Sm#nPc1j0(Q-$SCrO3Jd|r?+C1=yg5uM-JiQQStqG+bk?} zOaBFKdYSt8qAD3J-}JKXMoDCy3$xhy8`}=Uznp!3eQQ?N)$E`$aDQ%ncS$|{JRAoT zYtaoiegl8D6?|P%zxX*_(KZ~S?IQ2(o^Ai5y+LaY)o)(;9IU_J-q&J>S9SD@mBq$2 znuHmbWr2Z&x0;im>cteXwF+kIp1qM?DLoL#b&L+stkGP34D<%1%%tfSg;$XSYIhu; z;~Q#D;8eQkh}UlYN|-gBp;IW40w7a(-vvu9cwRD@#|oL8mfBF_U|jKH;bRfef(7Fb z1nY6MGtckQQ#qEgYkmk}Zw*isD<}ZxSV$VNs*nN%)2?Vxa1wIk;CS^To3I)8F90x= z5qe$~Lly;ttfo-`*=>s3K=+Xd$yNor5BqnWkKI2+YRJxpI&(LxU$K+j`S4N-Sz9|e zIs}|FvuJjcZ2P5>q@0+ZtocU2Zb2c41QddJeQwiC*oKu1HRxog!=1?RkW6F}qra?4 zq_`&Yje;#9)9vtm37Qcc?&nY$~fo&vZ?esy3d; zd474aXnDYV)c?U>fwq0K$6ii;C}#2N9d%Fy56ONhJt?eCA&3N4M^r-ETnJ0U{6mqO zqeus>n5&%<#KwI5_In(S3IrAo%Yp&W4Gn=!J`)75HT9oApdQ=rYVX-RTbV!DDH3^obJ%fyb@cFAhMumR)FL-k|7C8E z{Wx9HO;5+tyt!;sv$gGa(67Am&ziH={p!GH4EcEKj@|DTtEwsZOdK5PrbFn19k42B zPhU*%i6L8EHEue#u}9m)l<)p&$fCViY5Jz6K&zwY?~R9U%`&B#&3N~(9G!58cOoPS zf_#cs3`$ve_(EHSxA8AlHW|u4Hc3rE2V?S$8f#i{2VZlWxKHO(<3ELK0{Vh=RkSWY zx5%0u3_1VF*|Rm)`3OD0^N5epgVIbOGj{>v;-~ zGwM~Xn7oz}^9lIRFsVfG>2J3 z(az+z6kdy^O6AQn%%VL_qhZS;46tzZxk|ypZH^7jANGv-rmkgDm{p@n-d~5RK3RPS z|07Y#6|0}R$<1Wg#5MN6U3$B{eY=0**F?`4dp2#{v3q=(dTz6sayRcq6Lf_dQHi3y zQ;L&llhj{dt7t5l++JQ4CCY@u@xhz9^t`Ts4}mWi@S!eWpRT>Z(AEK^VDwwq^{q5ycM)RQyakN!!bR_k#4o^h!D|G|j>D1pLj0!V*tOg6b??By49n!1 z@p;Ml0t@0uxmQT=hCchxNm%^}17mlU99bTtGx@CEI4D0g5G)`-`?MZH%IwAn=hW&! z)tv)J^C9|nX*P0r8w2NtXv0O7mTld94RK$(k?<3*Qq7ubFL4%f*h&Sn5s|~T!F60W zxRi}7DM2dO*&n+8g9-w`b7t&yf2gNW>X~Ha{d~hR`~3J@&aU8}d(s=jz4q>AhWwTAJDTluc>9(snX9SOhV1FM=2(D7<2Idp8+Rpnzy zh5qjG)Seu<1!(En5@OlEoz+DsNwCuhfv_L*G_bMql%9_|Fu8t6(1lfo36ooz;dK4d(ssiaQz&KN^`LosJo( zxn*Paev>Oy5j2-@(*Cg+zYv2K6HiY4EgBhCDbkUIs@H?O+X99_222kMM zj5j_b=go5#*?cLow?Teu9nDLYkss_&goAIfYf`#sLdxa}jmxXTx76`@lC2o{Dwizn zxnTFy+-k~L@FAK6#RxWcbFK=oVnxK-ly!y3{1Shge_gwUsNPz(7e2w)jJokhC4K~! znp)D)hTZNrNbX_JgfsU=#&H+j%3Zme+q@0AMQXvFEXA(dn$FXD#e%l3v;gBT;t}_G z(NLR-#*1gaaAN#QrX;&yTzs?0V=iEO5}RWbbJ#jMFVhuXq`r28_GWxFjOb-@-UWNg z`?#TOq|fHbWpm0t-{mh-2?RAV*2+ub?qMzBj4pZG>ft1HvE`oQ-Z%_Fa8~dE^=<;L zqm5*TC@=~M1Yzr(DIoL1?)P24$N(o)^2|Smn^%eW9w`trkOdC$TlC{ls%{WZayxyC zFrhvAOfy0wA8#)IGmbO!b3UAtPJ=TlSLWd&C)Ikc`F*?H3T~E$3F{Pw7)uguHab%5 z*DWsZv;DAZ>cVleC7syWlM;PJ8I;JRYjv00wORcc*6!!3c3GWlrRkI1Ci1losNGy( zlMy2krv}8x!5pkwL;2DPu}r2~Zk~tP>j%Uu;T~1%6kK(hVG#cQ%7L`?O~#C!U8_44 zZoa6Was9@+KQ!L`e5)ZfeWxw+saqz__p;a7?pK8K#C3(Xx78^VY{C$^J%8$2JP}e$ zwYe*#sW);t)3u93<4S6^0)M0h^?h40WU0w-yJ{JOty@s@s(=-@xWwE?vM@SR$=pY% zIvm|_mylg@YqjnY<#wh%H?l9ZFOJQrT5;^_S4jVyrKRWE|vyH_n z&Mmhkn}aGBzV$NgZW_7~OEN#pkwH4OOtPwyxbu|$r_1p8j2le4KQMb}4eTb{FAVOP zcc`I(r+iq72qDKYF<1169(uCOr42!ZYf_Xzv`^;fvD8A~O#v%^U=q*&vL<35_RE#O zvG&w&S>tZYvqb}E_hopD=Pxwf!`xXXfGDf<&SPHHUb_=xE?1`=yQbHTgFag9W{inAqiLj>rXMouciyOSWg*k2%7lMBB9e1z z;S3YrlNJ3HR3qZ?(HMgQCw~pGJPG4W{M-8`PdnGUi>(RoYX>WK0&e^B!5TxBD)WP> zK`Go8Bewq5t*nvt8PBP{KBy~fgiDL!N30N!NFn#isLI;m(aG95+_roh>K?F;OEedi zO^BL@x?mnQ)DNVUOsrYdQ}@(a>V7ogez7gT>?byvx+rytVakyeRuhBqNM9(jV~?wh z5dLX9i|?_O4i6PBbD;mESTO-~oQK%zm>9JTLMZ<_!Abkcgnrw-@c4d0?Dd&W4f}AV zxj6HdHO9?S*@Xx$eCE?Dgk2ggY$%ZHiwzR7X3|lU-nma1eYYI#FLytRc*cQs2k~#4 zg3ZyZi!#m>A6H+}%2d>0)t`1Cwr9IW+D~KFlE{VT0`yu-e4c+_LK-lBDLI7O%1cN{ zKviA3#kVemk?43BN_+!!Y;EY*;hF(@OogQ9kn2mVIP7LaP6UNPS9v63?urWXz|oiZ zx8p34c^Kv6Xc9wKEYaD~f<^QS^qI-k>vc4I_C%Wa$H^aGKH6Vjrp^4hDn*lGV!;_h z!u~dg@f)99JERt7aDp0nijDM?AAb-{lEvh;NP66xJcytVtoc?t|pz;OvM8&+PX(VlG!`XlvZov)dbI`#sq8qI#~J6Z_u_ ze=+&=+XH=euL_IBTI;TfJh7T-lp9N6`SX1nBI@JE9~KxY1_~83K48$QP*@IFY95t- zn!7JgOdlm>Bj+lGiW1*H?VbXb+|G(>1q|a8d&7S3bc9%z=6JYHqPjQBNLA0&0P_T%AggOW9>Ejxa2t;e z#T{-Z-GYx*^I!i4d8iPrU04*m9c(o-=a9Z?Y{UkQJ@ta`(2)`~%-5JT z2=mRiC8N=gvG_SN*2$$g){x=cBKHvJ$TyIsnecFQFmlC0bpg$ioLV6c!j1Gyv9q;> zeJ^jj^xRWp8>w8V&C$BCjAs)$U)0}0vGL?T@U5#k3e$~zd=7C)iS=msjK>#Za%M7J zPqemYTY~gcZbpdc>9$M(eW7YT`wq>BL&2eo1*5ZeKUowvK*RK4`a`UoCr3McIb=ho#hI zRjC>F<_OwTz@3qmgL|6!sFy(qbZ(O6-P6)iVng+C4lbd#ExBqN?21q}7fQsY2GOzK z)-#!0ys^cBmFNEMK1|KI&M%>J8^CtHgL%w<^M6KnV)8W7Q&3KQu2#rEBPo`c^Fz{~PXyN&uZK@4k6n zwz_F}5-U_S+6{&aXLzk++d3^`pJQC7yN&wUvZMR8<4_zRO$U=45`M!mlS#(zb`-9B z*(-f8PU}h~&8aH((zOIFUtP*lE+FR@PD{KNi5Jr;J<;8?OZBR|2_62C3nONDm-SV$jeR8{`u_finjb{hhgIHcm-uOrv|IWv6KdPWSxih_PlL<>rvaV z$pbeZjO;tnq`dv$NPz~!POj|bUy4_pDo_(~HT$GCoe`5SpR)T#!@~o>(8BCM5qVqB z`0!L=J%r^-?Ne0MfL{JWW5z0Wz{=GZ8=vD+&!!I;iH9dI|l!QH$W8k4y$JWSCxRbv10R~xi8n`(F_o3gVQ zKkaifrgw^yjOVrRP3Xw?OaG1c#dLQerEL7G(QP=FxR_|cue}RpoOQYgine9Ls1E~Z1w~V1tP>kvSywM z-mml%6B06a7o9=Qy{pNsD|q#X0QmnX{*KN55d18?mr>P-{{YARJ;t89A{74t?|#9^JHDdfR+!D2VEhFQ z+0g4Ag=)o@7f%=W{j6Lyx$ags`eq`epqwyq=q|y{6*)8nP9b4ssopb}#eBnl;Ic_B z3)QI42!Q1%LGr|NVGsxg7)}f>8Nej$R1&2rDNnN|Q-$A67@?o>y@U9`TwfC?f@8>V z6m8zFZ*zY6wf*^tqhsQaBj=}mX#k%wG)Pb!@MK|-9Pk9gJF5RE@Mf?`W`lx3I-Olu1WJYk(9l1Xwah%CK{WCDetD5a2fqMg_&YJ#V(TN;KK?;7k^F%^H zwR%m(5RU}+BTmZ@9tEvWZ1z+Of;U)kVP}5|Ph5i=7ngi5`x!#{$*F#n&T!T*glm`Z zR71->gl@yNeyFhE==9!0m+iP39TKptP$Fb-R<}&$V<|u0|5q2=!gxcmaKBL;-f$^hiwx3Udu8c89T*{bU zCawSPJJT{6k2Dg;k!#Y{vMX@8Rc|0GGlAi#5lES}p&(_tQm*^;wlX-U=V9|1&w5StwYQ1?;o0`w zXhEFe*?;;2hdYeMc3Uv+o@c(nr7M9H36 zT=!;As~SAW9g)FPy)}V9lQ6-0P_S+^Cs~b6U|u;U!WL5Gx?qE$ZgT{h@_k#7tz%*Q%ZaP==GT2-Q`d- zXrOHRcN4)P*i+^HB0@{WB;rmsZL z*^=SVp}WI|7M@;daPP?oniar1{x*Ji=^goXdTJwLIPjPl%SN_&avNftFr}f85ktoy zkmNB&!QU=?(&dcl(#r`exvOu;vd=2Vj`bof6su!q1)#Rl1I#s`TC2!K_(Ac6So{th zhS-+9@>NV~a8{ZoZAKX=VNnzDc_kKb1+05$}U$FuCWEah-lC@v5gGrg0?C_cVcdwS)g9@w7qT4h$6Zw=J|j4XA*^f zjsiRgk93BP(5C%&%H_!QTuESG>9GNs)b~-$bcSmb_sWGE+v~MYskS;>H zmQ{7li-lipRgX?_$R$2Is05aVJ-Iw1c3lQZG(2y`r(O}y=y+EXX~_r z=Z`y44(^n)tC~Z9W1{7!D_lLM@(@MoqqS?N#N1>;Xa^+!=ieeerWe!Zol=Fvkv^Re z2@Y2(a$R2Ty0T(IvgmnXcDeV?613BAdor#aV}Gy2m^zJEOHgI>)mQFiRJ5D!FV`~D zMmT%P2Ud)H1t6Mrm)j}&pYtO>%kEqJnYYQZFOLklz-}(@AJOp)Gjq zT^dw3Mun@x#i~d0P%+jv_rq*-DIGiWV6L#uFrx}uC$kWHbw;?IiuDK z>B0@3!W4iKd-|n*AR+*-q?c!I%=EPa-gXsS1E47A6fgEl~ zGUuX!SM#e{k877=f|~lVmrW~s3MaqKZJeDX8Wu%2L`DZ}-;!y_AFyT^ebt*iBVPc|22Y6}m-a;uJc=*0I$KJZMPa)p)Kt$KT%GSqUP z2iBIm@n_GuEWkZCWcJ@M^@7cscdhKtOo5vGxc*na_BXzbnadhCMaETJ^n*tFo|UnK zxX+9$oEiF)u69gw0_BZ%8Wkh$QAUq)gnkS+OH^QG z8{v*f5%E}tQ0_UEkjeob3{Y1Vw`a{R&J}yoX3x4LVg45_RuB#J1w6kX z@ce3P;%GFxZ^~aj{w4dDJbBgOFylv(@iP;)pE1Xi(Zv~5Qa?SVShzjG92@i3KJ%bB zpP@LT*TxqA0{iD;5-EWZwl!)o*c$hMg|h-nI~PKpC(dyCObx$y%E#Kib#kKvFhz{2 z+d3D1`l6(1F~OvmOkcXLSLWpwQ%Au=DYH(*3|Jys&rSr089q;dolYj=7i_zCY_(tl zak=C*JfCjP3m*Ym76xEPy&!!yOrqwAcRdyFI4zELp356`C_Mbx;-N(Q^NxEle>_6; zp$S{(&oEViqSXpz``ZRc3GAkvzRh;xp}Y&`kQ%l_f974^T+%?*X%%+$+CUZi3aKl4 zHA*hBx2mah;Ui&xt;U{0q9(eXG3T42l$aTRk=EL%Vc1O8{2@R6Qep`wQr2oDjdZTn=z-YOn~6gX++ZJD2!=sW=`KEH zFbGvFJJ)@_-&RgKR<@&06HLrD#h9m0n`%!B_NcXjbz68lf^gBN%_zA#xrrE@MO{<* z7T;++k6MrGerH;KX8g~WT_J+;WA{7SkbeEy^l!_O5~n3nRxUi2r;s&^<5ej%wPp+KuU!+P{1CdG&M5=(9@1r16 ziSC|yh{@CD;|RzTe)`MuGoAUb3@4pK9En32 z{fe*Dk*~Ltu!3EV)uWF&h1%XJb7=~O$#OKIJZD(5FrYk)?7ezNo-^OY_F#mvv+f*b z4sZF>^e=(gW8;3t(|#rwXt{rA_9tSS&;P{$;$3iG;CfUDFqf#<#S!RLqlvMV&vy0o zA%5pueFOE`>8Na1WDWDOx2ZvKhzGAEaBlSW7ci}=Nu3g+Ote=xYGWPZ6c9Omf!f5$ zuN_~PJ0$p6OhuaAekaUvcL1flJ8~y{)nX`ug(4YBD~<@HI)~&U1n|D$sudE?T}QgTfUnT zvEAqDk8D7blrn98w-U;j!$_GwjcYoeq#?2x<7$N}qxE4tQWpw_w;)&dxZqFg_%<+f z1hUr?HS1sgNJKo>!TfpAL0Uwn8uh6a1~^TrrTU4METt*|yz14?#9w10SMLh8*<*IL z`U|~s_&X!+gv+_y&X%tLSmgs?#mQp)t;gLFvL4SM!#q}W;gHz>Z!Nd5|CDOL7#tDG zVlx|Y2zgOI_NdW+r>@DSdY3F$%Kn}BFOUe`Ds=E{?sefY3^?nM^Emfvmc>0IyD!Uu zoe-t;s$k+!(_p+Cu=MZX6$nTA+)`0f4^LNDBj3Rry8phRPVP5J_=xCc|3G5#F#o|9 zQ}*NrkyOOa+Q~)a<$*2!OTPUAw$H}u+3odOK37Aq_jJ?rgRgs}N8V7OmkSKst;IUPS?28^MR3{ZQ@kxup^J0}&9M z=LxU(1?fYjmrt@44@Ay0C&c{n@1o4G4e!XPLpGi>0`@W={W6y&aN^JLog&SuvAltjhKO-5rpJ6erE%Uzgp8m3ho(U z)xUB_Mt}WiExOSoYb|10O_M$n{K{7wxgIjdqaCPr^>;$&XqCLYz5wa9N@i1~sQSh# z37am#1xGE|j^0W%M0xWCXNH4{6XiyVTEghCi}z}>sHjGE(Hls@M0mIu7y;&Eva6@@ z(Dg&c!Df8Pl1m)r{jb(ke#$;_*!BhOwIuh711148uGsU-YkdiXvUuk*_1A}>?&I$g zqhDNob~coHa_7sPi_6s7VyR!b3lu_Cj?8b-s#3_jeJ7)L#i_GKNI|};yI=gwf^Iv z$NDry(|(9ZT!N1y;Y-Tf7!E-`{LfLJC`5Aor`DC&0N|6tz=!&cB;8>%+F0h zEFiyqCh;!zwMmBi{et*O{H$+ru;;3|1WFzyf-ra;Pr0t`v2R|bgYyPGBB;eF3f?+F zv6#^UANr}<)tEAZ7p7Z!kg9ug0$-Y#->pHLOqV1m-DB|ZcX3NNKw`5INvrwLEhSu$ znvCpHW9aOj4((>aHwdce*TqdEj@bvEDm8UY)HZyH6OZ#I)ZPWN(Y~^_qjAx9W_DL< zzz+Lkt2yr8vMNR|f8@zNm&pkQOHmCnZL8lc1SU~}Uw9z#$Eo?4{tZJ)Ac#nqAR@hB zpw*gsCKw6%*m@dr!vf`vSy&P+ZCREREn!0fHY{azP%4hCTb+EQ{&a-><4@$ zNb?5aqk<84p`u{}R2U-Jye3JGG~tea)?=K;Dbfu{OS;Mqi_{li{bP(EHJ}k;Uk03w zLFn_gVkcZ6j>@Wn!U?W7+k;HdmPP7N3XPF7wcSWt24Z?Ote`Jf*tc0J>Gw%rmbQh}2^v24zDOv4X z#S@h=R@Q8Vb~0CjdZ|KvBr1(BA&A66blMSsA-T>AGp|HUylu=N!G7D1RG`X8wTjwW zC}~&xtbZqGe(sW#{}Qc!_AFhr^vZ(>S2g*Y<2N{?PVXCPrHuS|Ui!bB)JdC79K1|r#Wu`$AQ>mnKQxCBrWmYZ|2l!bQlAh@PyYbZ@Xu9C4wu2)%l)SzFnqLlFvLk&s5ttU?P5 zx~P|k?NJl&#nQSe8~DIAn0M%5{<9BHn?FcYT!%j>F?X-xf4{2GUBj8^aqWG>mm!{7 zyx6q<;LjiGP*H~upgi~lJ6tR2NQ5@xL2`^4-A&y@?NBN7d4A~xb;vD66>MIV;U}+2 zKz79gWt@DKiYJL?y;;GK7gD-MNmXrzHG(0iQh%*e9we71LD(}FOtG`w!W*?=hsd$LoJbzT5DO$gA zP3oe1;)O#JV#iQr(EAqN>!uC8YN9f44Q7kRq0j%TEmoF2^#+omZVndYl%lmjP?+Ia z$&}uiyxADlPJO`GoR+^%D*`odc5q!Qnb)TUDWshqbv)}CyRf_qZcJNoa<|f3@bs!? znn>3aiEd*2ZLiA9kT$U}?Dt!vvJ3MFBT20jujWlMTw9WFdECVZ-7Ey?l8ehlx0o)> z&q)UP8^T*P{-Cm>+3unSC}UuUj2Q)*_!5iXW=-D6VLc7lH)6NX-^OpLtJD!_@g&X0 zC<3S;?I!Lv?)Fu4bPr=h=cb4upM`MruO6s&_qL4Oztzvz;-gAOJf!twGY$GX{|;kz zsb&{lsgTt%u({hR?#pG*V|(OF3$R6Pv{LMUHH}dOMRtKQ_PBdSF2(NU94Ul9y9-8) z!OB|ttn!k1tg~h9k1GDpd4q9L8+_|3e)BBXs|{8Gz8gb65gdcsM&(qP*TmEQ!?#Xs zw>h(8#6>{a>65^=0uh^6zH1-vBIkiu6)C7t3ox~PXx6psTg>b)U^j1*dhAM7O-j6N zP~=te#idxzLVYzP@2$sDSQ`sNhlC_#9U4Mw&1sN}DtzJB134ePXe8Li?BHPQHnmiN zsQ!Czkh|$mjmO+V_#}jad&**b&Jdw?vvf|+P#|;J14GFw{Rf$}C+x2z>r(RJ+TyAL z#VvPiisy#;R1@*Htd_na8#q$Z_EcT8-Y%pc78#nz_^a1ldEQIxNu58ig`qEb zwWT2)bvgm)hJg*ox`lUn$`DF#1NEhmml^L!U(LBdbl+H2-kx0Sscxej0)zw45}_n%QUaN>z`4{Z|b<{soq`Xz_*# zC+<4oZ<~pMs4Qmsk4gU z3uOkDwN3$m*SY7SjMHJuw$}G1Nh57VQo9J3eQ-sb_T7SL(bc1E&E5WGPY`#9jvkvX zpd|KFpc6Y%xX-+mfF(Xe7)2KN*gj{HRfbX?k*<(AMtu@Ppz2w?5c~OQED}e8S};4i zU9JjZe$jYI5f9E}%ME#3nhll{MsAB}s}kcl4YS)6J_JO>G2X z?Xl4^0dAmsnvogz-=Zq=F1>$QQoH-De&yJs2LA~!C$D;6PcmR(HJ^3yLNbBAZM!@|I3Y+)iqNR$%0+m}6Jv7?wiue*uYln0^MJiYq+Y zv@|0&VI0Fs4W8Qg?US`COV^SvJ?B(sCh@KNgxET{EK`F8ivb8RVy#VCt_%@`@&f9L?r`Q3Esf-z^WG)I*x=V*I zew#aJ-%}w?l zPBi(N`^V7iQ@M#WZJP8ve#r_BFx=w0t2OMz3u*UR4KwM0*h6Z$l!hqt+m`z=)iH{8Wy$pC@7RMNKl9R@JRT3NKTIANa{vhN z4OuJf>ep8Kg^Q0{zU_XM$!yFhhQ{*LeGOo2*{8FTgeWsIGGsUOo(E_lecvQJonfQ5 zA0fj3^`EO{0~?6V0YzeyAoKjf(f}2yTMwcSrkUJj>jubds!v52U)B$_m2=Bs>sV0hor*uOfD&+8877MWa#%pg7zi3Oh-3UPw43Lfqr72 zLEv-K69`NSO?@$rI84SZ)Wgcg@qup2;!t_d9}3}KUs_wb%#}QAwBNwQYu-faHRJlu z&sX93&TEXL?KH(<>$|N#K--~IGC26o^nFV5xdltOKQob}`H87*QOO2zY_+J<=t>a zrNP4ejX3fadVNUVy&^#N!3b2M(cw4k^li7Bu?H4Mmo5?>TTELrp2Y8e`CNZIX_?e* zxxbE11{Kh8&YD!T>XQi51bKTg_ben$`>SAjNjUFENA^sq4OL7S;YJqt z%#B^$3hCL?w-j#N6W%X4yoP}ksQ^;s#eV&+Z;;f_m{*PyJCpCL=99fDr(>smJUj`} zV29=K!++GOdbk`gxrmRg?w=uqAI?vJAA_)&&{9bWu|2AmhX@)j@fl2UYIkod5yEZm zlqga}m8B$#28#{Rb_jDr_0JLo@7HIBsizM$gs|#IVo`@w{QV<~!gr&%769LwT*jDR z-)cVckXSo>$-MHUZYBvKZ0)DKx=%Im-fq7YFlOQC4+mSU-x`tbT+SiNT-0G1>$np0 zvdFxLCQHhDVf~mTfodqkeXj(h<`(b)fVkZ&sdOc$Z0y?#_m-y5UxZAU&Iil?e~3F; z+g@U;mNE?VF`*n}Ag=+@%>!iVa1e{n;2sTaX-f-JmuKtsCr7pwwLu!~MOVCU3;@+V z(B;^cE$GF=51k5`Xis_bvYwl3|f)>Se#6Y?&b`-OEU| zJGy$mIdUXT$`Dx@v;3%&KhkBN%ZMw|>hU*pcrWX5rM>K;8MRtvF2sp164Hrsk50fl zW!sw#i@?h6#5Jm9wFIA?0Jcn$InODU`+vOogHrD>1KF0i4let1JIAI%A^*jhE{Bxg z@t?hYX|%XS9w6O`$tizhDOCy|@B0yJ$@n}>ko{cuDBN<3ZTZ}G*DGNCLjIY!%Ha#{ z1Y9CR;D0`#&4q?-03NQn@UgVRn_E+atV7AYhC}?^ZZV*gBi)T=Vd?6j3i^F z_w;8XqKE|irq8*_Z-!1|F!!b?B&~bY+O>1(p>-oShKOL19aZ%Z(W{eSUH#*GXAbJ( zh_JHTMp`5Shk{}}J73&nt9L9dWt4y-$x$Kcd$qqVkQltY0@&;fvLF~bxg;o!p0g4DbeJs#SM$`e=E)0Wn;V~5td%obrpNdO8Zc&O0c66wLW8P>Mms*5f7in@ zQgQJ{{+~&j%{s=Al5D`K5^%$TH$7|WJGG(NfvEM1D$vs> z8F3_P*_s|(OI=r5FNBvpu4oCu&Zp{RC+@i1&5hK@-yeEM2+n)QM9eQ!@E045vu^Q# zXh&M9hLoz|FM2c4d*rR_^wcxYe2{1|4IfM^E?LZ6fPcWK5;XG_7>u8Wr1xk`O|OK% z>q+Qdp&cAGrjCD^5?|Qxg^pcBa;p7V00lSsfIk*B4!I9XcWmABBWU|KHu>){z{b`y zMZ(RIV?A88FPeYz9D3jBx6(9yn|3*R?DXvJ;z)n5>HAf6<#eR2pN$(WcLKTe>jab= z#orNo4jz1X--*+xlSGzuB<*>nUZ`&mtHS->|CGS2IY1uX@FdKFE|ED!+p9L;9-O;< zj=aRPjz@#~@2U=NTAXU}rbDfPY0|j<&IS7eFim##7&rAUb5f6ZQ%d)|i|gZj<(QcF zLB^E*8y(wK#D_M(9_>{@djPr_0fUmNtOsJ~o)YXuL0a4c;1pc_CLB`UrlCa73x`00 zzzCp?L0q#+mK|RHiQKk7KmE%;PZ5R<4{3+$(zOA* zgLUZ~J$kPhzlnXKWKe`D>LMD(DC&ZP{8Ey%j1;USRNqxz`!>g>?ViF@9VB1S77qQ% z$Fo#;ppt-_;n@lCD#Js|cm3&kpyCdj5TkP6LKcrHoscUK{e1=>*iDD^?@63(kc5g6 zhodQz)(P2W`EEcf1)Xd3Qu!2wt&uaa47qz5!saLD-L_t5*PHe67s?@f6Y}uG951LJ zeSa8*r{5ew1$38;Vp%U$ zZ9b3%&fggZxd2SNTqY+|K|dmj*(yoi_rZVfKyZ+;C5>;$*$c zMDi3_^301g<-A14;8X8XdQJ8J>PnjGz?-88EdVzm0nLV;hBPkjW;jk13cfcmO>uEO zn)zT;Nyoc_uxG_V6KVwoX37EXqG4*__w#;j_V(>BbMYKDGJk=n+eI>eKfgI!;qS9m zUC6%|R=n&66TlfKP{cpnYZfqS;KZ-0Js!Ot`kLsv_XJ}vtkhD_j{xPW67<^zrLCep zR!egiy&ehp;Wd{)n{aHf+Bm<6%Jwyna`MMB^IfcL``|AB$@GjEQL74cElU1jfT(k? zJ(-gnQsP6>I)R7kUk@~8P=xEA-n)qBaq|{Jy6>xId>S~2ws-S2p86@D3G|nXF1t|i zP_*nA6a|+UROQ&JGdv77VX7ds>MC)6W}b=HE0}~)KH?trDf{p;k|A`ub)z1bgx+&s zBEYoOTwxLWqpevgNqU}ueJ>^IsWqzWU?9{K$ci*6ozIe`jpEFX?(XZHNg_3&+g$?C z=#(2)uu~gBMU?<3bZmdBPvaMM>rDx1a-d)Hpq&~rFira=YL&%>awXlOzDzQp14w)BJ@EDB?#c{ys+)>3WKCQS`=Mz@;e2?=pO<5h z74iw4g#MUfpmFUQk01*&9R3Miv$%j1j6h$Hzp5zQ$ACv`$->i-)rk_=kQR5FcQsvz zimzJx(>$qBSE-HqPu?bx6`>qka*K45Y5h$%_5#L>hRO%+r)O)G*0$1#V#=&Vb$#uR zRE2`AT6v1=9KTCb42@TsCG7L%Wu-9mLLvOJ%9!v_Xs${rs_+Cxpzu^w=Lf&7b0fPg z!d%QnU;kc-rc%jhk!5*@e3xYjLOXC`mQSb;zN)b-LDV%Z?lIeDesIbcg=ayBL2J$( zS~2@xuwWMCq%V#y&$V!wi!K+L7MM*H2zp!pV=on+z+q7ws0yMwXpOyz_JuCGoi*WV zViwONa++7(S-mu`1j&0$Z*9~0B!i~!wZ47Rm zTi()a(blD|jD@;YX00xClmAIWk!9mv&} zN}43RfZArdYQI(l!Ie!4K2A>cmD~h0Merp9rJoB zFUw1Ij@iY@`t)heW;&?Id~xa`S*^8gW&d*5`p1)o+Slb*kH$znjgV_1-k(USO(Qh$ zMb)z4s4t)ncR3-0K1zbY@(n}bg9NJ@T#DD0*#QVq(221DlrMiU0xfrSgcLeqToKe+ z{eK9jAmyYxvSeE`cU%v*fbce5cX1%|HAjQzl*H|2CRH^NZgH^>+4k~!j-XFrPr>wW zF-_{?Sk-!v!7T`6cQI8|d*WIA*RK{k3CtlsAfsSf$4=hn zK@9GkHQ%fh{=skGBqfwAw>fDbc8-eva}<3!WFxQxq~dl2u&jK&i&T#5ikJ`t6&ZAO zH?ljtQd`|Gmdq?nM26=*a@Pb2du<)m6R}%Pgg0J4H1Wc`xVT_H!zWEPex*!^^U8Na zd#FMAFxel5%F>1Zw|uAqvb5;LyiJwp{~_zG!=n7Y=urU?X+-H%Qo1_?1SAycF6l

&!FySpTYMj9Ntn|tQ-{oVWgp6C9<%-}mPzMQl7+H0@1&os!=#UM*-IJ<^E zDLzQU)$@l9?NR<5>Qh*`10E|yi2Kg7;p3wx&9-AQ{m%KmgqZaQ%|B6Xm%a>7ZV{LA z7}sb%T9UhtFuq<$c_rmLYkgQ-*OprxzX5#*5vT zNbmd)t=3T%F)ekKuNVaU6&aimGvI2`=b6R7&U3`oCs2W`XNBLn2e)Y679MQxy;@7Z{}S_sP6A)6{9E8>prX zn~sl{-)k1RUc~mCLHS09bG7hEX9T`+ z92#>!Xc1OZ<{)1)(_pXkvV9Tk{6wo+D}+41J|~8;zTKpKUaz(kTYK1OJHE&j2-JHv zW>bd-^Y_f$8YWtU?#;5lIof{Cxs5^9Ic_%v!eXpsAnRLn9B=H}q29;N*RYVNoWUn$ z{y86NFh-;VK&yn`5Q15-dRGEy3y!EEC%PELww^m*1Ar&)Px^IIL7|RcGAPy&R{#l+N`W?M$ue|#c1n%T%+}; zcaXqJh@}EM;5SL~5Dh>V`?-s&uiBILX9hr5^Q1{xC+WVm<=*=;gdcexD6CZ=B6faF zDEsZF{CVW>NHT3Krof<@CpIt#J^6m>L5}3;#}N5HJsh&jkNmvAJW61(VpW=ASKYw- z&!UbZ2@@n~S9g2yst|k0EA$jy4(MsJ#RG00`(nN`v&wGK{H4mKL{-)tJs#Hr5L|#~ zfMo*j0q!g_HNH*gv7v;^o%zAk?|&U-?v4~ANl0x64iyOcu8k!UWr8cXh-KR0{E3M@ z`4!OPH-pQEueD>|Xm29#G_p1>C*Ks#;lT1Iwz_PA+5}MS2Pd;ZI@qfLM+6&t(4Si~^pjBcJeZ&61Yx*R7W*t* zD#QKB}T&aYn}hJ(=}K^_~l#pn3fSfe}NbIz8y4XEtf1s?N4oH zTw&XSCzsnrsLD3|#<#_mdM7ef51mmvT9Qx(zG3^&v*LFui~hi7{EWG z->Qh1779^?x25#}Cfd>8)w;h?JyLNh)gt76ho@VthI0Q)xA_0_myxHar7nDr<`+SF z%(6Id&XJn@2Xq;aHSjJUfns{>|I7i#UV(bkpBRs_)oXNkhc0a|wexH16~y)by`Esj_EO$cEj|eOAqTzF34+KPE@eADXTjhbCMM_`_cC6r|RQi9DCc@P`t2uG9q zethj5!=Cc0BFZ<+u39DwLYFcUaU7R0_C6Vw?Q0H}>}yHA?!+F3WdGvZwSVWPW(T$u z85HEhAAFwxmcL1e17Sv@p7aj696!zzjA-}o(_P5{hf0Oc{B{yf^hmNsFaM9Xe*!x# zm_?TiR(GwfgSQ8OP{Uv;k1j0$HZUzlcx?yE7Zq_eaE*fc)7`b7BmU*?U7QEK^Bryh z;CpnB*^;ciq4^Y>^nS=BZeU=8Au?2z_D9-Kc7IAnGU&CG&^iJEbaHt1Pf{AH7}n#f z+T~1b#wsH2RI}&2K_s0vt~;WXi-S4g=w(I!J`<4z`HV!26^?nUrua$Jzh0!r3j)E% zzkVVH^k{KxpVxUCD!fTf%igc&`|#RF$yM0N_?88|5QbPY@zZ-*W_`j2tl-~3l@efDOqw$&Ezw1Rl|F^;6Bbo(W&XYQfSex$%e0 z1G&@s+nh}qDz|UsSn}oYPBzvIi&dWZiusn3s=NR5WJT`j-nYRxQ+_qa7{c&%euU8i zK$H$p{vs@R(_qgYY!i6^gI-e?HSKk*hsp+lbCb79ezd2M;0NP7FV|x7TlWuH6wgJS z{x1GXkN30XvIB)%Cfq+UX-o|Fu`)#r@M(RVR84Cy2hDBtt_O_+mVbQ?D#T|dIjl`l z+89qR*k)SPpEPYD5ahASN>~+DpAc~)lnLoE{Pyx!oZA`tUq5C@Br4SKk2qvAi%_4^9f60(`A>L~zxJCf7r% z%f96}orX1R6mauR$&fJrFPLO0i`)vRv?HrjRa1CeWd4F|;8Y{jSR7 z$&!2W9b!jGU@RNPBT@qmRkf8Bs@ONN#y1ob4h^9)@3&qIY;4Gy-XhFZUfm@RacO(m z4yO0uCM{V6KiIsx$SGt!NOg9e((z;a{1i#@hK~bbzDt8pAIWA_wOMJA{r84pkI*RT zL1^f;idDw+6B~?IV(|Vuk|f~ralk8g0F$cOKX(I@gd#1gLh{Ahcs0RDWF5)`)}5bX zh(tBJ*anDDoEIQMS%7{9$3XvsV%OLACyG2NQ9$yx^@vzP!4pMw8a`^}q<8t%455I$ z*Wr>_#gn1Li%NWT#x@fvzvxcj75Q|H5?0S&94Ed1WC%`rpve6@wbKm9AU1A+GEn6v z8|O!R@QpdX8b;Oh+vd+coUhIGpSow|(9qW!FA`ibk zGF!K3I~^rh-g#%X0zVHGKA_AC%BEd~s4FS2BjitD-`^WK<+!9g3>~A692!7>+#+E_-J1IUYiDI98~L>rAiIXs>DTu^Xh|ni>i5cvcw+#;NX*Y@2XX^Rm_X4 z0dotU0+-(N^2b`)p!=#Wa*`@fwvn=j#}?0cwW=eSf-Kf}_eBXb zm|FY3r5zk_u_*md$rLY#saN4>Qe@*D;7cdo8Y^4TjEPOP8f4W_Go{=~xQ?qicixp> z$}>RmKjMfGZc3HfXS3vEAO%f{wyJoClOZvoSkaWGzou@-;&e(ed-&nR^Bi%ZC z?6u!F+AxRQo==(rWAoc>bB>>2ZNmerMntCc9urX|S4hNIT3O$Misnzir2hxyXeG!0rclryhe zwY_;nzMIup@O0ue=-Js8o8~wRg}TuBe_O;R!(m*m@mW?x8HF7t`<1)I@dwqUrj`A1 zkJHaTP15mxV3+yuK4lDo-#+t+@fcBC)cSC0-Mq!>h1MgnH;=ykG}Emwmwv^HS$fQ8 znxMB5eVhK5pUR=ep;ro%a%J5OHO}Xvf0XFVMxM>ar+M>8se1U0uqm30OUl1W)BW_WFD`L z$j!*p8YW|3jxO#8Ac4bswBxIQTqTT3DYuwuH0(-XR3`5I(3=Od(+THSu_g)2xxmVn zz!Aop`Y6qxJQO@UiFf0BOS{{2?tQ9N=c=%a~OT*?xPZ;_8uUOnaZNdY$+Xi$MFOsg*vd%?h zgL+9Bq^^)?AvRJ%Tj^O~s@-%Kc+AE{=ZNQZNM`|aF+Rk}U(!~52;=*hQo8enh~{vh zIVnB+WZ*V%k+w~gt1b8-Pv>(8%5dK}`iBM^#T=@p%V1P1B{^U}4GU*lb7xYY)Sz`2 ztGqfKc=IuHrDfMO?~p<|Rq#7+H_6(4t@z`MV-{~!WS#o-ze1l&@bkv1%j8rm%8MwM zaqQy$d3FodXWq8GUX~~%gkPT!6XGV(y#L&@3_br&AekaJIw{{&x#IBR0HLDm^V8wW zw?NYjui4;QIGDre#+y%nQg>h(iJhgM_AqJ3wqa09{3w0M-?B55@(&br`8aiEJTZ<(Hg+WS?L;L+WmP|Z_AqjY{P#6*fhB__4SpAt- zKZ%}sEy8TafW4NSY^pfhoFEp~J-@h&oF+)9zql`kWE5PXMGk&0nEjH?r1xI({Rz!A z&BxUeZ+;lcdA;q0qvhH77T>dXjFC$3V18g`f;p!AXBgXa%RaM!Z>?FOoMY>@JKu!< z0>Y6I?&kA@lQn={gil$i#(JTZ7}AB5u;4@fa+@c$a?58d1T%KYz@-f$3h?OqH_vpsnEYP<7O*BWOhr>mJK{ONLzHwgz+jk#TDZ@{?iq=w)#Yq?t z8RGspZw8s0^(;Pt`@ha_7N2Yf0OOPzu^t~wbPc%a(RhwXB?{#Ip@nt8vwc&Uaep=k zJ!<3&fJP~def-+E4I4)#rOP#cd2rTCUxa;DdsS zD3xWlUTB)`a)g<@1_C|@k;K?9HIM$o>W^_@UmFKaaSlGApNZE)v0+UUOPuqS6A8k8 z+Ap45vd#3V`z?I6gsQZD*4wQ59G7Jl$6BDE+AqAh`qkF_e_+ckZNXh-4>o|jETw)y z@`1cL0%9%L8JT!|7iR^ma&P~SRc`B|)^pLcbG)nmJZX^#tZ$SA1cBb!r?K*dhO9Skg0FHd(o9E)ZPRRxFfAE}T z_lbe0g*eaBq~N>jsy*D_lAfCdmH0sJf!P9gE_ryNmBpS>1{vR8rT)ueY`V@EYiMVC z(O#1W=9d74K8uYgdQqTJl^UuVli|w0v3`okHYF+UmYlBQjEy64rY9#dzV~{%&4DQET zMk5XGZ>rj)+~8zDK%x0&I6a4F#QPG|l$(yN~X1M$Nt zkFh39Rq`=I=4k^$gB?TO$0uDLkKQbns%u2yI|&E(pC#+ga3I(#rZ!osf8)X@LT9Ep z4J}?dd#O5FHY=eR>}Qqp%;%vf7TMLxh1^P+Ul3@J?Ef*a3CnqA6b>mmlc90iI zy!U@=Ev*kQ3l6X1B*6U81j&fPS1tOu2<6P1q@|%;VPQL>{am!`=D2hgr;2*UmGo*8 zIx)NX{zDU9M6p}wCQ9#52`Lu{`iq)EoU5>SSqn6Eq^*Vod=V2F( zS=EE#BZ67uwkA4~$$)=<=J+cV!5E>EUX>9{wGsET&iCYUV-lLV*;uVG=LCN}`kTH1 zk@Zy})##JxXDWX6Vv7lYiSET*2cmP6IJ_Uj4y}W|iYHM8(L_(1SUY@tn5MVtOh>#( zZ4TO@0DPY>pxsMfHD()@l3F%16yRZhJhigVl32QHm&FE}siprkQ|Cqcf}c)#?>yY_ z;FE*_VtBImn&;L3Puwvu?TkcAh$#lqePCzk##^&=-@$?MIJTy3hq-C#yuh&AQP6@l zE!{VI+!(CLE*#e|fr8q}jG1l7kVoseEP`-uxrm;)X8wJqaOfT-&TnvQTmU%s=W%t}porf5UmDao%~ zN~oiVh8Uzv16`j=eZtih#S((mVzA0P zat>4P&}mu-?$8koBi;hRpACB8wpx}7%Orm-#hmq$QL zmBd`pM)u!{f(Fzu@O+u1_gR2*jSejF4*AvabFaN#ujgJTlJLh$PI(cFFIY6_J>Q>v zSlCPFh3P{rRYd2fb3zTDn7QNzn1$H8U-68k#N2R~g;S~Qs`c`Amc~Ppil0uW1e%5X zd4c|m?0sQtUezaiF(jvMtb#8|CSl)Z>*9p$UA|*hAbHdh8ewgU?F5)kU|VTUxz%bM zCcf{tmg8qtYmm2FV;T&5H8`JGX)DWJ1a(y*ypUN zsW$8?muK;ol$gV?{g=e*CyXbA?+e?1fyRkFUHvKGa1Xk3n)aUGI<(A{xKi}TyX%M+ zYD;7?UYqu^lDq;ZZkVKUh&i9I!vE44Bc9KtwZtrn zvJP{e6j%6;x4hU!aj&qms&(x^G2(LEiR%fL_6!Z6-hNzeypFgd z*qYCdb)!UK*mEXiTr0lN;$S_K6Z6io+WgkKmRI1c`*z>GH>BkYgc+W-ldL=e(bMWe z$bE$8XanMo;4$cB!*RGv@BhK!D*%V9#+U?t{&!+f2qCrd#?}dI6KSA}Y&!y2a)qc|= zGx&BJi&`v^HZtuv>yK)i54}3W0%zr3zpl{to5m%wjKRoO$Bfeg7(P$2FaLvsG@LdIw*aP>n7hggZ-NA&=I&Idqs#5PCix=rU#BHY7BB#Q6rL z6-*nh&E|7&Zs2icoD>0TO820CcDQtD^2K6=&$@pI)`2a@_N1GS&(_$a+gH4gU2~ez z$eDYe=EI8@Qpeh6R(x(ZFP$4+bfPqo`--BBTZQWxnO{r1@h{YTNo22JqH(Egd=>g}O9)lur$v&fj9X>e({2U-+0v~?j0*Hy=>~V4;=79S z^)>7>gB+A5lsZ!TJs%wuSAp76a~3XD$l%+}aMXQNs7YQi>@m*X8eFAvJMzcW>;3fD znE@wC_|gF)MtB76UI!g^@4;`>+w%x4x{TVjl=wqrP9 zVsrh&2J`S>e;v}V<`zYXJ_gXsA+AY32n}~W>OFfq5I}uAIYsC5U6F7}vBbTe{dxWKO-&9nN++VBVo~B4y|uZUIUW*t6-DVz z$wX_2LA57bo%QI*QJ^BukG8h7CCuI4)H99VOtLIoo_4Qbi1X1q=4llhq)fD9GM;XD zje60`{sp!114Rc{T1ZCc2-nW_;tIykR*am1t@Y^K2b24;pD$iN@DZ7<8{FM1vz;ny+jqmkl>IxHOke%hPm@3l4^TMVHdabC2wOJ3h_CL zIUkpoJ%G@abM#*w@R>S)h$;-c>A$<)k}!8sPx00cZ=Jp2XuY;FM0%gZBY%4(?i*s^ zIWwJCoK&dN1M@r%b;v+Mu*>+E*!a@i!>J|Ln5C3;C)(P{nZn`E>|HrHzMwL7cuwE+xO0pr=@+fzU+3ag zQM(==uljpOZy`Ny{)(py6l;9JI+uKB!W&wQZ_%S<%j}nKluhn6A!rI8VmVnRB9<;9 z&~|PNEQ9y4u}i;X!tisR>AJ=X?KGzDW9w&-EM(Sq?sqMXsmsE4! zN3FYQA29gBs$j+KbN}yAMc|v`EQHz=h;j&kEf~}KQdZZ9;@Y_ zRF%%a1cyZ+>TMCB3j*?|ed|m0ja_LNXRj_-FP?HNgoSOfBr)q2xgUIxu2*5#m62rZ zaD+NGfVY}XD1|HSPb!sUqy#TzSiF63go@T%TUl|JGYN#EWC&^0MlD>Q7u?r8B9gf$;ItyTk z{c65uIpwJ}TJ-^1sc5RhdC#J#!2#P|<0%rJ%TkD0$YaCPgLbP#^@c1R&v5l&W^)#~ z6oOzXRZYI&&+S%!xn-yO7Y+CFLK?b|B8fpgL+&(wi)EK&*Yk3Dg=hhxS|CNXkRGY} z%ZG#Vm~T}sk8#hUfNLQBiemI)rqz0o@k44&<_t!Fz$X@yIQ zgY_ImC}n(Q(@#>>9HRRPvnGZ^H@>m5I+Nc~*E$6m<$a-^NRCUL?!OoJKWc|YaMbxI zN$oQgyAW3W0snZ_YMR=BK<-&ejwYd8ogFN6FQ94#zd4w0^7DO0K?&&#^8fV`yOH zVm^s6%4T%JhwtY3rMz8nGfx$Ec_-via^;hDO?f9WsoB-Oz1^3B&9~IEXsg}JI%VX7 zezt?b1}X-l@0zw>=aT#pJ{o_$^l{58<*G^&$7Ge2dSvY;&@cDcD>?XUT+H0s21fM~ zo{hTe*3Hn?>K!_#UdP;|^;k1#ZP!BETSgU>1w~{oP9el-((}(~q&CyQ}nN3BS z8VP@~?&jlb@&$Rms|rM9Hy$BRfICQe_Qq{}Js&yR9#jG9@ONp#%88WAS~7=;?BqUb z%^)@1z7QyH8c16=CcaL+#@gj*z<{uQtUE^&THn73y*bM{pg8oYYKsb6I9}iO6F^on0joi#0M-FkQ^MWF7i6m+5@6DS! zb)TmW->)~dFYYsUw=b4TVs0HUxU;!>c0Sp%{Yj)Ju}>b!v`U_H{_l^!O`V4&Vx9j2 z5;j>qz#p1yzw~~-xMpvnMyrha>hNhz8TXC@b?Y>LqBsAmILn;@@r@?#r+#B{snza` zSn3Z*qRE*nO`JGou}r>YkK`-1hayohQNz{pBXZKB@*^J|VOP!ibB3QylA1Ott&Ey+ zq1*FWkaf2_2&o1i6KRb{gIf8KL@^-~DZUSYZ=FpQN4yuw3kUD8D27KMPl>s-q=HSg zI?WBMQOLP}+ng~v5!%G^=c9`b1nBp3ps@yUMcjj8%sbkuSkg@ja;ZPO6nGDvEn(9` zTIcq#??bTO?=!I{s8w6sSezs;Bh4ACEGS^m8Pv5cfbEdZh8`2sPp402|N1GhQawYl zf^21DcaLn<;}!u@m(~8do)5vojECZ5Rn$4Q#V>8xt?-sdxDi2{n%z-mDf6-{lD2;- z($F=v4lR3fFLld@yC`bi^xR~Mj}JsK1DfT}=`ykztD9WR%2L}#|2cY-aj(M`R(kdA z-j!2o;Ou=3ejpD}?VQM~>9wR`x|FBkHVMv6U*D4VH_&bk@$Vp)FLjCJ`|Z%ZRELRA z6{)Qs?{6Sh9;iV6bU`lQSKGVt)TJ509oO@MAu?$s18reb%5t39*QQwhHgwy2Oq7Ok z$TAytrp>pG95S7f_3qnm(nDhXx(9}%Iu#W3pY^y%Rw%f(>f71z9CGmo5MYF@zrtAP z8oxBt=%_i@DV57-BWJE3k{#6Ss5x^|?8n6j`=k+PNfCT5<#hTicweq3v;6a{kt{h{ znD-VqtS^>dr~5R2>|lI<(wOnbWK+KW%}I{U=enBl`3cQN~ zM>?&OnYTAk>=lqWP-A4GuVp?2$dLAgML0tBkLez`G=jwHuhVZRO3DB8JZB3J`) zu9hr!A1Llv_O9gSC_RnlSt37|yGBW@1h$aT+N`3gS(rF5I9gRptJ>PI(D3ev^j7Om zS+ZXCRxdrMVH_=enewxfUm$ZD_dCRY{{^@C+j7{h8d}&w>d6JK!~xLDsJ-UxBAIe_nZ8#U zIN%5GZdjit_w*U0Pam>YDHzxCbv%tcm~SnkkY3sAqTt=K)zLch&N*yCaixC(b_S0t zB455dv7Qod-4x$og30b~40=Z)lg<;qn^wgl@0-@m?-eib>~45zDP7C;0)2CiJ&y*u z+HU!LjEJp!IIGSj4Ow;)22Sc9-4%fy!rx^Z-LN=>nA%nL=>?B;BD+XQCx3K?dLBAbFJ-YVz=|0F__N~k+ z6gUs>yU1-H$66ZTEEK|QAHj{_6K(bb26Y*uAT<`VPWY_|*49upoJ9sADG!BH?r*wh z-dRN zRP*Y^F>6=8FqRClLZ`t$g#jU*pk&c=vH4Q`jcdxEC{;wyO?|Y5g~zxs7G`^Kq!|sG z`mzp@7CxnK$I}OuBps*e5K9#@o9hwhl{~XHB9VX>ONJy}AY>Xh^%T5b(51ni>*e_d z;F1w$hTc+txoF8q0aeiF+4}OOmoFd@qkEvzg@Q^q%CK@ScwBUZC-qefo506AF1dv5 zIS%*v#OOO#jvoU(i%|&^Of=h^e6GdF9$SkmYWmXpH;(tG?rqeVp)zc4bwDPe+~9) zM0n~-^7;xY{faLqJz!xN?VFzTkn{?y@TlE>UO@zS$Gwdzh z$Jc-=LkmZf!2m1t2C z(#z~Ps!CIxkjEfo8OH=pNWxEda)@SZ)KV;xa+!EZ6(XiHF)c7&@EWpX2?CsFs8u~P zWcB`61uvOFF91n5Mup1DR_4HQhd->cPgK?lEAo;Va0a4VUI1nP)E=vAo#pQink z5Q8wM>)@z=hk^U>wdZKTFi4m`Y*OX}{@i4~>t~ny=||B}$rdn-Qm)os_@EJjGSKVw z%~0wdAr5npTxWFG;f0i83e8v9fU`GAK@UP7eu!oSD@G$LYWYS-(tY(}ZzXM$U0Bh? z{fFip+(`a$=kPOBHX(dKN%ajB^0r!At+sl~If)x%T{{axk2iIm$@mO@o$|?NsK`ab zvt2cz|F&sVV2Wqt1>wBT#FjvLu3_MkC{^b(l0o-`r($OlZEvbfstw!OMgn?-K8WMK z7dsh?W{RamC!jv3dSswZL>TD&q3;`crATViibK17?wUzeJ6$OZ``*%8<#IoAr?==U zsmLH1$hdC4EJ*y2@i^A9vQDth`zgF^GvmuDy7ugPLD9A)(wX##*TX1;=p5yW z_s-LvpS4p4HzVjPzWj%j;fZ@Ro&CcJbe;V!h=|Kj#(1Sw1%WC)hPj(17aK3xRfY?9o=VO=-hLU_ zrgzYL&No+OS!v3?{Ma8nUW|=-uVpAe9*C~ZO>AnOj%jHgj;zy;uBmfXc4`@xf?)bE z!$4+Xtf`?##4?!XS>#l55vLlwW%7f$#^8VHq|K@{iR+0Knf+9`ng%R0aCuUTwYF}m zvA%^JRvBn2_qNag3#ArO8)UQcn%n&M238XMz8Kr5EP6 z-RqxM8K8OZ%j&n4+S;mZ4SDzQ+}&U+Kh$1#Z1zK7auqe_jd#Sf=&`{A@Ir%KGcUKo zWS8jY!k^V@k=G3I+vofrgrUI$oyLj`NqdwG&)^q|ov_V`r0ne(Gp27g>MhJszQ9}`mb zt8dn4u*yjA@o(zLgv!T#bbh&L78EmDlpMvW$H%`-`V3)ES+y^t=Vmm_@~-%N@U{VA z0-A+@KVK0n;v4^{UG;>I=@q`fRDoeu`M4O0cX`#50()2!D#3ms<+wlCKTpvR$#h0z z-Pp0s%!rPa+s)LEG#7muG<6?uKn%qi5}L5_Fpz%R`0@5=>ZrDyfq@^-MDLY}_X=~( zaS0Qtkwu4)Ohvq^T@c>V?-u&$6{t;UX$fTsD1$rE3_&279V~KIp7n<+HseV-d0vh< zWkMs1Rt%NI%Gh-bHDr*=n%d_7icfwX^@;9IG~_}Z7@#?8WNhK7eXuBYt{~PMW4XI& zk5s;Lrbs9C=`0IYW-D`Iw1GXo;^%B|N}7((R%tr2y5g1vkv!M}H3V*sxVr(gss3e! z1mAqR0m)p*4VD{c&xMaP*2hTM&*6+cG{8$B(48y#L<^xbaOirisp{s`uUqkgT}Tvr z%aBWspfw{emh*A#%^lq_tnBH4Bu>Ydrpno}I{o3BRqdAfa@beftcr0Bb2$FSieOGE zw$aPLuvL8?#UZV1+9I?aOkdjG{LGvQ{-J6ljLyGT-0a&#;87ij8)@sNAsG2r00>UF zARR7@TD!(y!3!YDk&~@E|`@ zy$dn~YQ;IIUT5x&43;=Nx*8OhMtQQxTfjKqO>Jiu1(_A9$bSBbZvE<+wIsY^AHVSc zNv2jD4!q2*C~BM=OG`Pm^WdKgutYPmp9*5 z9N&gnIvb2JF7USi{thve7P{)uPqk~B5Lob6nNf8Z?4&mEKy47ub@7JbsaOXzxjQ}4$%hUT}Wt( z_C!Al%-Uh(p(l}=eUM3uZmWJha>qnDMBj`N!!-=~p1~sXiVkr3??qc-b5JVZ>$|S2 zv%bqKg*MZW7}iN$*>sU9=qZ-zv4^ZhRAW-l!T~=PjkNknyF^GVZ?dBhe^?cLn%BHe zGw$QdhVP=+GTsO`A9xPCDZ2ca?i5l*>9$u9y_WGZR+fFqmL3cbY}+E*mis9HT6qxVnC@n$_h;z81Ga4wkcRc-X?E9E+O(`<-i+|{z+2mczXv_F?S zEOkf6BZ5@><$X8EJ77(rke#K^d!$(RIC@;3@w)Q|JgZSi*qV=T5W#mfQVtIjG+>1Pdho{-#p!%26ouc=m89Or z`jizJs%OjZ`2KhsHVt2_Rbj6U|Q}=oMftD1QANj4#}L%?|d7m z)l;yAdD6y|JS<9Q>vqhLuQbaJ2ZPg3;E4~1=O!2>(Gg#MH2X*>vfm;tH@3=zOHSID z=p!7y@=%Nfa3zhwSg-SmG}F=$>Y6(DQMB(G^}Jij40zf5uk8dK-i_YZ+wSw2l@`2} zRCUmHgJim^HXzpx(#Kn2Z!HoyZl%YNxcTTh8H@_EZD+s^dvLy-U1ITBW^#_ZkBcGw zT3Gjk?#zwVW*@;Q-0S{z{&Mz?{Y{GCYJ1vCw1?qB?zUsFP3xE*!U^_0MHd(4sr?Jn zj#rSyc*d0yi1hWp3+X-21Q=(X~A;8oMg z0>j%M!fqBD-%8^jJO?g~eSaFv@q>q=K=e-%{2>z-X~BuzqW(s%o3rN%E#4aUJP$tY zf#qUgIC2i^y>QU4rU1u8Ms20RAUrAa-XvzppNWXK^Zw=V_CrA;s%PLNczIq4pF|2T zggOt_3pTlCL+S2K<(h1l)P?uiY03U|sIwP*F;Fy`a!4#DII={~N#M+o51bRCwx~_} zmg_SVLpk^*(@K5kre06Dr0@I$a{~?FOmfwoTWp<+IBBB98D$Q2fibyV71eal^uLkn zO5Yi~@$x#4!jK?@&sdOjz~=X31cS0(TyUUOsc-8AEjY4vqBvH;e)0&McRD8HC)Hwf zlbcUb05#ZNR*w*BHWMmOY=X?ukYsdF*v-qgs8p5mpYC=Prm&PMD@F~H5o?;n_d7+F zqYe)Y=hL?}aXr%6LCd@kvBrNMAh?44h@CA=n{E(~ zKwrUIegFx4zw7yJ2-P0jKK0VUaILF5K~7r3w)w|~XO>7c@YS2iBc+eQvfV$HFt8W; zRI&WpD4}}@J?3T{203fg;}BZbeT>u`#j$>VaSYN@2T%IZtC6>Oi};NWnzoLcJ4zdi zCcdrd6aknWQiTr}wbsIG=0m-d;aO6cV1z8TfH{uKy5ZPeoDflX$5Alwtu_^@rbeALLCYFuXFwWGlcDTP&p}+d!h@(K1eZh-acHm9Y zx5pL?C0v%yp5p%dCAyBXzv@S@iJ%88pAkm|j#RmLXuh`IhV!N4~;`e|WFTOzx2xg#c#`QdX2U5ryJ8Pu-X2?2Wo8q}KONgLLmp!h6mDE?-?ICON`9p`Gp}Z&>vK6C9QynMeHj9Uj^IZ!85k-mp z9VO+_^B9z;7r2Pq3pj4RW_urRaX;j3e~=OWKuLX6p5tQ^gh^SRI_mh&K!QI5W%t^h zRI*hRJ=?lLe%yY}@?ngaMRehIEW-|JrTlocP*LOdQBZGkIZzPG+)5M@*i>DEOLHki zOR7pVmX~g9vdLj6m-VaY&-hG&oqA>|igK~9D>_|s0U4_$h5gjHT62N-xR?=oes34n zAErog(KljFM6_S4pJ#_#ml!&zva-U;p847mv{>0%mE$SdKiW6xC0_(D9M@k|GrnF- zqJ~$;P*5EQ)~60%a|y{H#vsR5(Wm!SF>+uN7~;#nUMfvkO?FZ*zJo+oT{-J4`gAxU zaN%A4B>C3owEsXIc8lWZ zczWjLxWeX=lTRYeRNfSlb+|IQ-!S;y%TeIy4_`xVuL@*a@xXx!T??_PLV-v&IN?A{ z1!E>6;ehG`IsCye=$`Y!o=tcOD^|<>)dRf6dqu*$1gQEuK-JU9B$*yJx1Py^A!9)g zX;%|*)MCU}0~_I27Q_nTqUcIoo~}!Kzse4VF^O9qg)Kt7yIQgN&xH51?ROc}HWD-; zaa0kRz4)?KH_dr0A>KI?Bydi{{0vU0v`y@IKOH~<55nQ)3Z2~<6k4&olREV(NDUJk z!ovlk?;5fe1Bdtp(feig=#+6YVyI^^c0TbQ+JXrrokNr|U3-?N$x(&^KX@0eex5Oa zs=9bS!uyW0qJ?k#b;Z3K{t9zN{O^XY7IA(E%;w^7Jw}--LVHVA?91zlg6W}}fNpZF z9?FX7S#U3K7y7*K5_Va*YiMb?7g`tYCA}ZLr0IMAM`UafCSw2BvCdA$P zsgv(bm;rI`!x~^ThF8u8O~r79y$c`AiKFLCd3+I%ij}uljhpGK@t~K2-}MjY8O@_j2s(WO1vE!DZf`&beN=-6jm|YLk0sv( zvZ!O;=lGPqfmOI^QQ4tupo|lfx?w$i>2RrjXKvrHhjPGtUN~Um^?zPF1t6p7us2SxE_)e@Scd4-~?pH&?gjSt+? z8Qw19F*g_BX}N)j+0Llt25wEe6$Q%tr@&G-)7n=062To_yvRQ+Ml2h6_rKw2dk61K zpgv9EK9)MIyUflsc$S1~X05i8w@sb4O&hOHUkYmC{Am7IAbOAd7tMaZLk!)1zb_bQ z)9~d9^)fO-wP{K+ZF@-fw~*`P`7yvHlBIeOy!LaTZ?Q+N2i(FFs0i@A@zkf!royJz z6E8aAtrXm45p@tCND((k?RbB|udm`1rJk^M$9r!~6r3m)^KB0AheAdzZfcO?yE}7P zmeD)oRRzf3O5OUsuJyz+BW;mLUu`5q|8ikIasTrBdcFg3jbgNrleE;%${4k8iF#s7 ze@+J7q_*E_P%^)B^yDus%q1Kr4H4f;JX z>HGP}%y+Z3oFN@NM5mn@y4R@u5!;i)QR29{#`xX&tNBNZ*fNAB9r=`j&9z#wW@Hs% zv6E+$S2q`R$7NBNCsrtwH_s*>Mkq0SklV<=0HEhNtwSi`Sd2Ubdt^WOES1Mtyl67T zje9qrXjnE5)r}}%ke(rk}?|HlcRfxF0wr(VXzP13bU)d53w%5Mt3}ffRLd~iZ zzKFLeig^Zeof4m?j(+Md>>9DL8S>3RRnqjOe?Oj&*4!3q`VO+U{eN^aqPcPS{|sbN^cHUkI2y$p^phkpz(rC*K`rOL!ojvWO|XlOuX zpRM9)Td)~P=)0g%k)?Jk_mq+`+WBYzN%OWoD5ILdRLWJjF)m@9Q-a+Xe}r0m3+>7$JE%K7uQgWjY5764pDWQpn zY!+pvGsTZK;MH>FB_ONDcXFv?;Jz!pFj2$6-LdFtFCnD1e>xDTdCT{~z@-vz%t5y- zRL-GQOFyxWEc~T`EFT2MV9wZ!O6oFY-Uj**pdxpSTq~sb-7ShxU7K>tX|4+<3f$cr zNV76}fC2Z?FK+}rG=r`bQSsDK$X510riv?T`HV-A^ zvu7Z#M9?<#uu(28e`ER&{Zo`6U+VM}AeGymCJ^aG9Q*}MhceJ~U`k4(5yOvWL)}W< z?;WRG{F-qT$n4VsEYtaH0>PdRNuA}lU*9fT%r=S;$kkPIxe8e^4w%1g={$Cz4IK859S5mX3xLsJYSapVi%~iany0xqekle`6xRnuVdmtf{ z&H_1IHOP?N4{wEr1JLb>H_MVnhFbRfwsUR zBiGok5tdgC2@P_(`A>~=h)DA0^NF-0=qVHlMrQAe2}wytW&wO4ZHfvE?%MC@go~Nt z-&F;&Q0EgH1yAq_mFM~w15z1Lon(+7KmrrO_dgW^Xp*}X!2sw~-Clg3jj;eAM6k2t z`&P+iUgq3e*H*gcVbi;X@POtU+y*BokJJlE)kz?PrPiwPQrp9;@>+CtdZQB`Fns?b zgIw(Pw!+gg2*0gH7WlPw>H}rd9o_=~yA#rjLi|(c^F#T_`&PfYyyrQh_#>&@TIPdo z>f8nXuMR)m@rh+E#veN9Xv`)x`ug*jrFi4*x~MjN(fM0N#m9d)8SXcNTtehXMG;2$ zMN3`dD_QfQ1PyQHSQFFK^QzB(yry<7-Bx;CA8aX6qgZq=EfiL#svT7j1YUwqqe4;y zbXJ#p3?lz`q(9SIk^KIV6CPUu3^yYG3N}UyPx5JZTM}{@8(PNsD+xVq<_=eP=3%JS ztx1iZAVph@eeWRutFnZ-)?$vHFCoEor%fbtf|1|Eb2749a(uL$0T*G%ta zIM*$wriPT&t?Vt;KMZB&;E|OI*F*iPF4ht2hcq3vH`TO?n|0Fw-S{FlvXRf~D^&SB!O{%~p;#PR~=5}c#-Bn>1tjI9$}`GrJ)O zhX(og2Y%Y76_+t&yW!fA1(tI%C!*XwJF+@8{flNGXfGOl3w9-EaCDfI2p$xQKHh)+ zxyAY$6bUl3Fv1Yv3t=HWII~UjOZxH6?#0ln%H9a?$I?sI%0~Z!_d|GAUzX?H@r2W9 z*2n3@`-F=3J#G7Yc*Wh@%gyG=!^hK|td4it#ryf$)9k$UZNTJb^sOR3qdfQu}OYU9vGh4+*`_mJ1H?8MsZiq`A(`Hk(u%U1iMhplxgtN-C6@!>u0(aht4=SAA%X0D@r zPn2N(`dz2ve5LOdN9V5Ncm(ijaiRY)9d%E~d*0zS;2O*Oa#`%{_V!-q`8IOV`x);2 zJ?#ea(z^_jAoCW-^I_-l60`f`;ojtG|Ez_e5U*8HUotCkSI8Hs- zRy%(^-rDQEruo~|*hyCZi}h_GZ^^;0HhB^sQ1h#Xq^lN54cQ87?GITV-uaE!=?&gi zL~HN&^A&>4qE77xn(HbbDvrfxZ#~ls-v=K1o-|*a z)SjQUJYK$mI!bMb&_5qmCxP3{NY4s=-_Lrh_;{~?eX-dCv}5|7nNk1;e@JYD>;eLI zXy#Z#1ES1y2j6y~HjlS7+5F6ed>EO+DSAR39)e*+;A21=dB{o&hT;2#urp&BU$T2S zQ-g2#G?&&PiLeHLLMVGxeq2)F9?a@ox+=-&|))70@Zbi`#lLo4VgzZ$$X2S zp_S+z5XaZKJ);?Cro(1ftqgYl8+@Ar&6%xldDMkdGm?zrUO2sfF9qS!w;3owahcN| zofqJ9u1{#-+ySi#f$zh&WZbixE=GPL`Oedgd!PsaXI&e5A6#)ZoaCj+XN&n z7$U!}1K~r}i}sH(VcXkpG#XKEyT$6dic2c8@F@D_eIurX-{pj4+S2}D_zn=#53H8X zP@NKLbG4*CcuGye6J>?^7~g)kByNG%L@jNND=X_Uo2c#qrict(BvgCe=qHJpC0N~F z4KQUC1l(z6i-+0@aTkJ^)2RlMi?U{-mvb!t2lXn{GYA^327RxZ@mky#F3OF^d6Um5 zK6(C-Sv)SQia$tYG+DDC-9HxTi7bxoy4EDoI~kJ&j6A&T_LlwkDmy$I?k+msKI9@QpaL*|JW$z6b^6sq|;OQ`Ht}z`)2#kjk(n% zz5rDS0oEm`MJj=4DpoAdkeRYBZwco{4km)yHa2uO+m6A)8wiGCmcDX{|NAqpA74F^ z(`1vw7WO7PrvfFI2!3_uF(kQhLB=S1avNzEz9!-*{b2jm@D`9vRY7@=WVlq<*c)Gc zxrHiJy>JLqf~!Wx?0>qb4&uESgb_ zkVyCpD$*an0WD6N`{EC#r@(5cEtS$e*Sd0ki7sFz-?rj3;5sdY5=dO>!GPcKoz^=I z8!eE7G#x)|(EcnQBnYRPDw%_jst_j*e=R4xvQwmx%VIOTngH?Q%^-S7_yDC4D9z($UepB7N-I2JZO?w_!wYcEQt3@0fVr{^KkKV9A8O&UcF zFnogvqgzxOwZPDvg>)+GD_M~yj0pUr3XQ@z1GBnWLZ|oJ%9OoyGnOHR**bvZMF}^5 zE3cVLB0!YKQjU#vW%sgjac9(h}x z*&?&1~v-M`VNeScrXK(4|XPZa2IVp{iCr!(DX31O1sUNBp5QGP=*Oo`yQ z3v`~5I3T;gf$7zBtqQiX^|TlC1XE85H$Qv)D)>%ny~d@!#1qM|dFzD5X-uD2p#ks) zYBtT*Pz^M(j>XLJG@SPF)37Kj~i^oEe@ninClm z6Sk!N{OFFb=2_LX*@8GF6C#_qu5i&KJS}Clt}FP(EF3*3;@=JigPao=!8@ zitb~MsHFc40>r9s8XQ_Xk+;~YMOi*3{zv9naapa8NfQZp7KifzBA&P7m@X^UunK|> z1bzU>4M*g5^U`z^qM(lU^Nf&l@r(&^Cl}TOfMPbrQwBC*7a5mHe{k;8wLaamS4;zh zb?_{rXMVP-iY|Gln2d>;g*fLym-Iwm6U9?e=qa$w1D1mB)0GLyFHKt^mC)E`dG*RE zTPWaNkDEH@nRH5tKb2+Cnwi+RlmewRri|Ho- zZaCJSSx<6XRaSmJADHJZO9s}fJ+fjsa9xD14M7V=hYjtyFsvM+NftB{0@}l3I90qxi{5; zRnxNE;1Om+Tq8zvbR|UH1=o^(ApRg_6J<4HdnR<>UHmzlsg4ZQWMg|a@Mh*v-Iz#~ z7ibC7xLw2k8h4O<_H#sjpJmFSzvfkhEzg)*|KV1-Qt>{T{JBQ*VB!4y3Y2^$ zGqL%MiT;r;OOCL8aX78tQyxU9wdI)U!5@U!+4vJW0Gh2;nq}xJb5nSy0TO0Ub;Hy_ zJP+6pY{WB~hC8jG2PKI3Y=P|cWtpvbJ*2#{gL7J*6 ziiRn>PgO)fY~Riv*mebUm$iSNFb<@29$Uip2;ecP0E% zhajB0(^ZPGuaTL}l*3#qwF#(;Bd0-|#ams{=FF@Ax+D?=+Td}}hQ8gj@n&Cti~(I$ zp>)*DPfyi%Dmn*{5BxB= znb*e%r%~6~H_uSp^k5SH^bvvV6nL=t;v7mSiGa57q@gGL%UN8aJeg7hSf^DrFjdB3 zbPn)%iw?U-;9exW@1eJS$1O8(>kyS{728^}=G2~dntbS1>LnBTPK6=DC9!(YlG~ze z&4cN{qWE;D)X}efocXbOmR8p3S?M(zOzNheT{zQ!G1oUBnt+00qJh1oBXIrM;FtVo z#m16VMr*!QP}qFEeS)&bzgl?fLcenU<`^p6E>5rAlA(gefTHe9qjUp`08ezGbI)?F z4ln%Dupj}KUYjTLFC?k4;rxuSE67VU!2(`$!qgUdqwl92k)cg#&!*6{etx zA&Z=Q3-NzjYFO%-UCwkH*^aDVMQ}%VI~cFx<^WGsm8vAet+8yOMB!z6WlCLQ!DO;y z$oko}tV4+}Kg|B1hq&r+OC<5xx&m7ti0@ZsWm%ykq(+U1rcS3)dqPL!YydiFUG!;V zg0Bf2KBvjryDOPcEtkU`5Jw0i4c>#UIL~mYk`Gyfm9y8Y@t0cWrm@0s$y~M6UIS~* zeu}^$VTvvu?3T~c_7$d3v@)6b3EE%`0UMKvU0v`vySx<(mgD)D-v1ZA-!#?JvaVi& zNR#H%w&B3cIlfFAuYw}c#7%{EIwT-?9v|7vu&(vJunW@1xJv)|I@f9o>IOXDAw$PQ zD-%}7)KZLV#kfrDOB;X!ntoYuW}HVxz|>Gnr6yo3LTCgN;tHH5qScum^65?UpaPLJZ;Asts|~} zR-I1vo7epQ|92SH%(fgv-Zt}_i1bsMUHUCQ*)cR0Q{<&3dy&ZGy~rPEX=U&HJ-{pA z0AV&yFte^C zg*^mcI`^?@WVV*`m=Ep3z9B7D!b+~&e|CBjr0lQ`qsCqBd|%r*sj1rXWfMgASlsu; zjoG-D-WvEk3(IS)uQHYW;_6|*1sI^Cu^x;qOSd!^X5Al{Y>oo^X%|m>2HzZjiq_rEURy z7?ZE^1xHeOsKeeRa0T`NZWJ`MPVy8z9h-|fp&_Z2N*PZ)W9+>CV z)4ydp|5U(d6F5}L?oWj<^hYm&c-|n4Kq3CKan$-k{VtxLw2zm{0w)YQ^Nn#h`MjJ) z@nwIJwQ7V~1amxfG^+_0V&;N@oe=8GIU|=RRVOGjCr?M9<5s!;LHoaJA)k-wMGR}S z{1>bhTFOb({`$gUy)%dJMGN>=e~702(#GQI)(ZZ~k>7`fZJXE}@0cMx}awz zj#JFz)R^D1H<}|lQK3*R6?Yuza=*%U4I&f>&E$IWGi{B|Q%+PNnL}(mtJ!|s-~RXf z5f;0}WqV8VyWi){91DLq$T$9DDp`7bRWH*e%t!)-WK={`MdM-kDNk8_gBZzVo|OL^ zw0(;VfXB-dF73Q#_upd!62vXs!V1I5xGZ1wSVk`^jQ`Z9uWKelehd5VHndUC6I~$P z$K<&)4?KAKcZ#gEIRA+THNhy@I-iF6v+{Y2QEPY^6tQP`!2(p)>s`2hr-1e2figxE5e2xX<3t(6+mOt@|+l*ih55wxAqk28mu$<-Xyg#jF+LUK8s*{Wi9VI=*oA(`?zAYMK$?K|F=I$9Yz zMmuG1-@KpCZ>-y!NVx+7s(If;-(D`fJnztjP%~9gbgdKixvB(^d>yY7guQslzvn8?u0uew6Pm=<9iM~m^u?FR%(FPeUHf0Ml1$6&ubU}etMe%^_2M8kQ zMk|WzM?q@1_IdFq*{C&Adn{4gV6X^u0#sY z>AuY{ddn2Fh?2ns$SAL1Pv{fHsT}#&gOcE+Lq)e0`>9c++(3Sb`M1t+cbantwK`~F z<>C6yE?V64|Bso`42nX4L*>7nBlv50gmd9LzEVi3D z@7GJAXUEp%t7CJ|y9RlcL#?-WX~Rmmt|KXm7aJSh9!oO2abypFqNE63GDfJF>@2MS#^%kS{^uPwLpA?UU`qUbPMKZD_2BDG8M%8;dg+Ht&M3oEOLv z8z6p1K#^k#m(>*L6I{vXB1B&Xj<#T|wwgRv;^(=Sp?X0JKMO5nfqJi32BF($!jwVX zyhhKZvqUu>S;t7ZI2WuKZ#3jigA>qpk7DdUtTDWx4F4WyVS`W=kh#Z%8xZvpreIl%65tc1y>v4$)vhz?zObp1K~k27TW=b#uR z!_#(;1@K*^Kdb%2CKW0?y0N$abC#4LQP9%VEemw34M> z#Hv6kGTs&S+#SGVWv1KLL{Le;~?r=KejFNPK%k)kAQ=b#$xq0UWP=#L}EB9?g=$K8`1#HfDCYI~! ztY*k^8owWHWD9&xLtmMK(svy?J_qjJJ!Xp$M0j-|n0CvO?*z1IB#C`Zf-0mD-;|!zbnMEhn4bpD@v4 z$x`%CuqTfxr5m^4hA`iD%}#Y@D$g_|5r4l(S)xi0k6*Xk3--kf-+KN8@@8mq>Sdp@ z6?zgmsDb`xR&rD3&n#rgqS<<9d1A5yEr358Y#GbfMIa_ihO?foe*0Ek!zRoL$r=q>N~(?tQW{8BX$5^RyfM=vr#IwzAK_Z5V(zq;s4kuDJ`Xl_d$AYYpmQa#w~w zRT!cvkc<+75wF7!#T^J>^O+PTA2o*`;7~Nx7 z29v0F$Q`B%W2S49TMUG7usM$=&Qh$_mQmQ}MJ}aEz~@NKUC~ENS4mfNNXXB?<_&eN zGpSS3$+7*@bi8=5-s99gTW6v`y>_9GqOlH0!H*e(vju6(ilFct*zS%Pf5D7UrsFu( z0wmmbjJbBUU3E9;MEiAKUpn3=ydMre9yZ<&*@z|^J(7MJQs$I^!gB=|b53M`KS>2oh6l?GmwCNA-$T0FSzL@>V7@u%omcTK1(asNq zaC=Ia+J1!qmE^rqBcWC_KIpV6sN(pnPaGIr zNNo+XEt(v4ery(`LK}5=N&`g6QfGed5}3qN4s(TQGoj}eq4Xcn$hMZKfN%X`+IP{4VdN!&9j(1v(dw$vA7g1mx)c3` z{)tJ0wq17+P=;vH^vRmYBDN|mQp}KTewL_3UuhwRn|RbMoA*l+uiM>0#rw9k*Yiw+ zwf383E28HCuj^eB@3V-`8-^zauj|XrL&ufwK+!rD0YzK6B9+eHo~hEYwgpXecpbod zz~f=dEmS5Y@t8V6l^k0h#dxzYV0Ox*JD4quN4DZ*6WM+?!*!HtCB0w$-Os2aDQaZk zqFaMqv`#b}vKsJRXR!j?0=Y#1hI5_ z6?2tS0$TkHC`8lcZewp{56Ijv4TnJ`stIUz=7 zZ#|&0Oukw5KGyGF!gyHVqm=yiS#O1qTX8B7CP@KZ_UJNv8yVQ!ao+T&8BoA?&qC(L z3*bC5C2TW_H}ys#(Cls%Pr(cw2v~c$Z1+9oM)3qDkrfyfb|x!3rEsW*(uAn?!@!NM zCh5wyU=#JRbBt5M+Gb;nyq6Kep^rKVN?N;8{C;oe`l{uU9{`Re;hg^GMf?Gz@kJ#mYNAh+{D}Il%J^A)6#arSgbT;fbyc@PXNrK9d~$S9 zBJo5?o@NMtNq1+u#oO`ek=w(84i@cDD}99!nZR+t>CWs>EXrK*mTDy0t3b?t^d|tk zU%6ImyA?v-6~(>l%Ibf5dbXiz*`3+xlJF%oIj`?dgyuPUHE5O_x+14W6}lgS$mMn) zTXWlotv77)&;)pW0YJ?Ov&4t#srLu3cGg(-m8Y5mYM~xcrNqC84p?pP$_bGsUFt&8 zr#s!C(h+2ZMoF7fn+`Q1i3qEo8VKn-nRpoP{E@v2-kl^Q1uJJ6O=!Hw_q0g=eMpuj z{ht7%Qc5y#o%0w{sfa{oxXCbK6%W-82qwF^5Y7wp?=gAikFEnX7A?y#Ec@-AKIP;M z_;xusZL3IKf72dOje5w^_u$<56a3yZY+J(b)DZRQomzpQ#o z4Y=3e=Lt%KSO_Lf%9#tIq?o;N$4Zj+g8Ge;p)w&-5qS@ zp0BjrF$k#0{Gy3E*6n7YYX5ukMh$Q{9V&BYg#^6GjD&+EVooE~{Hp;e4{GO?8A|mx zeWuKoWYD9~h->bTsTCn?7-2Dc9jag^o9^YIGqS?r|5;<=_sKA*C`PdgXwjgbm5`0W z&m^KZ9@QNh_$7vN1P6~s_GwU45@Dn4V=hPM|B2x)vHU!rV?SNov9G0tfP;AY*&`89 zs7SpiTRK2aL`cu5^kU|&9^`zI`)Na=QMejrma@Trh!s5gHN*0n&&;0FJ+r9Y0U0AhWKI(1?p&2hez6u@o{2v^16agR^g>0H}xPG9*&d-u1+!lgXT7?016DH$|)#} zc=LDXRGQJ5BGAkPZ$p=gG7$Il z2m{uwqSMu6Uh%tXrD)T*Vfv6JFTAMX{_zQlAgSsJLLmpXiR9%J1iq!N%n*AiYfRL$ ztHb-{{O+T)f*tl%y0J)X@mFsR;<{oHI*I1T89z>cYvU{B7UQ>oN1@AdgCP+dzfo!K zY5c6nJ6kN<^ZpF9fggLB2n=P)m4f;7vEd_qdn27jvgh{9S$9d-OszQPeSC+r4r9S8 zh2EM$Grz_Hn+5BcgeWF|lOvITey1dx)ZD14=|w}EQ&hYSZGmv7;?x=Gx@|QbOwr@r z5D~Aq#hz4JyfjdcnyP13Fy%f_h*6@WNlV*-fMMaD_&q_4G>skpS>iN%jQ$&MtMkQx z(demKk4e3*O-%|mK)~Hm6D3Ac_((;cQIc{_KzSvOLrEI9TxrdUpxDh;nH+%D z;80VnyDcv)B&mj~OW`^HLE)wHcH7SOiP)yt=71SS*yzs`v5KN*iPd4_bO#NkcaW1> zndTCJRmDqMT`#xV;;}cUI!k-W%eJZURb7)&Lfqd#W748BDr}$tCa{OrAu~V{>OR!d zm1@jx`IYszDG75*eSINDs^)*005s|cOT?-9ZMCV}{OB!sGzWT4zIEg($FgCnFS0-B z%vkip5$103%TD05|A*FmL0oA1xXQeeefW3VJ_D`8D+96i6)Zu`Tz>KIM|?6=W|>b` zn7hTj9$C2*{afL`gX5$7Zd?$tsV<+|=lS)tJm8 zbT*@xO$DA9gt=+rqc7t*FH#-*5V#+bpX_Azc5NwCDpEsv@NYD7<82~j2rSCVDWyqo zVM;_z%!+fZNxtfN@029#nd<1&z)%S_s33=fv$q?}rO7JMjg*6r! zJ7F3eWBekw$4a7!_*;`{I{C)>*Vy7h4ZckN>-p5X#=HNN6S#(j%qIVogr%6>>s@N0 z$+^GvyRThxQ#7j$721jnY>YJ+0Xr?ozPJcz6ycqHWgNPUvcvWBn(TprPE@8oS#vN2 zuVP@fu}Ti?LE)&sWLc$mSZRU5!IMBlv5r*NA;}w?g`#hpk`h+E)#>YZaO)n8X;tkT zSltLz6`)*KNs~X5^phN)Lr&i@WIg7zd>-%0mWI8#x}@FOAnR%rJYkaJG|dW^>_~^( zv$&JRb_Vh!yt!?36=$i0DRNfyMRCcq+SE4o`Skm=JDvqvH`U$jz&+je(_o?>$DSs_U zR}E(IcsiZFAJ*gnWU*|aXkhL=jROQX-0!LR1N&X=`#*<6(N+Zsb9`F`q|KPMAG@u|8@wV83eTiKPP$&T{u!cWz=|a4Xq1x}U@_Nm|j=DRrtclTumBZS%< zDhY9;3J?p5dK5(}OY^VqeED8lSIw;-Tt6x{J_>J&@m@CGw*9WZ%iU|d1tYTMeZKx< z{ml9Hr-Xx6``0qb00?n{CX+YsoE`0IgVO6tX|0ZyxK-!*WUJ%qygGk(YyG{#Pwaa? zo~QR~Yp+{^_g7H@okvBe`<8_dw2O9}NHb)Czq9qZX&L+}@VI4?f2DRuIsm(VvVq881f?F|?L3 z(KpCTx`njt2jkV?Tr#5{%-~8j*%x3s$Ldv2+Em!%wbD+z-kyC&I{h&Ry*NQQ>QEpu zB|XgGZr99=j`lZX**~=5#I!UGl2(nLe>XSMNkj|Jg6UL;9|MMu zru#h^Xjney-2tl6p+jAp%|difl*W7{rMrZ;((m>T(6RYAl+@j{%JuR>neKS{{5l{* zJJ&!|aUsy$$l(Vn#Ac#TMgibl>Se|#T?D1=1RD-L#`XYAS8Tt4hj1UZ*{|a{yu%%M zi$6ZFu77XH0`Gmiq-gW}*_vwM;)}d}pTN{>^pOdHIt+Dx*kWZqV8X9}D)1L7bx#TUCFMtUoH4JghO&28 z>?-V7;67rIu}SL9CgajnnYE7a)Y3}~Jb++9RxTQ!yS(8#d#OEdQTl0Te zB;W5FrLQFRMbbL-ci*LOgclBwHGd2{$I3SS=V=P^Yf9S%R(iX#`HuHI-=Z zJ#Bp^xM33*QA#)=7z!MVv^0_Ub)OSS)`PTFNWL9EpFE(3HdJ(ksp}LCiCA0 zR~g+o)!+j382K|D8;1tP%+D3x3e87SSf=Jfrrg?~w$`8;EtS_4qm(ErcHUTHVt-7HMI;8+OP2&`uWIP4K z@2YB#U?$@=md-}$)z`DddupCCBxMm*l6N3$!v2FQHDxP^euX}jyODWAKfLDlT2pQCZ7g_^djOeRA8MqJRZ_Y@px*YB-Fx34CXHFMqv;=+^m>jI%KHSo} zJ!2hzqoJv_eT|*Y*EE8~rV|xzc$eZ-Ec)@@<|^|&8IvDLeX6zEHn;bm(QbkTa^w5W z$OznMWmU_Z=k}nj99z6sTc35t?5y-x9wO>n#Kpafc=s$|C&x?7Xg`_#cuNn`J4ued zJI+NOD;v~LrmC^+$OpwprGrUTD}t6-+fo6`>eA#dbL22$dG3VK>WHmuazt<1hudY> z5DtzVpZX*EB^$Wm5#Rn7dc1L3uOP!FMY*+;rYtlG(}^DfB))3K4L^jKwpkXAC>N0L z6O;38fGg8|Nj)>M;K`}DweeTjN!>Rfy3zMkMqbYJ;f$*%vy(Zp^jFs!*Eciw?)v-j zkM2NnLzf0JtrlIT^j1>a@pJVgZ*HoG2#MKIUPCt&Z0Ij>N9L1i#;efw3orisNV=E55zw-0jPcj@LABoF`-0tSwhvSC!*@;E}tw zJz&hD?GXu%B)#*{+moU;*A{_M)}Tv?e`?^}_>kM02mWse!N&WH zIUV`>g7DiD+tu-hr{%@%8r}G4g0uCBy-ky~x%Uk2bmZA+eAgbj4M{pMCHGH(?a-G}_pDiK zq9c(!7$dOwgxc)1xZZ{~XwbiZIyLZQ<9Nw!LrMz|2o(Ss|RQWnKDM!*;wU$=Rh!sOy^c9$^O-~5qiseAmoF26L~K>}`FgC?hT+>kzo ziL$>Ye3Ru-Ymo{svYg|2=7j0H@~a`H!^Mc?Wn_n{{LM%&!qL=SLVYP^$3qg_7~Pcm znP*QRNd!c2oL4H(-Jy)VY}dAfcmU$l zwv~+8Qjwr#VlM75oi=IWkkG$zo&(rJs*LYXS*)CEF6V&c&=6Jv;YrZ8>7EBpdE3JL z{FNjsVax9aZPg4LYJ^~Xi7aQ`KPK}v#h^jI zpviX(&!EN40!QF+m0I_23mxhmb59?iq15KgdB2!yg3vyT$cIJ-ji8R zNz|6{?9_Vo3**+LU2Ar}#@3($(Yt^TZV=&t+T~mWBuj=o5K)=!1q4l8;}-a<*GlKN z4`7q|$a&PVU^CpiL%RSaN}Z&DW8bhgw=mmoqVrui{44i}y#rp1d&Az030L50#*olR zOrK$6nd@g(kO*bwtsohDd4IE(2tg2=L zcY2Mm8W$qp!y*hdeFE`5 zxVh=-Wbx9EH7})UBk9oP%Q&|8^%_VFf?BMx^hr~_gNBj2Nm!xfk_oqsrJ_#Zsa&t}DY^~Vv84EMAO^tl#l+)3Y z+Qhe%;TZ)vjg9I7s7k?=U*+ONJ^CpbSe9wF2Us0LFIIcn>|TvFi<>c?hUb|?iSl}h zzxmR%S2tgEiOa zsaCCdI$T~;c1~SRa|1iAC1Fun_COT@yWj;)b^s#oI^lkvGKPD@e9-Q;{k!s1#Hh&_ z1l&}BI0L8TRaxK-qtR`x`+k z>j!?Ysu^bRwFK#U6cCgy&dOzS{IRjVcBWZ<0}a}a&aAS@?qrcgBGVMq%c~y=b=CE@ zbIdah;^!NaEzu)kesSKQQ#b$jJfYA+1?AxUppm`3lfjKUP$LAaK(S|`4VC_R^Wgtiw)18nSHp(zL}v_Sb%TEmj3@Q%xBe*_UFZu%+K%5->kMOMj>1uSd1xY&C5 z3c8LSataPf2O3iU++9|(z1PvWTt9R%Wn%5i$?f92?&Z&q--{JH$s~8TeWi_DThf|i z?3??^LDV-mT4;HpX+HlO7h~XJ`@3HFsoz2TXz#)3xb#Fn-z4>#^V4A86(oYe@&tD5 zyXk_q=K*b8Mt!Z_y{;#WXh*Wq!q$WcRXvkAy`ajrBBvL1D=Z>iiqIo@S8B#H! zuM?cH_D=gnEti+uj}W3h%A3P}OC8^$YBS);d(;OtAUUqL9CYu*Jf6!Qk_`4i2|%3E znze9sHbWxD+FRv(?9pK+lD|aG9O9mk;d0l=U*ir3DT#gaaJ!T~T`s%?N)XWLOE^Vb zklXJ=8!YHphpvTl(l!VR4mZCnU~d=89T+r!=-ZIB*Zd%V-gni)4w0~cg4gr>18V5} zv1*}ui3Pd*KlG|V?)5V{|NLO^FSbJl3>e`zGwoswV6Y=Ssio20l*(%h=3H~Qz$Iq^ z%&oUN$LtfW(jY6-BP~JBpi$048oIiH(G|D=hCZJcz-3JK?TAf1v>h2RZQQKQY*fL8 zFu$?_RJU5#SpL2)B5O;h?VQ#evhM^Jue8*%uv7Tq`mEMtpS05=oc+b=4K5r|sxs6$ zHONJzs^b#$0&4MCbGXQ!^Ehju-d7dP9T~T#iZ(KQ2Nw4fIk=2>h5HH@3P0oA#9{4f zRJP-%{UWzV8SK|* zQ5za>*n#<;nhm)kpqLkVuI%`RZF8THId>0YmUDvLAP}%zt1zPiZ@p$FlB>p zL{PPM&hbeL1-lbvd?vg5;OP%`B%(5=H)*@mGJ^@uv@}4{^D)Zm2w|yYG%rRD^DVx% zJrKM|WbM;s@KNWo$-*VBs6d6m(SJ)qBJgUofN7JyshJoSDl!$CsSMg3)`?6@t`71w z!e(CK`pPNBD4#;mx0Kk?Mbg=jQqwpemXZfi6Rn%bG@!6QEz9twPt}R!Vc*2ZW{k^X z)ry2T3wzmWeo9z%=_BT$4coN9?%-wUM5wB~1EJp~SEbxI>D!i~{i$4vztp8#9MxTK z6%Z&j+}OiL(K67YxRJlPBBQFIxbS=1Ky^>C8LJkR8n^2@XGR^mN1RG|zs=>{|4s#u z$f|*wv^gsGKp+u|XR5ba^&lDOyMl?V4xH*yU>D7ld9_tLN6+6i=m<1uF3+tZG(}`$ z-({z=-kS*jn~o`v+-cP z@V|p=pCa_|$^F6_J?E18fkBXkVq&Fjf5e#cG(Yn<)sfHcXaDI2TvfoLnpND&jZJW&7fs|b{atvFcxcRol?Tq*YEbdqa zP>fK9U4@KZbMtw7!vdI%+T-m;EVqE+w$EJ;_{E6VrgI2RCJx%STX-8dyk@@u7(Z;PSBRd z+;Z1kuUvh)rXKD<1S05k&XUmtkv&KTjy=J4quJai(FhbiQ{Hw)A5xXg#94%>4}|hw zFu~FBTl!sQs1b6S3h63j<@PO6nz$RAgQ&8C4g98QPLaLK`Xw&-Q48ufvs_R4It;t@WK^r8>vymtHvH|m zpbRwW=yI1}#AVNcbE&_<++ocb_L;`Y)^oU3BEB{hoXD zyS&)eQWN}N%T^u^)xyRtAxndiWu`&)L?YW5OS)tiLRl+}CJonE#u^o&8^(4MvJ_>? zl6@y6+r^BsFWH8%Goguu?;L&q&hwl(?>X=L{;16MEEAEK4D8R!FbD^iFF^h<3b`fqh5m%ha=Qg@c4 zc}_5}YN9iW1X|VN)Gp?2P@hNw^-GS1S2$MYm8@q}MfNAm{egJ}jXONaX0nY)1z?AF z(##0$L~D51jrmVocgtF{t7B!Ovtu-f`Kg=mDk2_*(zH!P!VGenXYS5}|15)aJ^NS)ucnHBPn*Y8M%LeN#%IX)mi>6OZdVO+z0UEs zw1)4s-K^uih~Zi;&ohl%aS`O<%%Yu4Qb1t8<)wn~b=utp6>qoQHDydul)Ygi5nR~% zm*5N(seZP6tia0Bw^U&C2E2wc3wA3N^US587aly!di{r8%hSTCj1xOEQl+HKSMOWf ztTFmGg&jS+oOOHwRETWRTH|v1+BUf_nkH@jc1#q>-#iqi|M>b#s*O&IRIwiCIahhN znls1iT$W-e1H2`!c1A!&(s@yz1jWvpQ%DF|-b_I2Km^O*Gx(h2-zBl4kVVRJx|E1efJ=ZGz2 zg}sNmDZIB=Gq|}h!K|5Uo^-5=BHWfO-+i?8zWQCDb#5n9&M~|LSaZXKLJ3FNSLgHE zU_Yg__dlFup@T>X(71h!J@RKFv*R6D7Ah8IsF3!5juTtIB0F)9hvtN}sg+1C+;U8L zBrKeW$D4xuJrpBJ5uM^7M%%^h>S?JwVAmCf^MH%HnInrfsOtvDYjUZwWuuUP-=C%#Qh z%$XQm2d26+H}cYF|A4V8nXNkQGf$r+jMNtu;*i)8&01MjE^fc_U`5Yw!GOHrLqP8` zx})xAVwgqN&ua26IQ<%yJUi$oVq~4Bc`4fKAAC8MJKuyfA@>u2!Waln3x}WxnyDhN^)^*ke5ttK73&Ba2LZa0@G|wPAqnzKDC0wCDHE z`U5UzwxU@A<9D?M8`@PhU3MFX3n&t5FeNPLeTLNpF4eLC^uj`zRP!oHc}w8L#u*DJ z({GXgBEG=U)&e)s_e~7t%!*~w3#=E@1uKD~+rL*J1wxi^Nmc>8M$=(%_Xin%_;DD}5)-49&?N} zr3-6jFA^>Lj6SI4VN zkFy&o9Vkv*o9ZAGe7)wsyv(UrpLAS#9$ls>VikZb-a0?F#Nqr0>oj=F)24gvmBPff z_N_Y+?BW4S8)D6X#VtbxNJ0X1u3k5x`L8ZZ5^LGkuu054#jnI#9_f3$e4kOPm%%wD zwJO-jO-ak>J-R9R%t zQ+%_FH(7;mY-Yyk^_{BO0MR<&gs+wBdT!m_S|}o2RcO@ySLhIReVGNp=K?qGRC)|? zmep{rNgAZYATA#3e|lycqvXA?>0ySbFuKbfj&X0{TLYK2<3#TH2e5FthQlxLocl5r zEW-#+!Du*&mU#vC#lw7N%CZV>F#lgo0Mp$a8FkHofA-*R&(!RfdW0C!4~`Y==%qLJ z>aEQ=vwN+bWVo5WjEKh+_pUho{gewcn21NnXvK1S>}%84K$_gE^opy9b&1Y8=)P%~ zWc4wjDF6}-4SM+m6`$M@6n@*v1}py-bTWXw3|doo_~VImmi?y~A0u7h_g~pjoP~&2%flSKBK1SD}%E&_aV}9&Wxv^!HZ?fvbwuF zdg&pc@MR&OD%hx8eyiW>QbR>p*M%-UJzDXkF%evZ%bc(h@xwT^v9AWuo@^b?nwx+X zWF)Yx!3I=EZ+x~X*CIcDkegOLg`&q{;DQlShO!&P=~?j$QtKQCBB6jHRwQ63wE`kS zrmhCCfLz-)Pw({=3JHLOoEalc=SMv`DwqFN!^893nKHP%i;LmxX%pHfuHz!4PbAq*u0xcSM~OHf~|3h`yEk6(hR$$NsZ( zwX+DU*mX=yu4~EJWs|<{AKTqlAJ*(sVujvA+WmdG<;|-QwwYpM-Lh}CLB@g$aXylD zRK^e55VM6ph-?_c_1!%iRnJP`WfxY;Zp%*IY8yxjEv|v~dea4rXH# zh!YqlX>!kV=x69|0cMvj@ibSMx~s|2@d5fr`B(lVL&oV?Q_Oyx#SGh#f@d~ z$21OvQ))9n7l-BKz4iZVZ5Nz=U>d?Mx%^+%bwqsK7y7aBQ6!*F=SjafOEMD1`vwm2 zn%|u>2awzXw>G5b)EV6_uc9?$cJiC!>Ahm9)iYnR`7UWD!0rBf`fpoD<|=vl(QyjE zPRMmd)LjKWPfh3yA#^4#a?xs16XK}I_-OHyW~?tGI^oT10F<6rg09^vy%lCbPoRM*hGqb59lf{$5K27LJ5EORzAn9hmdN% zF=(lpcK(?cXXC2AijLea)nYUPW_p5u%K(6fKp(BpEdTcH-vPKpu2;qakj#UyW)K0L z5v|JlpVxI=Ux=caeHa?Ne33QYfURgiLX$e24)%t3o*#SyK(|SXll!ao{k^$ecLeAC za@i=7x!;9}ShHtUrdtj*)LPJot;+1Ps@k0QM-Dp@yB%Ji;%Bh?$U+@R7<|xOq*dS!(l;Xk zA%RErK;1Bk%MDdKqfh3?ORAW)=iKbIw!=eHPD#hx=ZcdR8$pi>7#0SVlBB}R-tX;& z3Q;966&b|m7r5QK(b|X{6|#Qs>*_hRJI2n(pvF#+G_3yicGD?F`opMl^giq9FfXs? z^?oL`y_rhtfe}9t-(Uv3&^L@XE(pw}#j_b5Yz=D~N@G1%1rH|| zBtWL_Zz_|+W#b`n47Z@D1`+k3ROfGob>J)_L(#OQ|J+(9SBRFUv-^atjW)ntjq%w< zu|kA*{_rk;Ay ztGUMZuU3nL36!&^Dk|RT#f+~AFy>i5Ak~2>qdF40Gn2V$$6ondh8R4qUZF$2G5twt z&9bwol|1|3oZg6~arD3@@4ilPJ*W>D;bCOz1?8gJltVmW4Z%5ACAI2vgJjOnFh6j< z_}A(zS-Y|b>;vnNwFhecnDNtjm3dfwcYozK5RD#K?D{+gvKS;F=!GU>+@%BSs!@F30rf~z&d~^V_?Qz5;&hK?2wd9ZR< z62(iev=72UONZ^^UADHD%qm`YC{O!rr87I#2rq$rE(EojYSwwK*vG7u=q}&m{9{5`d4;i$?AfT*^IpTy^pD&w|Gdk4Cya#ZKyvK+`#@+W*Fo(DpK`{|5` z5Ee&o-?E2AHsuRU@im1J(0erPcW@{wuP1QAr#=sw^qx;>z8K3HCk+LSucy3 z$Y{|AWd8-#BQwH6{YMxF)#&%OZeYFsbfXMkRx_RW@p~nQt+t)#n9-;Cu^01&!wQ@K z{Z_*_0JGOxtGKnph9xsWQeK@+hpF}M0#q2Ok8{sY&O|NltbV<9cF03?Nc*_qY#1Zr z{$s4?YMkhE;om$@*gx`Xlu|!;n`r*R-_(wA|J15`T}y3qD+x@M<dE_?{*#EYES=U?(yHmg*yXTCL^8rS<6 zEUcwl_O7|q?$WjMBphAL7nFq!$iuZLXR^tpk}CdlhBB1u_}I2%`0IUPz+HsiBOc++Q6;-@Kc3uu_0rC)c&S)`3Sw9n6u#L$X~ANnI}Tqri5 z?;0u!I~e5l+tyQ_3gK@qfCo^t!a|6p?Z%-QBH52w%D2ZP&xB)ujTG%D(_kgx-}&#- zF>~Rv9e#gPzczw)u>qN_&_r9Ce?;K(@uWo1x{H}L8|qI9fPrZe56vu=n|fibY9zx} zqk8intDhi%Hy{gcQ_{qg(PQGP8Ik7IeILaez~4_nruZ#Pe_YZ=ckE`yUSXoVk!yTH zSx$764~}4jE_>wLzQMGfa|%9V3_de*>nn?T;`=szix)4je3ELZHS>205Nis>oaT~& zhnAUjzaCwv>;E($7+BK{Tv;5YB`IDHH$LD%i1X`W94w9V0Uf;s4$-5OS}{AY!6V?c z;w0TMTtO*j<-8+BiECX_j@q4ZFx1%gfP`!X*f)bfqqRMgj&Sr6L_{JF-`JaUVv0mt zcwt8imk$%p9Ugig(KlDg47=o`jlBtH{-exi&HD{U%zX6AO0!3_9M_|ifBcdSrC!)+ zHD7{9rBZX^=Z_j>YWuFV#2tq69nq%m?A~r`Y3^`5nr%Lyq!fJQvF`~@d>x4sr2NSI zCTmd`ICB)6;Byq#?0tK5`p~?&OK0DEkF4}aWNX1kx+}Te1=6o-)vJ+)S#hV8q_sZO v)ve`Hh!l0FT@(#D+7$ZTBQfBPgZ)1GZ{{{n`k}4zZb|jaPV-#|E5rW))hwVj literal 0 HcmV?d00001 From 348b0ef9217a8264f325841ee0aaf28176590abb Mon Sep 17 00:00:00 2001 From: miconis Date: Wed, 24 Mar 2021 15:51:27 +0100 Subject: [PATCH 07/10] bug fix, implementation of the workflow for the creation of raw_organizations (openorgs dedup), addition of the pid lists to the openorgs postgres db --- .../dhp/oa/dedup/AbstractSparkAction.java | 16 ++ .../dhp/oa/dedup/SparkPrepareNewOrgs.java | 10 +- .../dhp/oa/dedup/SparkPrepareOrgRels.java | 26 +- .../dnetlib/dhp/oa/dedup/model/OrgSimRel.java | 14 +- .../raw/MigrateDbEntitiesApplication.java | 7 + .../oa/graph/raw/common/MigrateAction.java | 3 +- .../oozie_app/config-default.xml | 18 ++ .../raw_organizations/oozie_app/workflow.xml | 270 ++++++++++++++++++ .../dhp/oa/graph/sql/queryOrganizations.sql | 4 +- .../sql/queryOrganizationsFromOpenOrgsDB.sql | 4 + 10 files changed, 354 insertions(+), 18 deletions(-) create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_organizations/oozie_app/config-default.xml create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_organizations/oozie_app/workflow.xml diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/AbstractSparkAction.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/AbstractSparkAction.java index 9a1127764..28f6e3107 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/AbstractSparkAction.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/AbstractSparkAction.java @@ -6,7 +6,9 @@ import java.io.Serializable; import java.io.StringReader; import java.util.ArrayList; import java.util.List; +import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; import org.apache.spark.SparkConf; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.SaveMode; @@ -31,6 +33,9 @@ abstract class AbstractSparkAction implements Serializable { protected static final int NUM_PARTITIONS = 1000; protected static final int NUM_CONNECTIONS = 20; + protected static final String TYPE_VALUE_SEPARATOR = "###"; + protected static final String SP_SEPARATOR = "@@@"; + protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper() .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); @@ -97,4 +102,15 @@ abstract class AbstractSparkAction implements Serializable { protected static void removeOutputDir(SparkSession spark, String path) { HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); } + + protected static String structuredPropertyListToString(List list) { + + return list + .stream() + .filter(p -> p.getQualifier() != null) + .filter(p -> StringUtils.isNotBlank(p.getQualifier().getClassid())) + .filter(p -> StringUtils.isNotBlank(p.getValue())) + .map(p -> p.getValue() + TYPE_VALUE_SEPARATOR + p.getQualifier().getClassid()) + .collect(Collectors.joining(SP_SEPARATOR)); + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPrepareNewOrgs.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPrepareNewOrgs.java index 3b29e1e17..465f56c83 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPrepareNewOrgs.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPrepareNewOrgs.java @@ -12,7 +12,6 @@ import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.HttpClients; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.FilterFunction; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.Dataset; @@ -107,8 +106,9 @@ public class SparkPrepareNewOrgs extends AbstractSparkAction { .mode(SaveMode.Append) .jdbc(dbUrl, dbTable, connectionProperties); - if (!apiUrl.isEmpty()) - updateSimRels(apiUrl); + // TODO de-comment once finished +// if (!apiUrl.isEmpty()) +// updateSimRels(apiUrl); } @@ -181,7 +181,9 @@ public class SparkPrepareNewOrgs extends AbstractSparkAction { r._1()._2().getLegalshortname() != null ? r._1()._2().getLegalshortname().getValue() : "", r._1()._2().getCountry() != null ? r._1()._2().getCountry().getClassid() : "", r._1()._2().getWebsiteurl() != null ? r._1()._2().getWebsiteurl().getValue() : "", - r._1()._2().getCollectedfrom().get(0).getValue(), ""), + r._1()._2().getCollectedfrom().get(0).getValue(), + "", + structuredPropertyListToString(r._1()._2().getPid())), Encoders.bean(OrgSimRel.class)); } 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 cbca0b326..e2d9ae9c6 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 @@ -14,6 +14,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; +import org.apache.spark.util.LongAccumulator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -209,15 +210,19 @@ public class SparkPrepareOrgRels extends AbstractSparkAction { Dataset> relations2 = relations .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(), - r._1()._3()), + (MapFunction, Tuple2>, OrgSimRel>) r -> { + + return new OrgSimRel( + r._1()._1(), + r._2()._2().getOriginalId().get(0), + r._2()._2().getLegalname() != null ? r._2()._2().getLegalname().getValue() : "", + r._2()._2().getLegalshortname() != null ? r._2()._2().getLegalshortname().getValue() : "", + r._2()._2().getCountry() != null ? r._2()._2().getCountry().getClassid() : "", + r._2()._2().getWebsiteurl() != null ? r._2()._2().getWebsiteurl().getValue() : "", + r._2()._2().getCollectedfrom().get(0).getValue(), + r._1()._3(), + structuredPropertyListToString(r._2()._2().getPid())); + }, Encoders.bean(OrgSimRel.class)) .map( (MapFunction>) o -> new Tuple2<>(o.getLocal_id(), o), @@ -311,7 +316,8 @@ public class SparkPrepareOrgRels extends AbstractSparkAction { 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()), + "group::" + r._1()._1(), + structuredPropertyListToString(r._2()._2().getPid())), Encoders.bean(OrgSimRel.class)) .map( (MapFunction>) o -> new Tuple2<>(o.getLocal_id(), o), diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/OrgSimRel.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/OrgSimRel.java index 65f383500..adff1ab8a 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/OrgSimRel.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/model/OrgSimRel.java @@ -13,12 +13,13 @@ public class OrgSimRel implements Serializable { String oa_url; String oa_collectedfrom; String group_id; + String pid_list; // separator for type-pid: "###"; separator for pids: "@@@" public OrgSimRel() { } public OrgSimRel(String local_id, String oa_original_id, String oa_name, String oa_acronym, String oa_country, - String oa_url, String oa_collectedfrom, String group_id) { + String oa_url, String oa_collectedfrom, String group_id, String pid_list) { this.local_id = local_id; this.oa_original_id = oa_original_id; this.oa_name = oa_name; @@ -27,6 +28,7 @@ public class OrgSimRel implements Serializable { this.oa_url = oa_url; this.oa_collectedfrom = oa_collectedfrom; this.group_id = group_id; + this.pid_list = pid_list; } public String getLocal_id() { @@ -93,6 +95,14 @@ public class OrgSimRel implements Serializable { this.group_id = group_id; } + public String getPid_list() { + return pid_list; + } + + public void setPid_list(String pid_list) { + this.pid_list = pid_list; + } + @Override public String toString() { return "OrgSimRel{" + @@ -103,6 +113,8 @@ public class OrgSimRel implements Serializable { ", oa_country='" + oa_country + '\'' + ", oa_url='" + oa_url + '\'' + ", oa_collectedfrom='" + oa_collectedfrom + '\'' + + ", group_id='" + group_id + '\'' + + ", pid_list='" + pid_list + '\'' + '}'; } } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java index 3e5030eaa..4d7de6f7f 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java @@ -164,6 +164,13 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i log.info("Processing Openorgs Merge Rels..."); smdbe.execute("querySimilarityFromOpenOrgsDB.sql", smdbe::processOrgOrgSimRels); + break; + + case openaire_organizations: + + log.info("Processing Organizations..."); + smdbe.execute("queryOrganizations.sql", smdbe::processOrganization, verifyNamespacePrefix); + break; } log.info("All done."); diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MigrateAction.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MigrateAction.java index d9ee9bb6a..06ebeb994 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MigrateAction.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MigrateAction.java @@ -5,5 +5,6 @@ package eu.dnetlib.dhp.oa.graph.raw.common; public enum MigrateAction { claims, // migrate claims to the raw graph openorgs, // migrate organizations from openorgs to the raw graph - openaire // migrate openaire entities to the raw graph + openaire, // migrate openaire entities to the raw graph + openaire_organizations // migrate openaire organizations entities to the raw graph } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_organizations/oozie_app/config-default.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_organizations/oozie_app/config-default.xml new file mode 100644 index 000000000..2e0ed9aee --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_organizations/oozie_app/config-default.xml @@ -0,0 +1,18 @@ + + + jobTracker + yarnRM + + + nameNode + hdfs://nameservice1 + + + oozie.use.system.libpath + true + + + oozie.action.sharelib.for.spark + spark2 + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_organizations/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_organizations/oozie_app/workflow.xml new file mode 100644 index 000000000..95b66dc34 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_organizations/oozie_app/workflow.xml @@ -0,0 +1,270 @@ + + + + + graphOutputPath + the target path to store raw graph + + + reuseContent + false + should import content from the aggregator or reuse a previous version + + + contentPath + path location to store (or reuse) content from the aggregator + + + postgresURL + the postgres URL to access to the database + + + postgresUser + the user postgres + + + postgresPassword + the password postgres + + + postgresOpenOrgsURL + the postgres URL to access to the OpenOrgs database + + + postgresOpenOrgsUser + the user of OpenOrgs database + + + postgresOpenOrgsPassword + the password of OpenOrgs database + + + dbSchema + beta + the database schema according to the D-Net infrastructure (beta or production) + + + isLookupUrl + the address of the lookUp service + + + nsPrefixBlacklist + + a blacklist of nsprefixes (comma separeted) + + + sparkDriverMemory + memory for driver process + + + sparkExecutorMemory + memory for individual executor + + + sparkExecutorCores + number of cores used by single executor + + + oozieActionShareLibForSpark2 + oozie action sharelib for spark 2.* + + + spark2ExtraListeners + com.cloudera.spark.lineage.NavigatorAppListener + spark 2.* extra listeners classname + + + spark2SqlQueryExecutionListeners + com.cloudera.spark.lineage.NavigatorQueryListener + spark 2.* sql query execution listeners classname + + + spark2YarnHistoryServerAddress + spark 2.* yarn history server address + + + spark2EventLogDir + spark 2.* event log dir location + + + + + ${jobTracker} + ${nameNode} + + + mapreduce.job.queuename + ${queueName} + + + oozie.launcher.mapred.job.queue.name + ${oozieLauncherQueueName} + + + oozie.action.sharelib.for.spark + ${oozieActionShareLibForSpark2} + + + + + + + + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] + + + + + ${wf:conf('reuseContent') eq false} + ${wf:conf('reuseContent') eq true} + + + + + + + + + + + + + + + eu.dnetlib.dhp.oa.graph.raw.MigrateDbEntitiesApplication + --hdfsPath${contentPath}/db_openaire_organizations + --postgresUrl${postgresURL} + --postgresUser${postgresUser} + --postgresPassword${postgresPassword} + --isLookupUrl${isLookupUrl} + --actionopenaire_organizations + --dbschema${dbSchema} + --nsPrefixBlacklist${nsPrefixBlacklist} + + + + + + + + + + + eu.dnetlib.dhp.oa.graph.raw.MigrateDbEntitiesApplication + --hdfsPath${contentPath}/db_openorgs + --postgresUrl${postgresOpenOrgsURL} + --postgresUser${postgresOpenOrgsUser} + --postgresPassword${postgresOpenOrgsPassword} + --isLookupUrl${isLookupUrl} + --actionopenorgs + --dbschema${dbSchema} + --nsPrefixBlacklist${nsPrefixBlacklist} + + + + + + + + + + yarn + cluster + GenerateEntities + eu.dnetlib.dhp.oa.graph.raw.GenerateEntitiesApplication + dhp-graph-mapper-${projectVersion}.jar + + --executor-memory ${sparkExecutorMemory} + --executor-cores ${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + + --sourcePaths${contentPath}/db_openaire_organizations,${contentPath}/db_openorgs + --targetPath${workingDir}/entities + --isLookupUrl${isLookupUrl} + + + + + + + + yarn + cluster + GenerateGraph + eu.dnetlib.dhp.oa.graph.raw.DispatchEntitiesApplication + dhp-graph-mapper-${projectVersion}.jar + + --executor-memory ${sparkExecutorMemory} + --executor-cores ${sparkExecutorCores} + --driver-memory=${sparkDriverMemory} + --conf spark.extraListeners=${spark2ExtraListeners} + --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} + --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} + --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} + --conf spark.sql.shuffle.partitions=7680 + + --sourcePath${workingDir}/entities + --graphRawPath${graphOutputPath} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizations.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizations.sql index 938744b11..9a8f98931 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizations.sql +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizations.sql @@ -24,7 +24,7 @@ SELECT d.officialname AS collectedfromname, o.country || '@@@dnet:countries' AS country, 'sysimport:crosswalk:entityregistry@@@dnet:provenance_actions' AS provenanceaction, - array_remove(array_agg(DISTINCT i.pid || '###' || i.issuertype), NULL) AS pid + array_agg(DISTINCT i.pid || '###' || i.issuertype || '@@@dnet:pid_types') AS pid FROM dsm_organizations o LEFT OUTER JOIN dsm_datasources d ON (d.id = o.collectedfrom) LEFT OUTER JOIN dsm_organizationpids p ON (p.organization = o.id) @@ -50,4 +50,4 @@ GROUP BY o.trust, d.id, d.officialname, - o.country + o.country; diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizationsFromOpenOrgsDB.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizationsFromOpenOrgsDB.sql index 82ece5a1c..dbe0c136b 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizationsFromOpenOrgsDB.sql +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizationsFromOpenOrgsDB.sql @@ -31,6 +31,8 @@ FROM organizations o LEFT OUTER JOIN urls u ON (u.id = o.id) LEFT OUTER JOIN other_ids i ON (i.id = o.id) LEFT OUTER JOIN other_names n ON (n.id = o.id) +WHERE + o.status = 'approved' GROUP BY o.id, o.name, @@ -72,6 +74,8 @@ FROM other_names n LEFT OUTER JOIN organizations o ON (n.id = o.id) LEFT OUTER JOIN urls u ON (u.id = o.id) LEFT OUTER JOIN other_ids i ON (i.id = o.id) +WHERE + o.status = 'approved' GROUP BY o.id, o.creation_date, From 5dfb66b0fa13d1a6f74f161b7208dc72f33da207 Mon Sep 17 00:00:00 2001 From: miconis Date: Thu, 25 Mar 2021 10:29:34 +0100 Subject: [PATCH 08/10] minor changes --- .../java/eu/dnetlib/dhp/oa/dedup/SparkPrepareNewOrgs.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPrepareNewOrgs.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPrepareNewOrgs.java index 465f56c83..950676677 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPrepareNewOrgs.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkPrepareNewOrgs.java @@ -106,9 +106,8 @@ public class SparkPrepareNewOrgs extends AbstractSparkAction { .mode(SaveMode.Append) .jdbc(dbUrl, dbTable, connectionProperties); - // TODO de-comment once finished -// if (!apiUrl.isEmpty()) -// updateSimRels(apiUrl); + if (!apiUrl.isEmpty()) + updateSimRels(apiUrl); } From 2355cc4e9b4bd5547d256434780b9db6cdd31ece Mon Sep 17 00:00:00 2001 From: miconis Date: Mon, 29 Mar 2021 10:07:12 +0200 Subject: [PATCH 09/10] minor changes and bug fix --- .../oaf/utils/OrganizationPidComparator.java | 5 ++ .../dnetlib/dhp/schema/oaf/utils/PidType.java | 2 +- .../dhp/oa/dedup/AbstractSparkAction.java | 4 ++ .../dhp/oa/dedup/DedupRecordFactory.java | 2 +- .../eu/dnetlib/dhp/oa/dedup/IdGenerator.java | 9 ++- .../oa/dedup/SparkCopyOpenorgsMergeRels.java | 55 +++++++++--------- .../oa/dedup/SparkCopyOpenorgsSimRels.java | 2 +- .../dhp/oa/dedup/SparkCreateSimRels.java | 2 +- .../dhp/oa/dedup/SparkUpdateEntity.java | 26 ++++++--- .../dhp/oa/dedup/scan/oozie_app/workflow.xml | 57 +++++++++++++------ .../dhp/oa/dedup/EntityMergerTest.java | 2 +- .../dnetlib/dhp/oa/dedup/IdGeneratorTest.java | 11 ++++ .../dnetlib/dhp/oa/dedup/SparkDedupTest.java | 23 +++++--- .../dhp/oa/dedup/SparkOpenorgsTest.java | 5 +- .../dedup/json/organization_idgeneration.json | 3 + .../raw/MigrateDbEntitiesApplication.java | 39 ++++++++----- .../oa/graph/raw/common/MigrateAction.java | 3 +- .../raw_organizations/oozie_app/workflow.xml | 2 +- ...gsDB.sql => queryOpenOrgsForOrgsDedup.sql} | 0 .../graph/sql/queryOpenOrgsForProvision.sql | 41 +++++++++++++ ...> queryOpenOrgsSimilarityForOrgsDedup.sql} | 8 +-- .../queryOpenOrgsSimilarityForProvision.sql | 12 ++++ 22 files changed, 224 insertions(+), 89 deletions(-) create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/json/organization_idgeneration.json rename dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/{queryOrganizationsFromOpenOrgsDB.sql => queryOpenOrgsForOrgsDedup.sql} (100%) create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsForProvision.sql rename dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/{querySimilarityFromOpenOrgsDB.sql => queryOpenOrgsSimilarityForOrgsDedup.sql} (89%) create mode 100644 dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsSimilarityForProvision.sql diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/OrganizationPidComparator.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/OrganizationPidComparator.java index 57285fb82..3a6df2924 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/OrganizationPidComparator.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/OrganizationPidComparator.java @@ -13,6 +13,11 @@ public class OrganizationPidComparator implements Comparator PidType lClass = PidType.tryValueOf(left.getQualifier().getClassid()); PidType rClass = PidType.tryValueOf(right.getQualifier().getClassid()); + if (lClass.equals(PidType.openorgs)) + return -1; + if (rClass.equals(PidType.openorgs)) + return 1; + if (lClass.equals(PidType.GRID)) return -1; if (rClass.equals(PidType.GRID)) diff --git a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/PidType.java b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/PidType.java index 62f682026..5a297be5e 100644 --- a/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/PidType.java +++ b/dhp-common/src/main/java/eu/dnetlib/dhp/schema/oaf/utils/PidType.java @@ -9,7 +9,7 @@ public enum PidType { doi, pmid, pmc, handle, arXiv, nct, pdb, // Organization - GRID, mag_id, urn, + openorgs, corda, corda_h2020, GRID, mag_id, urn, // Used by dedup undefined, original; diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/AbstractSparkAction.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/AbstractSparkAction.java index 28f6e3107..708d67f6e 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/AbstractSparkAction.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/AbstractSparkAction.java @@ -99,6 +99,10 @@ abstract class AbstractSparkAction implements Serializable { dataset.write().option("compression", "gzip").mode(mode).json(outPath); } + protected static void saveParquet(Dataset dataset, String outPath, SaveMode mode) { + dataset.write().option("compression", "gzip").mode(mode).parquet(outPath); + } + protected static void removeOutputDir(SparkSession spark, String path) { HdfsSupport.remove(path, spark.sparkContext().hadoopConfiguration()); } 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 99cd7c31f..fe9bd74ce 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 @@ -89,7 +89,7 @@ public class DedupRecordFactory { t -> { T duplicate = t._2(); - // prepare the list of pids to use for the id generation + // prepare the list of pids to be used for the id generation bestPids.add(Identifier.newInstance(duplicate)); entity.mergeFrom(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 51e54ee4f..dd9b16790 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 @@ -36,7 +36,14 @@ public class IdGenerator implements Serializable { } private static String dedupify(String ns) { - StringBuilder prefix = new StringBuilder(substringBefore(ns, "_")).append("_dedup"); + + StringBuilder prefix; + if (PidType.valueOf(substringBefore(ns, "_")) == PidType.openorgs) { + prefix = new StringBuilder(substringBefore(ns, "_")); + } else { + prefix = new StringBuilder(substringBefore(ns, "_")).append("_dedup"); + } + while (prefix.length() < 12) { prefix.append("_"); } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java index 201043a08..6bd1a00b9 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsMergeRels.java @@ -23,12 +23,13 @@ import eu.dnetlib.dhp.schema.oaf.DataInfo; import eu.dnetlib.dhp.schema.oaf.KeyValue; import eu.dnetlib.dhp.schema.oaf.Qualifier; import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.utils.DHPUtils; import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import eu.dnetlib.pace.config.DedupConfig; +import net.sf.saxon.ma.trie.Tuple2; -//copy simrels (verified) from relation to the workdir in order to make them available for the deduplication public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction { private static final Logger log = LoggerFactory.getLogger(SparkCopyOpenorgsMergeRels.class); public static final String PROVENANCE_ACTION_CLASS = "sysimport:dedup"; @@ -84,24 +85,32 @@ public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction { .map(patchRelFn(), Encoders.bean(Relation.class)) .toJavaRDD() .filter(this::isOpenorgs) - .filter(this::filterOpenorgsRels) - .filter(this::excludeOpenorgsMesh) - .filter(this::excludeNonOpenorgs); // excludes relations with no openorgs id involved + .filter(this::filterOpenorgsRels); + + JavaRDD selfRawRels = rawRels + .map(r -> r.getSource()) + .distinct() + .map(s -> rel(s, s, "isSimilarTo", dedupConf)); log.info("Number of raw Openorgs Relations collected: {}", rawRels.count()); // turn openorgs isSimilarTo relations into mergerels - JavaRDD mergeRelsRDD = rawRels.flatMap(rel -> { - List mergerels = new ArrayList<>(); + JavaRDD mergeRelsRDD = rawRels + .union(selfRawRels) + .map(r -> { + r.setSource(createDedupID(r.getSource())); // create the dedup_id to align it to the openaire dedup + // format + return r; + }) + .flatMap(rel -> { - String openorgsId = rel.getSource().contains("openorgs____") ? rel.getSource() : rel.getTarget(); - String mergedId = rel.getSource().contains("openorgs____") ? rel.getTarget() : rel.getSource(); + List mergerels = new ArrayList<>(); - mergerels.add(rel(openorgsId, mergedId, "merges", dedupConf)); - mergerels.add(rel(mergedId, openorgsId, "isMergedIn", dedupConf)); + mergerels.add(rel(rel.getSource(), rel.getTarget(), "merges", dedupConf)); + mergerels.add(rel(rel.getTarget(), rel.getSource(), "isMergedIn", dedupConf)); - return mergerels.iterator(); - }); + return mergerels.iterator(); + }); log.info("Number of Openorgs Merge Relations created: {}", mergeRelsRDD.count()); @@ -144,22 +153,6 @@ public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction { return false; } - private boolean excludeOpenorgsMesh(Relation rel) { - - if (rel.getSource().contains("openorgsmesh") || rel.getTarget().contains("openorgsmesh")) { - return false; - } - return true; - } - - private boolean excludeNonOpenorgs(Relation rel) { - - if (rel.getSource().contains("openorgs____") || rel.getTarget().contains("openorgs____")) { - return true; - } - return false; - } - private Relation rel(String source, String target, String relClass, DedupConfig dedupConf) { String entityType = dedupConf.getWf().getEntityType(); @@ -189,4 +182,10 @@ public class SparkCopyOpenorgsMergeRels extends AbstractSparkAction { r.setDataInfo(info); return r; } + + public String createDedupID(String id) { + + String prefix = id.split("\\|")[0]; + return prefix + "|dedup_wf_001::" + DHPUtils.md5(id); + } } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsSimRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsSimRels.java index b7f88a5f6..8cffacd7e 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsSimRels.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsSimRels.java @@ -82,7 +82,7 @@ public class SparkCopyOpenorgsSimRels extends AbstractSparkAction { .map(patchRelFn(), Encoders.bean(Relation.class)) .filter(this::filterOpenorgsRels); - save(rawRels, outputPath, SaveMode.Append); + saveParquet(rawRels, outputPath, SaveMode.Append); log.info("Copied " + rawRels.count() + " Similarity Relations"); } diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateSimRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateSimRels.java index 6963312e0..96693ebf0 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateSimRels.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCreateSimRels.java @@ -109,7 +109,7 @@ public class SparkCreateSimRels extends AbstractSparkAction { .rdd(), Encoders.bean(Relation.class)); - save(simRels, outputPath, SaveMode.Append); + saveParquet(simRels, outputPath, SaveMode.Append); log.info("Generated " + simRels.count() + " Similarity Relations"); diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkUpdateEntity.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkUpdateEntity.java index 779fb91d6..5ebc00d5a 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkUpdateEntity.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkUpdateEntity.java @@ -13,6 +13,7 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.api.java.function.PairFunction; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; @@ -22,11 +23,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.common.EntityType; import eu.dnetlib.dhp.schema.common.ModelSupport; -import eu.dnetlib.dhp.schema.oaf.DataInfo; -import eu.dnetlib.dhp.schema.oaf.Oaf; -import eu.dnetlib.dhp.schema.oaf.OafEntity; -import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.schema.oaf.*; +import eu.dnetlib.dhp.schema.oaf.utils.PidType; import eu.dnetlib.dhp.utils.ISLookupClientFactory; import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; import eu.dnetlib.pace.util.MapDocumentUtil; @@ -103,12 +103,22 @@ public class SparkUpdateEntity extends AbstractSparkAction { MapDocumentUtil.getJPathString(IDJSONPATH, s), s)); JavaRDD map = entitiesWithId .leftOuterJoin(mergedIds) - .map( - k -> k._2()._2().isPresent() - ? updateDeletedByInference(k._2()._1(), clazz) - : k._2()._1()); + .map(k -> { + if (k._2()._2().isPresent()) { + return updateDeletedByInference(k._2()._1(), clazz); + } + return k._2()._1(); + }); + + if (type == EntityType.organization) // exclude openorgs with deletedbyinference=true + map = map.filter(it -> { + Organization org = OBJECT_MAPPER.readValue(it, Organization.class); + return !org.getId().contains("openorgs____") || (org.getId().contains("openorgs____") + && !org.getDataInfo().getDeletedbyinference()); + }); sourceEntity = map.union(sc.textFile(dedupRecordPath)); + } sourceEntity.saveAsTextFile(outputPath, GzipCodec.class); diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml index c28a2a921..4b39cb56a 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/scan/oozie_app/workflow.xml @@ -83,7 +83,7 @@ - + Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] @@ -98,16 +98,14 @@ - - - - - - - - - - + + + + + + + + @@ -213,17 +211,16 @@ --actionSetId${actionSetIdOpenorgs} --numPartitions8000 - + - - + yarn cluster - Copy Openorgs Entities - eu.dnetlib.dhp.oa.dedup.SparkCopyOpenorgs + Create Organizations Dedup Records + eu.dnetlib.dhp.oa.dedup.SparkCreateDedupRecord dhp-dedup-openaire-${projectVersion}.jar --executor-memory=${sparkExecutorMemory} @@ -237,12 +234,40 @@ --graphBasePath${graphBasePath} --workingPath${workingPath} + --isLookUpUrl${isLookUpUrl} --actionSetId${actionSetIdOpenorgs} + + + + + + + + + + + + + + + + + + + + + + + + + + + yarn 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 3f10af5b8..787295c41 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 @@ -112,7 +112,7 @@ public class EntityMergerTest implements Serializable { assertEquals("2018-09-30", pub_merged.getDateofacceptance().getValue()); // verify authors - assertEquals(9, pub_merged.getAuthor().size()); + assertEquals(13, pub_merged.getAuthor().size()); assertEquals(4, AuthorMerger.countAuthorsPids(pub_merged.getAuthor())); // verify title diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/IdGeneratorTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/IdGeneratorTest.java index a6604dd30..6b0b8dfa2 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/IdGeneratorTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/IdGeneratorTest.java @@ -36,6 +36,8 @@ public class IdGeneratorTest { private static List> bestIds2; private static List> bestIds3; + private static List> bestIdsOrg; + private static String testEntityBasePath; @BeforeAll @@ -48,6 +50,8 @@ public class IdGeneratorTest { bestIds = createBestIds(testEntityBasePath + "/publication_idgeneration.json", Publication.class); bestIds2 = createBestIds(testEntityBasePath + "/publication_idgeneration2.json", Publication.class); bestIds3 = createBestIds(testEntityBasePath + "/publication_idgeneration3.json", Publication.class); + + bestIdsOrg = createBestIds(testEntityBasePath + "/organization_idgeneration.json", Organization.class); } @Test @@ -76,6 +80,13 @@ public class IdGeneratorTest { assertEquals("50|dedup_wf_001::0829b5191605bdbea36d6502b8c1ce1g", id2); } + @Test + public void generateIdOrganizationTest() { + String id1 = IdGenerator.generate(bestIdsOrg, "20|defaultID"); + + assertEquals("20|openorgs____::599c15a70fcb03be6ba08f75f14d6076", id1); + } + protected static List> createBestIds(String path, Class clazz) { final Stream> ids = readSample(path, clazz) .stream() 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 c706061a0..33da45feb 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 @@ -174,27 +174,27 @@ public class SparkDedupTest implements Serializable { long orgs_simrel = spark .read() - .load(testOutputBasePath + "/" + testActionSetId + "/organization_simrel") + .load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "organization")) .count(); long pubs_simrel = spark .read() - .load(testOutputBasePath + "/" + testActionSetId + "/publication_simrel") + .load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "publication")) .count(); long sw_simrel = spark .read() - .load(testOutputBasePath + "/" + testActionSetId + "/software_simrel") + .load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "software")) .count(); long ds_simrel = spark .read() - .load(testOutputBasePath + "/" + testActionSetId + "/dataset_simrel") + .load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "dataset")) .count(); long orp_simrel = spark .read() - .load(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_simrel") + .load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "otherresearchproduct")) .count(); assertEquals(3082, orgs_simrel); @@ -204,6 +204,7 @@ public class SparkDedupTest implements Serializable { assertEquals(6750, orp_simrel); } + @Disabled @Test @Order(2) public void collectSimRelsTest() throws Exception { @@ -254,9 +255,15 @@ public class SparkDedupTest implements Serializable { long orp_simrel = spark .read() - .load(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_simrel") + .json(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_simrel") .count(); +// System.out.println("orgs_simrel = " + orgs_simrel); +// System.out.println("pubs_simrel = " + pubs_simrel); +// System.out.println("sw_simrel = " + sw_simrel); +// System.out.println("ds_simrel = " + ds_simrel); +// System.out.println("orp_simrel = " + orp_simrel); + assertEquals(3672, orgs_simrel); assertEquals(10459, pubs_simrel); assertEquals(3767, sw_simrel); @@ -456,7 +463,7 @@ public class SparkDedupTest implements Serializable { testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_deduprecord") .count(); - assertEquals(84, orgs_deduprecord); + assertEquals(85, orgs_deduprecord); assertEquals(65, pubs_deduprecord); assertEquals(51, sw_deduprecord); assertEquals(97, ds_deduprecord); @@ -540,7 +547,7 @@ public class SparkDedupTest implements Serializable { .count(); assertEquals(896, publications); - assertEquals(837, organizations); + assertEquals(838, organizations); assertEquals(100, projects); assertEquals(100, datasource); assertEquals(200, softwares); diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsTest.java b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsTest.java index 6ad2145a9..7aaed3de7 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsTest.java +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsTest.java @@ -110,6 +110,7 @@ public class SparkOpenorgsTest implements Serializable { "/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json"))); } + @Disabled @Test public void copyOpenorgsTest() throws Exception { @@ -162,7 +163,7 @@ public class SparkOpenorgsTest implements Serializable { .load(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel") .count(); - assertEquals(6, orgs_mergerel); + assertEquals(384, orgs_mergerel); } @@ -191,7 +192,7 @@ public class SparkOpenorgsTest implements Serializable { .textFile(testOutputBasePath + "/" + testActionSetId + "/organization_simrel") .count(); - assertEquals(96, orgs_simrel); + assertEquals(73, orgs_simrel); } @Test diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/json/organization_idgeneration.json b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/json/organization_idgeneration.json new file mode 100644 index 000000000..7e8ec63c7 --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/json/organization_idgeneration.json @@ -0,0 +1,3 @@ +{"eclegalbody": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}, "ecresearchorganization": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}, "legalname": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "Universitas Dr Soetomo"}, "pid": [], "websiteurl": null, "oaiprovenance": null, "logourl": null, "collectedfrom": [{"dataInfo": null, "value": "DOAJ-Articles", "key": "10|driver______::bee53aa31dc2cbb538c10c2b65fa5824"}], "dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "alternativeNames": [], "echighereducation": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}, "id": "20|doajarticles::0af3389716873a78a03f2316de09845b", "eclegalperson": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}, "lastupdatetimestamp": 1616749318035, "ecinternationalorganizationeurinterests": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}, "dateofcollection": "2020-05-25", "dateoftransformation": "2020-05-25", "ecnonprofit": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}, "ecenterprise": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}, "ecinternationalorganization": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}, "ecnutscode": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}, "legalshortname": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "Universitas Dr Soetomo"}, "country": {"classid": "ID", "classname": "Indonesia", "schemename": "dnet:countries", "schemeid": "dnet:countries"}, "extraInfo": [], "originalId": ["doajarticles::Universitas_Dr_Soetomo"], "ecsmevalidated": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}} +{"eclegalbody": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}, "ecresearchorganization": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}, "legalname": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "University of DR Soetomo"}, "pid": [], "websiteurl": null, "oaiprovenance": null, "logourl": null, "collectedfrom": [{"dataInfo": null, "value": "DOAJ-Articles", "key": "10|driver______::bee53aa31dc2cbb538c10c2b65fa5824"}], "dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "alternativeNames": [], "echighereducation": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}, "id": "20|doajarticles::4a639ae8f8668ea44699e98ee5a8f1b9", "eclegalperson": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}, "lastupdatetimestamp": 1616749318035, "ecinternationalorganizationeurinterests": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}, "dateofcollection": "2018-09-18", "dateoftransformation": "2018-09-18", "ecnonprofit": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}, "ecenterprise": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}, "ecinternationalorganization": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}, "ecnutscode": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}, "legalshortname": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "University of DR Soetomo"}, "country": {"classid": "ID", "classname": "Indonesia", "schemename": "dnet:countries", "schemeid": "dnet:countries"}, "extraInfo": [], "originalId": ["doajarticles::University_of_DR_Soetomo"], "ecsmevalidated": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.900"}, "value": "false"}} +{"eclegalbody": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.950"}, "value": "false"}, "ecresearchorganization": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.950"}, "value": "false"}, "legalname": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.950"}, "value": "Universitas Dr. Soetomo"}, "pid": [{"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.950"}, "qualifier": {"classid": "ISNI", "classname": "International Standard Name Identifier", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "0000 0004 1758 8103"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.950"}, "qualifier": {"classid": "GRID", "classname": "GRID", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "grid.444390.e"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.950"}, "qualifier": {"classid": "ROR", "classname": "ROR", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "https://ror.org/04s03g948"}, {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.950"}, "qualifier": {"classid": "Wikidata", "classname": "Wikidata", "schemename": "dnet:pid_types", "schemeid": "dnet:pid_types"}, "value": "Q12523318"}], "websiteurl": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.950"}, "value": "https://unitomo.ac.id/"}, "oaiprovenance": null, "logourl": null, "collectedfrom": [{"dataInfo": null, "value": "OpenOrgs Database", "key": "10|openaire____::0362fcdb3076765d9c0041ad331553e8"}], "dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.950"}, "alternativeNames": [], "echighereducation": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.950"}, "value": "false"}, "id": "20|openorgs____::599c15a70fcb03be6ba08f75f14d6076", "eclegalperson": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.950"}, "value": "false"}, "lastupdatetimestamp": 1616749318824, "ecinternationalorganizationeurinterests": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.950"}, "value": "false"}, "dateofcollection": "2020-07-16", "dateoftransformation": "2020-07-16", "ecnonprofit": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.950"}, "value": "false"}, "ecenterprise": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.950"}, "value": "false"}, "ecinternationalorganization": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.950"}, "value": "false"}, "ecnutscode": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.950"}, "value": "false"}, "legalshortname": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.950"}, "value": "UNITOMO"}, "country": {"classid": "ID", "classname": "Indonesia", "schemename": "dnet:countries", "schemeid": "dnet:countries"}, "extraInfo": [], "originalId": ["openorgs____::0000034824"], "ecsmevalidated": {"dataInfo": {"deletedbyinference": false, "provenanceaction": {"classid": "sysimport:crosswalk:entityregistry", "classname": "sysimport:crosswalk:entityregistry", "schemename": "dnet:provenanceActions", "schemeid": "dnet:provenanceActions"}, "inferred": false, "inferenceprovenance": "", "invisible": false, "trust": "0.950"}, "value": "false"}} \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java index 1776689bd..823fd83d3 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/MigrateDbEntitiesApplication.java @@ -163,14 +163,25 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i .execute( "queryProjectOrganization.sql", smdbe::processProjectOrganization, verifyNamespacePrefix); break; - case openorgs: + case openorgs_dedup: log.info("Processing Openorgs..."); smdbe .execute( - "queryOrganizationsFromOpenOrgsDB.sql", smdbe::processOrganization, verifyNamespacePrefix); + "queryOpenOrgsForOrgsDedup.sql", smdbe::processOrganization, verifyNamespacePrefix); log.info("Processing Openorgs Merge Rels..."); - smdbe.execute("querySimilarityFromOpenOrgsDB.sql", smdbe::processOrgOrgSimRels); + smdbe.execute("queryOpenOrgsSimilarityForOrgsDedup.sql", smdbe::processOrgOrgSimRels); + + break; + + case openorgs: + log.info("Processing Openorgs For Provision..."); + smdbe + .execute( + "queryOpenOrgsForProvision.sql", smdbe::processOrganization, verifyNamespacePrefix); + + log.info("Processing Openorgs Merge Rels..."); + smdbe.execute("queryOpenOrgsSimilarityForProvision.sql", smdbe::processOrgOrgSimRels); break; @@ -647,17 +658,19 @@ public class MigrateDbEntitiesApplication extends AbstractMigrationApplication i r1.setDataInfo(info); r1.setLastupdatetimestamp(lastUpdateTimestamp); - final Relation r2 = new Relation(); - r2.setRelType(ORG_ORG_RELTYPE); - r2.setSubRelType(ORG_ORG_SUBRELTYPE); - r2.setRelClass(relClass); - r2.setSource(orgId2); - r2.setTarget(orgId1); - r2.setCollectedfrom(collectedFrom); - r2.setDataInfo(info); - r2.setLastupdatetimestamp(lastUpdateTimestamp); + // removed because there's no difference between two sides //TODO +// final Relation r2 = new Relation(); +// r2.setRelType(ORG_ORG_RELTYPE); +// r2.setSubRelType(ORG_ORG_SUBRELTYPE); +// r2.setRelClass(relClass); +// r2.setSource(orgId2); +// r2.setTarget(orgId1); +// r2.setCollectedfrom(collectedFrom); +// r2.setDataInfo(info); +// r2.setLastupdatetimestamp(lastUpdateTimestamp); +// return Arrays.asList(r1, r2); - return Arrays.asList(r1, r2); + return Arrays.asList(r1); } catch (final Exception e) { throw new RuntimeException(e); } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MigrateAction.java b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MigrateAction.java index 06ebeb994..517cc8d62 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MigrateAction.java +++ b/dhp-workflows/dhp-graph-mapper/src/main/java/eu/dnetlib/dhp/oa/graph/raw/common/MigrateAction.java @@ -4,7 +4,8 @@ package eu.dnetlib.dhp.oa.graph.raw.common; //enum to specify the different actions available for the MigrateDbEntitiesApplication job public enum MigrateAction { claims, // migrate claims to the raw graph - openorgs, // migrate organizations from openorgs to the raw graph + openorgs_dedup, // migrate organizations from openorgs to the raw graph + openorgs, // migrate organization from openorgs to the raw graph for provision openaire, // migrate openaire entities to the raw graph openaire_organizations // migrate openaire organizations entities to the raw graph } diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_organizations/oozie_app/workflow.xml b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_organizations/oozie_app/workflow.xml index 95b66dc34..714d69697 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_organizations/oozie_app/workflow.xml +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/raw_organizations/oozie_app/workflow.xml @@ -156,7 +156,7 @@ --postgresUser${postgresOpenOrgsUser} --postgresPassword${postgresOpenOrgsPassword} --isLookupUrl${isLookupUrl} - --actionopenorgs + --actionopenorgs_dedup --dbschema${dbSchema} --nsPrefixBlacklist${nsPrefixBlacklist} diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizationsFromOpenOrgsDB.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsForOrgsDedup.sql similarity index 100% rename from dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOrganizationsFromOpenOrgsDB.sql rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsForOrgsDedup.sql diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsForProvision.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsForProvision.sql new file mode 100644 index 000000000..6f5f93789 --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsForProvision.sql @@ -0,0 +1,41 @@ +SELECT + o.id AS organizationid, + coalesce((array_agg(a.acronym))[1], o.name) AS legalshortname, + o.name AS legalname, + array_agg(DISTINCT n.name) AS "alternativeNames", + (array_agg(u.url))[1] AS websiteurl, + '' AS logourl, + o.creation_date AS dateofcollection, + o.modification_date AS dateoftransformation, + false AS inferred, + false AS deletedbyinference, + 0.95 AS trust, + '' AS inferenceprovenance, + 'openaire____::openorgs' AS collectedfromid, + 'OpenOrgs Database' AS collectedfromname, + o.country || '@@@dnet:countries' AS country, + 'sysimport:crosswalk:entityregistry@@@dnet:provenance_actions' AS provenanceaction, + array_agg(DISTINCT i.otherid || '###' || i.type || '@@@dnet:pid_types') AS pid, + null AS eclegalbody, + null AS eclegalperson, + null AS ecnonprofit, + null AS ecresearchorganization, + null AS echighereducation, + null AS ecinternationalorganizationeurinterests, + null AS ecinternationalorganization, + null AS ecenterprise, + null AS ecsmevalidated, + null AS ecnutscode +FROM organizations o + LEFT OUTER JOIN acronyms a ON (a.id = o.id) + LEFT OUTER JOIN urls u ON (u.id = o.id) + LEFT OUTER JOIN other_ids i ON (i.id = o.id) + LEFT OUTER JOIN other_names n ON (n.id = o.id) +WHERE + o.status = 'approved' +GROUP BY + o.id, + o.name, + o.creation_date, + o.modification_date, + o.country; \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/querySimilarityFromOpenOrgsDB.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsSimilarityForOrgsDedup.sql similarity index 89% rename from dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/querySimilarityFromOpenOrgsDB.sql rename to dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsSimilarityForOrgsDedup.sql index 138bf6a96..e509127df 100644 --- a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/querySimilarityFromOpenOrgsDB.sql +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsSimilarityForOrgsDedup.sql @@ -23,7 +23,7 @@ SELECT false AS deletedbyinference, 0.99 AS trust, '' AS inferenceprovenance, - 'isSimilarTo' AS relclass + 'isSimilarTo' AS relclass FROM other_names n LEFT OUTER JOIN organizations o ON (n.id = o.id) @@ -40,8 +40,4 @@ SELECT 0.99 AS trust, '' AS inferenceprovenance, 'isDifferentFrom' AS relclass -FROM oa_duplicates WHERE reltype = 'is_different' - - ---TODO ??? ---Creare relazioni isDifferentFrom anche tra i suggerimenti: (A is_similar B) and (A is_different C) => (B is_different C) \ No newline at end of file +FROM oa_duplicates WHERE reltype = 'is_different' \ No newline at end of file diff --git a/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsSimilarityForProvision.sql b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsSimilarityForProvision.sql new file mode 100644 index 000000000..db95cfe0b --- /dev/null +++ b/dhp-workflows/dhp-graph-mapper/src/main/resources/eu/dnetlib/dhp/oa/graph/sql/queryOpenOrgsSimilarityForProvision.sql @@ -0,0 +1,12 @@ +-- relations approved by the user and suggested by the dedup +SELECT + local_id AS id1, + oa_original_id AS id2, + 'openaire____::openorgs' AS collectedfromid, + 'OpenOrgs Database' AS collectedfromname, + false AS inferred, + false AS deletedbyinference, + 0.99 AS trust, + '' AS inferenceprovenance, + 'isSimilarTo' AS relclass +FROM oa_duplicates WHERE reltype = 'is_similar' OR reltype = 'suggested'; \ No newline at end of file From f446580e9fe5b1f4be898ddb77587435465648d3 Mon Sep 17 00:00:00 2001 From: miconis Date: Mon, 29 Mar 2021 16:10:46 +0200 Subject: [PATCH 10/10] code refactoring (useless classes and wf removed), implementation of the test for the openorgs dedup --- dhp-workflows/dhp-dedup-openaire/pom.xml | 6 + .../dhp/oa/dedup/SparkCollectSimRels.java | 184 -------- .../oa/dedup/SparkCopyOpenorgsSimRels.java | 1 + .../oa/dedup/collectSimRels_parameters.json | 44 -- .../neworgs/oozie_app/config-default.xml | 18 - .../oa/dedup/neworgs/oozie_app/workflow.xml | 208 --------- .../orgsdedup/oozie_app/config-default.xml | 18 - .../oa/dedup/orgsdedup/oozie_app/workflow.xml | 240 ----------- .../dnetlib/dhp/oa/dedup/SparkDedupTest.java | 78 +--- .../dhp/oa/dedup/SparkOpenorgsDedupTest.java | 408 ++++++++++++++++++ ...39-09c4-4aa5-83fe-4cc5405607ad-c000.txt.gz | Bin 0 -> 4844 bytes ...39-09c4-4aa5-83fe-4cc5405607ad-c000.txt.gz | Bin 0 -> 3428 bytes ...39-09c4-4aa5-83fe-4cc5405607ad-c000.txt.gz | Bin 0 -> 3191 bytes ...9f-4ae6-4db9-919d-85ddc0a60f92-c000.txt.gz | Bin 0 -> 683 bytes ...9f-4ae6-4db9-919d-85ddc0a60f92-c000.txt.gz | Bin 0 -> 1755 bytes .../graph/sql/queryOpenOrgsForOrgsDedup.sql | 4 +- .../queryOpenOrgsSimilarityForOrgsDedup.sql | 2 +- .../dhp/oa/graph/sql/queryOrganizations.sql | 2 +- 18 files changed, 424 insertions(+), 789 deletions(-) delete mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCollectSimRels.java delete mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/collectSimRels_parameters.json delete mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/neworgs/oozie_app/config-default.xml delete mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/neworgs/oozie_app/workflow.xml delete mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/orgsdedup/oozie_app/config-default.xml delete mode 100644 dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/orgsdedup/oozie_app/workflow.xml create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsDedupTest.java create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/openorgs_dedup/organization/part-00000-5248a339-09c4-4aa5-83fe-4cc5405607ad-c000.txt.gz create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/openorgs_dedup/organization/part-00001-5248a339-09c4-4aa5-83fe-4cc5405607ad-c000.txt.gz create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/openorgs_dedup/organization/part-00002-5248a339-09c4-4aa5-83fe-4cc5405607ad-c000.txt.gz create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/openorgs_dedup/relation/part-00000-94553c9f-4ae6-4db9-919d-85ddc0a60f92-c000.txt.gz create mode 100644 dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/openorgs_dedup/relation/part-00003-94553c9f-4ae6-4db9-919d-85ddc0a60f92-c000.txt.gz diff --git a/dhp-workflows/dhp-dedup-openaire/pom.xml b/dhp-workflows/dhp-dedup-openaire/pom.xml index 04e158542..52cc149a9 100644 --- a/dhp-workflows/dhp-dedup-openaire/pom.xml +++ b/dhp-workflows/dhp-dedup-openaire/pom.xml @@ -94,6 +94,12 @@ org.apache.httpcomponents httpclient + + com.h2database + h2 + 1.4.200 + test + diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCollectSimRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCollectSimRels.java deleted file mode 100644 index f9e6448b0..000000000 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCollectSimRels.java +++ /dev/null @@ -1,184 +0,0 @@ - -package eu.dnetlib.dhp.oa.dedup; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; - -import org.apache.commons.io.IOUtils; -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.sql.*; -import org.dom4j.DocumentException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import eu.dnetlib.dhp.application.ArgumentApplicationParser; -import eu.dnetlib.dhp.schema.oaf.DataInfo; -import eu.dnetlib.dhp.schema.oaf.Relation; -import eu.dnetlib.dhp.utils.ISLookupClientFactory; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; -import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; -import eu.dnetlib.pace.config.DedupConfig; -import scala.Tuple2; - -public class SparkCollectSimRels extends AbstractSparkAction { - - private static final Logger log = LoggerFactory.getLogger(SparkCollectSimRels.class); - - Dataset simGroupsDS; - Dataset groupsDS; - - public SparkCollectSimRels(ArgumentApplicationParser parser, SparkSession spark, Dataset simGroupsDS, - Dataset groupsDS) { - super(parser, spark); - this.simGroupsDS = simGroupsDS; - this.groupsDS = groupsDS; - } - - public static void main(String[] args) throws Exception { - ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkBlockStats.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/collectSimRels_parameters.json"))); - parser.parseArgument(args); - - SparkConf conf = new SparkConf(); - - final String dbUrl = parser.get("postgresUrl"); - final String dbUser = parser.get("postgresUser"); - final String dbPassword = parser.get("postgresPassword"); - - SparkSession spark = getSparkSession(conf); - - DataFrameReader readOptions = spark - .read() - .format("jdbc") - .option("url", dbUrl) - .option("user", dbUser) - .option("password", dbPassword); - - new SparkCollectSimRels( - parser, - spark, - readOptions.option("dbtable", "similarity_groups").load(), - readOptions.option("dbtable", "groups").load()) - .run(ISLookupClientFactory.getLookUpService(parser.get("isLookUpUrl"))); - } - - @Override - void run(ISLookUpService isLookUpService) throws DocumentException, ISLookUpException, IOException { - - // read oozie parameters - final String isLookUpUrl = parser.get("isLookUpUrl"); - final String actionSetId = parser.get("actionSetId"); - final String workingPath = parser.get("workingPath"); - final int numPartitions = Optional - .ofNullable(parser.get("numPartitions")) - .map(Integer::valueOf) - .orElse(NUM_PARTITIONS); - final String dbUrl = parser.get("postgresUrl"); - final String dbUser = parser.get("postgresUser"); - - log.info("numPartitions: '{}'", numPartitions); - log.info("isLookUpUrl: '{}'", isLookUpUrl); - log.info("actionSetId: '{}'", actionSetId); - log.info("workingPath: '{}'", workingPath); - log.info("postgresUser: {}", dbUser); - log.info("postgresUrl: {}", dbUrl); - log.info("postgresPassword: xxx"); - - JavaPairRDD> similarityGroup = simGroupsDS - .toJavaRDD() - .mapToPair(r -> new Tuple2<>(r.getString(0), r.getString(1))) - .groupByKey() - .mapToPair( - i -> new Tuple2<>(i._1(), StreamSupport - .stream(i._2().spliterator(), false) - .collect(Collectors.toList()))); - - JavaPairRDD groupIds = groupsDS - .toJavaRDD() - .mapToPair(r -> new Tuple2<>(r.getString(0), r.getString(1))); - - JavaRDD, List>> groups = similarityGroup - .leftOuterJoin(groupIds) - .filter(g -> g._2()._2().isPresent()) - .map(g -> new Tuple2<>(new Tuple2<>(g._1(), g._2()._2().get()), g._2()._1())); - - JavaRDD relations = groups.flatMap(g -> { - String firstId = g._2().get(0); - List rels = new ArrayList<>(); - - for (String id : g._2()) { - if (!firstId.equals(id)) - rels.add(createSimRel(firstId, id, g._1()._2())); - } - - return rels.iterator(); - }); - - Dataset resultRelations = spark - .createDataset( - relations.filter(r -> r.getRelType().equals("resultResult")).rdd(), - Encoders.bean(Relation.class)) - .repartition(numPartitions); - - Dataset organizationRelations = spark - .createDataset( - relations.filter(r -> r.getRelType().equals("organizationOrganization")).rdd(), - Encoders.bean(Relation.class)) - .repartition(numPartitions); - - for (DedupConfig dedupConf : getConfigurations(isLookUpService, actionSetId)) { - switch (dedupConf.getWf().getSubEntityValue()) { - case "organization": - savePostgresRelation(organizationRelations, workingPath, actionSetId, "organization"); - break; - default: - savePostgresRelation( - resultRelations, workingPath, actionSetId, dedupConf.getWf().getSubEntityValue()); - break; - } - } - - } - - private Relation createSimRel(String source, String target, String entity) { - final Relation r = new Relation(); - r.setSubRelType("dedupSimilarity"); - r.setRelClass("isSimilarTo"); - r.setDataInfo(new DataInfo()); - - switch (entity) { - case "result": - r.setSource("50|" + source); - r.setTarget("50|" + target); - r.setRelType("resultResult"); - break; - case "organization": - r.setSource("20|" + source); - r.setTarget("20|" + target); - r.setRelType("organizationOrganization"); - break; - default: - throw new IllegalArgumentException("unmanaged entity type: " + entity); - } - return r; - } - - private void savePostgresRelation(Dataset newRelations, String workingPath, String actionSetId, - String entityType) { - newRelations - .write() - .mode(SaveMode.Append) - .parquet(DedupUtility.createSimRelPath(workingPath, actionSetId, entityType)); - } - -} diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsSimRels.java b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsSimRels.java index 8cffacd7e..dbcd40289 100644 --- a/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsSimRels.java +++ b/dhp-workflows/dhp-dedup-openaire/src/main/java/eu/dnetlib/dhp/oa/dedup/SparkCopyOpenorgsSimRels.java @@ -9,6 +9,7 @@ import java.util.Optional; import org.apache.commons.io.IOUtils; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.ForeachFunction; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.graphx.Edge; import org.apache.spark.rdd.RDD; diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/collectSimRels_parameters.json b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/collectSimRels_parameters.json deleted file mode 100644 index da1011371..000000000 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/collectSimRels_parameters.json +++ /dev/null @@ -1,44 +0,0 @@ -[ - { - "paramName": "la", - "paramLongName": "isLookUpUrl", - "paramDescription": "address for the LookUp", - "paramRequired": true - }, - { - "paramName": "asi", - "paramLongName": "actionSetId", - "paramDescription": "action set identifier (name of the orchestrator)", - "paramRequired": true - }, - { - "paramName": "w", - "paramLongName": "workingPath", - "paramDescription": "path of the working directory", - "paramRequired": true - }, - { - "paramName": "np", - "paramLongName": "numPartitions", - "paramDescription": "number of partitions for the similarity relations intermediate phases", - "paramRequired": false - }, - { - "paramName": "purl", - "paramLongName": "postgresUrl", - "paramDescription": "the url of the postgres server", - "paramRequired": true - }, - { - "paramName": "pusr", - "paramLongName": "postgresUser", - "paramDescription": "the owner of the postgres database", - "paramRequired": true - }, - { - "paramName": "ppwd", - "paramLongName": "postgresPassword", - "paramDescription": "the password for the postgres user", - "paramRequired": true - } -] \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/neworgs/oozie_app/config-default.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/neworgs/oozie_app/config-default.xml deleted file mode 100644 index 2e0ed9aee..000000000 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/neworgs/oozie_app/config-default.xml +++ /dev/null @@ -1,18 +0,0 @@ - - - jobTracker - yarnRM - - - nameNode - hdfs://nameservice1 - - - oozie.use.system.libpath - true - - - oozie.action.sharelib.for.spark - spark2 - - \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/neworgs/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/neworgs/oozie_app/workflow.xml deleted file mode 100644 index 9bfdaaebd..000000000 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/neworgs/oozie_app/workflow.xml +++ /dev/null @@ -1,208 +0,0 @@ - - - - graphBasePath - the raw graph base path - - - isLookUpUrl - the address of the lookUp service - - - actionSetId - id of the actionSet - - - workingPath - path for the working directory - - - dedupGraphPath - path for the output graph - - - cutConnectedComponent - max number of elements in a connected component - - - dbUrl - the url of the database - - - dbUser - the user of the database - - - dbTable - the name of the table in the database - - - dbPwd - the passowrd of the user of the database - - - sparkDriverMemory - memory for driver process - - - sparkExecutorMemory - memory for individual executor - - - sparkExecutorCores - number of cores used by single executor - - - oozieActionShareLibForSpark2 - oozie action sharelib for spark 2.* - - - spark2ExtraListeners - com.cloudera.spark.lineage.NavigatorAppListener - spark 2.* extra listeners classname - - - spark2SqlQueryExecutionListeners - com.cloudera.spark.lineage.NavigatorQueryListener - spark 2.* sql query execution listeners classname - - - spark2YarnHistoryServerAddress - spark 2.* yarn history server address - - - spark2EventLogDir - spark 2.* event log dir location - - - - - ${jobTracker} - ${nameNode} - - - mapreduce.job.queuename - ${queueName} - - - oozie.launcher.mapred.job.queue.name - ${oozieLauncherQueueName} - - - oozie.action.sharelib.for.spark - ${oozieActionShareLibForSpark2} - - - - - - - - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - - - - - - - - - - - -pb - ${graphBasePath}/relation - ${workingPath}/${actionSetId}/organization_simrel - - - - - - - - yarn - cluster - Create Similarity Relations - eu.dnetlib.dhp.oa.dedup.SparkCreateSimRels - dhp-dedup-openaire-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 - - --graphBasePath${graphBasePath} - --isLookUpUrl${isLookUpUrl} - --actionSetId${actionSetId} - --workingPath${workingPath} - --numPartitions8000 - - - - - - - - yarn - cluster - Create Merge Relations - eu.dnetlib.dhp.oa.dedup.SparkCreateMergeRels - dhp-dedup-openaire-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 - - --graphBasePath${graphBasePath} - --workingPath${workingPath} - --isLookUpUrl${isLookUpUrl} - --actionSetId${actionSetId} - --cutConnectedComponent${cutConnectedComponent} - - - - - - - - yarn - cluster - Prepare New Organizations - eu.dnetlib.dhp.oa.dedup.SparkPrepareNewOrgs - dhp-dedup-openaire-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 - - --graphBasePath${graphBasePath} - --workingPath${workingPath} - --isLookUpUrl${isLookUpUrl} - --actionSetId${actionSetId} - --dbUrl${dbUrl} - --dbTable${dbTable} - --dbUser${dbUser} - --dbPwd${dbPwd} - --numConnections20 - - - - - - - \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/orgsdedup/oozie_app/config-default.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/orgsdedup/oozie_app/config-default.xml deleted file mode 100644 index 2e0ed9aee..000000000 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/orgsdedup/oozie_app/config-default.xml +++ /dev/null @@ -1,18 +0,0 @@ - - - jobTracker - yarnRM - - - nameNode - hdfs://nameservice1 - - - oozie.use.system.libpath - true - - - oozie.action.sharelib.for.spark - spark2 - - \ No newline at end of file diff --git a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/orgsdedup/oozie_app/workflow.xml b/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/orgsdedup/oozie_app/workflow.xml deleted file mode 100644 index e7c95ee8d..000000000 --- a/dhp-workflows/dhp-dedup-openaire/src/main/resources/eu/dnetlib/dhp/oa/dedup/orgsdedup/oozie_app/workflow.xml +++ /dev/null @@ -1,240 +0,0 @@ - - - - graphBasePath - the raw graph base path - - - isLookUpUrl - the address of the lookUp service - - - actionSetId - id of the actionSet - - - workingPath - path for the working directory - - - dedupGraphPath - path for the output graph - - - cutConnectedComponent - max number of elements in a connected component - - - dbUrl - the url of the database - - - dbUser - the user of the database - - - dbTable - the name of the table in the database - - - dbPwd - the passowrd of the user of the database - - - sparkDriverMemory - memory for driver process - - - sparkExecutorMemory - memory for individual executor - - - sparkExecutorCores - number of cores used by single executor - - - oozieActionShareLibForSpark2 - oozie action sharelib for spark 2.* - - - spark2ExtraListeners - com.cloudera.spark.lineage.NavigatorAppListener - spark 2.* extra listeners classname - - - spark2SqlQueryExecutionListeners - com.cloudera.spark.lineage.NavigatorQueryListener - spark 2.* sql query execution listeners classname - - - spark2YarnHistoryServerAddress - spark 2.* yarn history server address - - - spark2EventLogDir - spark 2.* event log dir location - - - - - ${jobTracker} - ${nameNode} - - - mapreduce.job.queuename - ${queueName} - - - oozie.launcher.mapred.job.queue.name - ${oozieLauncherQueueName} - - - oozie.action.sharelib.for.spark - ${oozieActionShareLibForSpark2} - - - - - - - - Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}] - - - - - - - - - - - - - -pb - /tmp/graph_openorgs_and_corda/relation - ${workingPath}/${actionSetId}/organization_simrel - - - - - - - - yarn - cluster - Create Similarity Relations - eu.dnetlib.dhp.oa.dedup.SparkCreateSimRels - dhp-dedup-openaire-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 - - --graphBasePath${graphBasePath} - --isLookUpUrl${isLookUpUrl} - --actionSetId${actionSetId} - --workingPath${workingPath} - --numPartitions8000 - - - - - - - - yarn - cluster - Create Merge Relations - eu.dnetlib.dhp.oa.dedup.SparkCreateMergeRels - dhp-dedup-openaire-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 - - --graphBasePath${graphBasePath} - --workingPath${workingPath} - --isLookUpUrl${isLookUpUrl} - --actionSetId${actionSetId} - --cutConnectedComponent${cutConnectedComponent} - - - - - - - - yarn - cluster - Prepare Organization Relations - eu.dnetlib.dhp.oa.dedup.SparkPrepareOrgRels - dhp-dedup-openaire-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 - - --graphBasePath${graphBasePath} - --workingPath${workingPath} - --isLookUpUrl${isLookUpUrl} - --actionSetId${actionSetId} - --dbUrl${dbUrl} - --dbTable${dbTable} - --dbUser${dbUser} - --dbPwd${dbPwd} - --numConnections20 - - - - - - - - yarn - cluster - Prepare New Organizations - eu.dnetlib.dhp.oa.dedup.SparkPrepareNewOrgs - dhp-dedup-openaire-${projectVersion}.jar - - --executor-memory=${sparkExecutorMemory} - --executor-cores=${sparkExecutorCores} - --driver-memory=${sparkDriverMemory} - --conf spark.extraListeners=${spark2ExtraListeners} - --conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners} - --conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress} - --conf spark.eventLog.dir=${nameNode}${spark2EventLogDir} - --conf spark.sql.shuffle.partitions=3840 - - --graphBasePath${graphBasePath} - --workingPath${workingPath} - --isLookUpUrl${isLookUpUrl} - --actionSetId${actionSetId} - --apiUrl${apiUrl} - --dbUrl${dbUrl} - --dbTable${dbTable} - --dbUser${dbUser} - --dbPwd${dbPwd} - --numConnections20 - - - - - - - \ No newline at end of file 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 33da45feb..851e72dee 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 @@ -204,76 +204,8 @@ public class SparkDedupTest implements Serializable { assertEquals(6750, orp_simrel); } - @Disabled @Test @Order(2) - public void collectSimRelsTest() throws Exception { - ArgumentApplicationParser parser = new ArgumentApplicationParser( - IOUtils - .toString( - SparkCollectSimRels.class - .getResourceAsStream( - "/eu/dnetlib/dhp/oa/dedup/collectSimRels_parameters.json"))); - parser - .parseArgument( - new String[] { - "-asi", testActionSetId, - "-la", "lookupurl", - "-w", testOutputBasePath, - "-np", "50", - "-purl", "jdbc:postgresql://localhost:5432/dnet_dedup", - "-pusr", "postgres_user", - "-ppwd", "" - }); - - new SparkCollectSimRels( - parser, - spark, - spark.read().load(testDedupAssertionsBasePath + "/similarity_groups"), - spark.read().load(testDedupAssertionsBasePath + "/groups")) - .run(isLookUpService); - - long orgs_simrel = spark - .read() - .load(testOutputBasePath + "/" + testActionSetId + "/organization_simrel") - .count(); - - long pubs_simrel = spark - .read() - .load(testOutputBasePath + "/" + testActionSetId + "/publication_simrel") - .count(); - - long sw_simrel = spark - .read() - .load(testOutputBasePath + "/" + testActionSetId + "/software_simrel") - .count(); - - long ds_simrel = spark - .read() - .load(testOutputBasePath + "/" + testActionSetId + "/dataset_simrel") - .count(); - - long orp_simrel = spark - .read() - .json(testOutputBasePath + "/" + testActionSetId + "/otherresearchproduct_simrel") - .count(); - -// System.out.println("orgs_simrel = " + orgs_simrel); -// System.out.println("pubs_simrel = " + pubs_simrel); -// System.out.println("sw_simrel = " + sw_simrel); -// System.out.println("ds_simrel = " + ds_simrel); -// System.out.println("orp_simrel = " + orp_simrel); - - assertEquals(3672, orgs_simrel); - assertEquals(10459, pubs_simrel); - assertEquals(3767, sw_simrel); - assertEquals(3865, ds_simrel); - assertEquals(10173, orp_simrel); - - } - - @Test - @Order(3) public void cutMergeRelsTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -369,7 +301,7 @@ public class SparkDedupTest implements Serializable { } @Test - @Order(4) + @Order(3) public void createMergeRelsTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -424,7 +356,7 @@ public class SparkDedupTest implements Serializable { } @Test - @Order(5) + @Order(4) public void createDedupRecordTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -471,7 +403,7 @@ public class SparkDedupTest implements Serializable { } @Test - @Order(6) + @Order(5) public void updateEntityTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -587,7 +519,7 @@ public class SparkDedupTest implements Serializable { } @Test - @Order(7) + @Order(6) public void propagateRelationTest() throws Exception { ArgumentApplicationParser parser = new ArgumentApplicationParser( @@ -637,7 +569,7 @@ public class SparkDedupTest implements Serializable { } @Test - @Order(8) + @Order(7) public void testRelations() throws Exception { testUniqueness("/eu/dnetlib/dhp/dedup/test/relation_1.json", 12, 10); testUniqueness("/eu/dnetlib/dhp/dedup/test/relation_2.json", 10, 2); 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 new file mode 100644 index 000000000..f33eca57f --- /dev/null +++ b/dhp-workflows/dhp-dedup-openaire/src/test/java/eu/dnetlib/dhp/oa/dedup/SparkOpenorgsDedupTest.java @@ -0,0 +1,408 @@ + +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.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Properties; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.IOUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FilterFunction; +import org.apache.spark.api.java.function.ForeachFunction; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.util.CollectionsUtils; +import org.junit.jupiter.api.*; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.platform.commons.util.StringUtils; +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 eu.dnetlib.dhp.application.ArgumentApplicationParser; +import eu.dnetlib.dhp.schema.oaf.OafMapperUtils; +import eu.dnetlib.dhp.schema.oaf.Organization; +import eu.dnetlib.dhp.schema.oaf.Relation; +import eu.dnetlib.dhp.utils.DHPUtils; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpException; +import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService; +import eu.dnetlib.pace.util.MapDocumentUtil; +import scala.Tuple2; + +@ExtendWith(MockitoExtension.class) +@TestMethodOrder(MethodOrderer.OrderAnnotation.class) +public class SparkOpenorgsDedupTest implements Serializable { + + private static String dbUrl = "jdbc:h2:mem:openorgs_test;DB_CLOSE_DELAY=-1;DATABASE_TO_UPPER=false"; + private static String dbUser = "sa"; + private static String dbTable = "tmp_dedup_events"; + private static String dbPwd = ""; + + @Mock(serializable = true) + ISLookUpService isLookUpService; + + protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper() + .configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + + 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-openorgs"; + + @BeforeAll + public static void cleanUp() throws IOException, URISyntaxException { + + testGraphBasePath = Paths + .get(SparkDedupTest.class.getResource("/eu/dnetlib/dhp/dedup/openorgs_dedup").toURI()) + .toFile() + .getAbsolutePath(); + testOutputBasePath = createTempDirectory(SparkDedupTest.class.getSimpleName() + "-") + .toAbsolutePath() + .toString(); + testDedupGraphBasePath = createTempDirectory(SparkDedupTest.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", "200"); + spark = SparkSession + .builder() + .appName(SparkDedupTest.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( + IOUtils + .toString( + SparkDedupTest.class + .getResourceAsStream( + "/eu/dnetlib/dhp/dedup/profiles/mock_orchestrator_openorgs.xml"))); + + lenient() + .when(isLookUpService.getResourceProfileByQuery(Mockito.contains("organization"))) + .thenReturn( + IOUtils + .toString( + SparkDedupTest.class + .getResourceAsStream( + "/eu/dnetlib/dhp/dedup/conf/org.curr.conf.json"))); + } + + @Test + @Order(1) + public void createSimRelsTest() throws Exception { + + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/createSimRels_parameters.json"))); + + parser + .parseArgument( + new String[] { + "-i", testGraphBasePath, + "-asi", testActionSetId, + "-la", "lookupurl", + "-w", testOutputBasePath, + "-np", "50" + }); + + new SparkCreateSimRels(parser, spark).run(isLookUpService); + + long orgs_simrel = spark + .read() + .load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "organization")) + .count(); + + assertEquals(288, orgs_simrel); + } + + @Test + @Order(2) + public void copyOpenorgsSimRels() throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCopyOpenorgsSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/copyOpenorgsMergeRels_parameters.json"))); + parser + .parseArgument( + new String[] { + "-i", testGraphBasePath, + "-asi", testActionSetId, + "-w", testOutputBasePath, + "-la", "lookupurl", + "-np", "50" + }); + + new SparkCopyOpenorgsSimRels(parser, spark).run(isLookUpService); + + long orgs_simrel = spark + .read() + .load(DedupUtility.createSimRelPath(testOutputBasePath, testActionSetId, "organization")) + .count(); + + assertEquals(324, orgs_simrel); + } + + @Test + @Order(3) + public void createMergeRelsTest() throws Exception { + + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateMergeRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/createCC_parameters.json"))); + + parser + .parseArgument( + new String[] { + "-i", + testGraphBasePath, + "-asi", + testActionSetId, + "-la", + "lookupurl", + "-w", + testOutputBasePath + }); + + new SparkCreateMergeRels(parser, spark).run(isLookUpService); + + long orgs_mergerel = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel") + .count(); + assertEquals(132, orgs_mergerel); + + // verify that a DiffRel is in the mergerels (to be sure that the job supposed to remove them has something to + // do) + List diffRels = jsc + .textFile(DedupUtility.createEntityPath(testGraphBasePath, "relation")) + .map(s -> OBJECT_MAPPER.readValue(s, Relation.class)) + .filter(r -> r.getRelClass().equals("isDifferentFrom")) + .map(r -> r.getTarget()) + .collect(); + assertEquals(18, diffRels.size()); + + List mergeRels = spark + .read() + .load(testOutputBasePath + "/" + testActionSetId + "/organization_mergerel") + .as(Encoders.bean(Relation.class)) + .toJavaRDD() + .map(r -> r.getTarget()) + .collect(); + assertFalse(Collections.disjoint(mergeRels, diffRels)); + + } + + @Test + @Order(4) + public void prepareOrgRelsTest() throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/prepareOrgRels_parameters.json"))); + parser + .parseArgument( + new String[] { + "-i", + testGraphBasePath, + "-asi", + testActionSetId, + "-la", + "lookupurl", + "-w", + testOutputBasePath, + "-du", + dbUrl, + "-dusr", + dbUser, + "-t", + dbTable, + "-dpwd", + dbPwd + }); + + new SparkPrepareOrgRels(parser, spark).run(isLookUpService); + + final Properties connectionProperties = new Properties(); + connectionProperties.put("user", dbUser); + connectionProperties.put("password", dbPwd); + + Connection connection = DriverManager.getConnection(dbUrl, connectionProperties); + + ResultSet resultSet = connection + .prepareStatement("SELECT COUNT(*) as total_rels FROM " + dbTable) + .executeQuery(); + if (resultSet.next()) { + int total_rels = resultSet.getInt("total_rels"); + assertEquals(32, total_rels); + } else + fail("No result in the sql DB"); + resultSet.close(); + + // verify the number of organizations with duplicates + ResultSet resultSet2 = connection + .prepareStatement("SELECT COUNT(DISTINCT(local_id)) as total_orgs FROM " + dbTable) + .executeQuery(); + if (resultSet2.next()) { + int total_orgs = resultSet2.getInt("total_orgs"); + assertEquals(6, total_orgs); + } else + fail("No result in the sql DB"); + resultSet2.close(); + + // verify that no DiffRel is in the DB + List diffRels = jsc + .textFile(DedupUtility.createEntityPath(testGraphBasePath, "relation")) + .map(s -> OBJECT_MAPPER.readValue(s, Relation.class)) + .filter(r -> r.getRelClass().equals("isDifferentFrom")) + .map(r -> r.getSource() + "@@@" + r.getTarget()) + .collect(); + + List dbRels = new ArrayList<>(); + ResultSet resultSet3 = connection + .prepareStatement("SELECT local_id, oa_original_id FROM " + dbTable) + .executeQuery(); + while (resultSet3.next()) { + String source = OafMapperUtils.createOpenaireId("organization", resultSet3.getString("local_id"), true); + String target = OafMapperUtils + .createOpenaireId("organization", resultSet3.getString("oa_original_id"), true); + dbRels.add(source + "@@@" + target); + } + resultSet3.close(); + assertTrue(Collections.disjoint(dbRels, diffRels)); + + connection.close(); + } + + @Test + @Order(5) + public void prepareNewOrgsTest() throws Exception { + ArgumentApplicationParser parser = new ArgumentApplicationParser( + IOUtils + .toString( + SparkCreateSimRels.class + .getResourceAsStream( + "/eu/dnetlib/dhp/oa/dedup/prepareNewOrgs_parameters.json"))); + parser + .parseArgument( + new String[] { + "-i", + testGraphBasePath, + "-asi", + testActionSetId, + "-la", + "lookupurl", + "-w", + testOutputBasePath, + "-du", + dbUrl, + "-dusr", + dbUser, + "-t", + dbTable, + "-dpwd", + dbPwd + }); + + new SparkPrepareNewOrgs(parser, spark).run(isLookUpService); + + final Properties connectionProperties = new Properties(); + connectionProperties.put("user", dbUser); + connectionProperties.put("password", dbPwd); + + long orgs_in_diffrel = jsc + .textFile(DedupUtility.createEntityPath(testGraphBasePath, "relation")) + .map(s -> OBJECT_MAPPER.readValue(s, Relation.class)) + .filter(r -> r.getRelClass().equals("isDifferentFrom")) + .map(r -> r.getTarget()) + .distinct() + .count(); + + Connection connection = DriverManager.getConnection(dbUrl, connectionProperties); + + jsc + .textFile(DedupUtility.createEntityPath(testGraphBasePath, "relation")) + .map(s -> OBJECT_MAPPER.readValue(s, Relation.class)) + .filter(r -> r.getRelClass().equals("isDifferentFrom")) + .map(r -> r.getTarget()) + .distinct() + .foreach(s -> System.out.println("difforgs = " + s)); + ResultSet resultSet0 = connection + .prepareStatement("SELECT oa_original_id FROM " + dbTable + " WHERE local_id = ''") + .executeQuery(); + while (resultSet0.next()) + System.out + .println( + "dborgs = " + OafMapperUtils.createOpenaireId(20, resultSet0.getString("oa_original_id"), true)); + resultSet0.close(); + + ResultSet resultSet = connection + .prepareStatement("SELECT COUNT(*) as total_new_orgs FROM " + dbTable + " WHERE local_id = ''") + .executeQuery(); + if (resultSet.next()) { + int total_new_orgs = resultSet.getInt("total_new_orgs"); + assertEquals(orgs_in_diffrel + 1, total_new_orgs); + } else + fail("No result in the sql DB"); + resultSet.close(); + } + + @AfterAll + public static void finalCleanUp() throws IOException { + FileUtils.deleteDirectory(new File(testOutputBasePath)); + FileUtils.deleteDirectory(new File(testDedupGraphBasePath)); + } + +} diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/openorgs_dedup/organization/part-00000-5248a339-09c4-4aa5-83fe-4cc5405607ad-c000.txt.gz b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/openorgs_dedup/organization/part-00000-5248a339-09c4-4aa5-83fe-4cc5405607ad-c000.txt.gz new file mode 100644 index 0000000000000000000000000000000000000000..ba58d823c16070b42dad4d612051d2996f74cc27 GIT binary patch literal 4844 zcmb7`_dlC$*v9*4DLRZ2luA*eX0=7AO^jMqYSyk=HA}QcJ+>rjG%dAfY0V<`DjIt$ zD#VH%p;l1Adwbr``v<)F>B@2Cll#2x^E$uB0Y=eY`0sO(ls@IosSz5vb#V7)WA#^A zun&Z9QmIRjyLW&FZ6on@z3_wVyt4BL%6Yh)56m_leo)aruz1pq&L5(jy6|j7`8Drqq&O=1On7s zmos?wWMhM|)U+^cMNPQm#dk@b(E1-TpFU`e2K)+`r>XpnKhW~)gplU*{`Tpn`=NhZ zvegG~z8|Trd606AM*Qq(hhOm^Wf=8xQ*2vlm4)s z;PnXqZEE|4_X3UL#?N6)az^#`_hg-^(M=wZG`P;Tk#tL7-+-$FOVcK(=Qp$2zG(** z@NnN#(F{TQkrS zMJZo->666ei{lVOKNgE{`mzDHo0f{|@47~=zkFDFu!--+GOXlH@cYTls!Kn$T9I!l zTtAIQYW5ep|J!GH=(S9v>tc5-w$$WkO|qP?l}_n^FD?&`UMqAol%_AeHfDma8ngO* z!&0vAnn>!O^gbY2ZCsYHoScJzhhS{tswKUk6ze{4&n?H+lJ*U>sLut1lZ3QDwgt0q6nw{S{Er*wF?0i8n;UO`b z3sY)b1tzmEL4#SxjD208VGE;uD{DnJywisc(h%M!_0Z7og!*W!K8mYh`F_FCLEObp6mS2=n z+>>y;Sm^qfMJiQq-RlR`^2V^g9A`h6oPKgjP8B}(U3=0#R$?uuRJq7LGv#XCOo__m z8Smxb1oMp}ynB+n_-+XNH5c@-6IBz9$!V=Sl2|-I`DW8u@Q*uV>(jppvI*-j{`BX_ zB~>_OnbQej=|?YAW!Sr?vE!Q_6u*(=t>JndV2F64BQ|*bQ?%Tug%R&12OV6qB{x`z zsgByOgqcU?@o^rGUJjzGj%9T zpO;};DJDa$S!Wonq|Y0iBTc?T4)$|sA>)7eHo!-{)_zd#`7us5c&;&Bxby@rR|IA)unq#VjRQtDNKv>&ddFZ=BN@=dQ?pE_N^hjZUk{o{y>bu_gy-TEDQ~ljf+c>TFDME)xzlGj}PszH^)CuC)vA~ zK@PNF_v*@O_r$nLZ-=J7Gv@|#xx2So`qidHqwOR1V^N4dcW)Rog;B?8cZ*#)J6)Qv zRygAtJsx+$iR*3!Se6yNBgRgzGJaHCqcqQN6W4++FkmzIZ3W?i>GX z`7y8=;)drk1!4CX+|hv(ee}M{q0LaVVa5Lu~2V6 zeZl2ks}83C$29_s^0X}e`je`+>W<$e;MiWK{b(*j3tG5t1nGEbSh;KkG$q+I@GkAD zJkt;nt9FPvzQiZeiL2v(*{OCQ^O?6^C8sxsV|%k31@3IMB~ry&PQwhT!naEJ=NVQC zqTC@L%*^2QBJkJKOBHY_S~1NRk2RU=x83>llNAwc0#`2=$#wNxbp?rxWzT z;UCn_=OV}k`MSD|vlX-Q8J8MCTWp?i3n%(Ezh+fS@FhngkKo;EVlShQA}P9Pi6;k= zU=pr>w?L=F^cY+!U$mN753h$Eg2tMjm7b8?6n<7&*Y6eHoVj(?vTD|?hNLh&pz^@o zv0Y<~)0Wa}T{vVEw^A^0OD0hNG;efW<30Ew_3l?ykfRoR1$#y9)x3CYJeE2%LSz6q zr7CEpmj#fa!8xYYrx!0rN^r{EQ|M#$t9gW*Gzc|Za?X0H_Dq=fn+=9+Y-sz+TtxF- zrEqiYF;>Q#ZsmorjJmI84}?>bw|Kk<##&W~j#e*A^AZ>0l`N`f!q`#oUHj0@DbF=) zHeuzdZoC?w^I|7Vi&UipQZI!Ls@dK@YR#*O@BVm04Xi)NTaX67G8Q`hhd{I7{fx5wpoxLC<;X~}3OVbO zBVBmt(Nb4oopfFDM~7le`L=|Vl?9&q6E8yDo8yLqh}YmHlaM-daYY2hfm8gUtwoH4$`T(No zwu_p(%rZG&Bl|bfkS$9i!OD zhCq6=O=V?oj%ImFz;}x#;@`~9zfKP`-vlEGTtuSb1s{bfutGlmM!kVNbM3;g{X$SH z?LY)QBHLkGDU#0|w6qeL2urTZb};bv2Of8m%N&H&h)#srPC5)%ydj*Vo+EsolIk); z`$Ho3k!d+q`~(#T3CZC+P@h>oC@7)w3G^q<{1aHxooOX&qd09QE4IONV|=V~4!Gb7+F5%K4|0OVd4KnAP@ z+9FBQ;qRtqGwxy=k!{5RZh96g9Du@2M4ME5K=OAyRl{50SlOrAJ2CZUnE% zm}XJo4RbRvaMOiRn|Uc%_U((h@e9dDL9x#uU2?MT<|3H?}rPnkl?JboZdfwW# zf~9Tcon_FpZ+8&_Dq{?7_myB$aA*^<^F$J#R`jY21

%+$z#WMe@wIZm@-y!Nfp%7X ztwNi|sDa?y?M$*Ql0k=a>gRf%Dcv?@oI0 zKUaI6>{5qCr9`EOGYkEa`xR*5lf9h1EPWGr$^8JXT)15nf^(ki0+pETN|iIZihWA# znP|ORR>Iu+^C#xUhB5Kf+%00JLQ0FNcyw2b+(9pphBOKwCX<_&x8#c|3Pq%a)o%a# zZ%iwN#5v0u&H7g!>T~DFrcd5)6PK8(_rGtwsZ|nmxgx+;gjuNB*DHn6sQ)jSi}fi7oWI zrQS?D`>U(F)bw9Bc5)xr%T!rZ3x>lJ$92q9WVWZDf?UMuwx#ar`6{6nLau6H1f_Uo zF@leEu8AUf?Nw1ZhqS{vg3w18K_pTv7lbt%4O%OFIYxkHJUZpL?pE9Y`=`y7Ck8}q z&XW3=U6dXp2t=pzZ$NZH{TH1;Fd#bRD4~G+?POr{qPPxb5TM=!7x5IO<7<=}c%A>= zUe>&K#nG+S0hN=OKpc910mknzPG2TtScQHJk3)Y4VO2R|oeW;7`8XMT{Cv}#1=bM> zf^9S9zzjU$P6jNjmLO5&ZcE15;L+^h8G+)_a@uYmoqtnUvtp~UksmassN7HhLfk^C&k<6sH1)p-IcMGZ7Jv}9kc9H( zYU&A$-tbq=MkAh|Da-^Hbm+C`zX~_;`EItgi56`U?kI`7s-Xq3u>Fs2n0JmxY)hl{z~A1P#=%oaccNN=k$<<73b78XLT&XEN6B*;MYWml zZz^s|8=3}~9?Q|S?Mzf|W2hVPX_F;kO&41CKw zPV~THCbo}Ft4Xboy4h&Pa3D@7p8*I3vx)s}=F&p`U&wE6PwlCa}ho2EkSi9tEYn z1j&ThLf=AFIp1@bp7Are|j^!c2vC- zz#hpgc&sfZ-{~-L<=++X%zfsK^SyZc^BlVblin#Xi_oJH!!WeCj6C#+*B)3Xs6dY@ za(er^&xkC|hhp#L4%<4}GA;zl1qLQOnJ3H>P&rq?4B(dG+7NaH1?&7j+kdu?ZbZa& z{xB_zW4Pl*aDAqMb!}%k5J@)G<|StF?>9{iCg>aa51+8b2&l+H(&^PX2u@j2)Q5+c z?{m!)wDqqkG%xPW5l}I};*fTp(2Eok2XEy+i@stnCmSKre1(sp&2wi7ask2svA$z; ze&n3h^Ect~(EWBKdNAp3np|4osoMOa#JzFiVy^eYJ4>o_M$kgF59VXTH?!ZqNIP&R zEwVn4<52d^>q)px<%lrP)g^@%H-(_jyP*ekV{Q@Xh91z3xkaEGdO$aJymwkCQ`79n zSP}O?<>o7p|5p{nEYWPMWFw6aF>wgLOswhI=rM}kIsk_lCg3BU7lSB#^u zL-+77Xc7gWkLS*p0vd?5x4-8OtOHmChI{Tfz><>CiX|O%SBhq{KU}t9KU%o)Z$eZ8 pU_LQ&F@X8lF4``hn@@WeU_R>H>VWxV_H~_sn>bb^Br2&c{12`hrJn!* literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/openorgs_dedup/organization/part-00001-5248a339-09c4-4aa5-83fe-4cc5405607ad-c000.txt.gz b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/openorgs_dedup/organization/part-00001-5248a339-09c4-4aa5-83fe-4cc5405607ad-c000.txt.gz new file mode 100644 index 0000000000000000000000000000000000000000..137790bdec7f86831a611bc4dec1e7cc8e42fa41 GIT binary patch literal 3428 zcmXAqdpuMBAIFVKt_iv461f|5DRWs!B)5q~=CYER>zGT*gvlk0N+!3V<`QY;Hsnq; z*G$PZmyo5ozwm=oX_L&dcI%pr&RJumVX!9@2NeT2`9(Dsn%=jbR)$-4M-e>Z1uwUfN!dSWTntPe|H_2y4tamxBkh)>Bpw_mM$>V|9I0pq*h z`uS&hfyF;H?20PN_WGH8sJB3NSs*y zf)53ORm#U|%P9@b8uOfhFAZVhZ|wCz489D^)8`$YS|Rx^P6 zPl|>`^{Z;(r9Ty88-dA5Ja?O6F{97t?~!P-Qwrp8^kIc2RZqP<$@w9-8D_*l{;$Gx zi__VD%gkaYLa!G`ml#}d=}>#iJ;P;>Hpe9uh^Oz4Z^E^K@7w(7$*Ko+dt2^-he02x z*yB4Z$3<_x676Pd*S49B(8Z-y3R6Q*`}v(+{IVV9NU^IA(<1*uz9+W82*tl5H3S~| z$Y)v%&S|=*NES7ljv&FQjD;@lr_Bz-$|G$KKuNN>*DctKoT=yrbvs72NZ!Z1KWKKi zyS5+5HL6@?{*=Wp--}U%#%A>cBgf11z}dNSnRE5rSomZUk7pc?-K80RI;5ZymOiRa}PR;TRB1_3Jp%>7KcJ$ zNl(t3pgxi7^hWE=&2*DZNo|}GaYBR`>Qu@1*!~4lUqU0?;I-A?r9)kzf`=0NLRpE-=alL-7F%~6RaO+HqTF{lbe_y zi{BbsPQ7hF-lE%ZR+d;oZeJG@o+rA~iTpf^o{ zmjNY~%`0@ek9M_2-jBCA)q*cKoV4qg=l8 zBVl&@0WS{*G^y6T;O!Hiuqs)1J~sQJNy9W!W_|n$1;bOf%aBDFo!rEA5!=b+?0+YDf)MpKzZlEtl@#>#vM)w?Kn zqSdla>FCw&75hkG<+VnkUDvflv0?|hVBLUH8~QT(E?Mp)?H7aau=0Co@~Y7)|FC{3 z?>)RHyj8!ok;&r&+Xr8`Q)}Z9kJiSJ9M);TG|^pfjrVIuZ*F9F5?|&W${*Od*H?S6 zoiNM!H*IZ@*zmgdH@MUKzIE-G z6*sO(wH~OQTZOnhv()Y9eFf}qmkyN<{e6}5I{Wp}c9L``@<)89bm;Ph<7I4RmMuTA z*TXk^n}Vi#*gRZE@tva{d7%0*C$M~~XPl+-s~>&Av=7B#%y$3R=9EyjJN-z@=k)t4 zD5r7N!7mpL!Uv3h8vPtr2EN2r;G+fG38dN(a_5e!&hOD+HAklf=AcJx67!p4;g){X1$eTb+<#Yw* zw2Fqzpe=+?5ZDO5=GWK2YxP3$#|f6QOcVqrifxH)*zT3KB@c@a8H3>LJ2F}JS1B+N zwQAmJB^gpE_e3|ucMeWy1ml6HU%JTFhnu7NcF@N*LuO%T!=*PDGtd_RGskAEYBg9C zSP=S8Sy8O0S`Aeegjl@{hy~GL4bTPXQq?(IIV|Uh(E|3Sek4bYrQ=r|p^G3Hfc=$3 zDy^(F0<8LHHb5jzdA?)KGd<1^var(8ThUjQ6Vm}e$|~MezcGNTO__M@)5ELE90U)r zM?G^8GTAbjgWyqrGzcE`%t7b{^b*}SjWhK{C&fpl9fY8jdt7tAP@CZ2k31yVWvYvX zW+zpwte_deVjGCU$)E@~ZuXCLybIYeh@csi?)c9a!BjCQFBCZ29vvJV%p|VJuE-=F z9ehMQI+#g3gDu0dMX^9aH4$$N?!>+~Lz-$U82^Ze<6Y$B1l(NC=06H<5fboSxgv6~ zi*AI{y^emYNO9GX4-W+hlUON4XA$NRm1ivg3oHw_M1-Kg)X8)aC~&Iy=*(qd4BtZ3 z{?0V{Ybn5~l4?0uKJff9PQ_aE6PnH%7DG#S`E-S^U3&f7@rpXW(Y~K3p5@_$`kX>U z|4`MJq1l3Ji|j__B@eU*T1ogs4qFbBISEWUVxIGlc@C5LH;Hdd<^nZfE2ZZ`69!Hx zSvmvrd>QFtTpbyjzb@QNwP`=;Nk8LRDb{7jozOd2%j3wWqUzOV|IAkwT;tf#no-z! zX>F`FJU?n1*J*74m_6Pe}x^(_jbQdRdwwbIF*tf{Y5SxE^3uoqbC z!R#Dey14mx`Iw!GMn$8T37=#?d6Y2fU&1J6!f|YIN1ZwuzD=Jd^CGd8l}8 zc=E$c?2j1Hl%yL^BbMXrY=S9owCR7+qR4w8=O6LwhT9y&feSggR-en8$*8^KD08Pa zXys?ECH=Ajrl;=~mDY2j(pz3dKap)MIq*!*hQY|fe;WpaWNnxS+opMWK59#kKzw=ipY!Hk9P`Uz6sTE-{#E&Lb-is-v}asii~J>wxd-WL2NS?rC#5@V)8W^ws4|w zCWR08;Eng%^=q$0>bCzR>#agu>6E6X{i&vAIftz=@pF6G^r{1KVA$3xU5`B<1`?4p z3z2tcEna+TJ8b@aM)lo-c!kc*$ox!1as8qe@N-vaf%KefLUHBbvE1$v9&cLIb=@ar z*rAbql7%rfJXf5*PB^>ayP%CHja56eNzq84B0cb!wIt|#jmgs>Yr!JZd-6PJ2g}>* zY=nvk#&Wjg$>#2@-AJi22-S+AGbVs(V|<4OMKjJAUOk&lcMB(P6$+F1jI+k|1*EN{ zt(e7VKh{3Nl#SIt*;p})0cV9@Yw<3UI?2(K(RXe~%#%Mx!~BX(&;WOpph}SL z2Cm~yPxV6-$(FITYr-{+NL<}K11`Wcr!me{cx( zb{sqg6@c0^cef5er$$2+YJar5q4o{dG1<}C!|J=zI%oHT17R0iWY}~t50iiRmRg8D zswcIB6p)gK#b43H^gEI&X}@Jhs%^tLo@Q<72(x2q6YjdNO7t86g}3{2rt&b?OdAIf zu?#!*JG6GIXiyA@*+LNk5qqwVSkO@mu^?s(Q#ewT_KUt<8 literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/openorgs_dedup/organization/part-00002-5248a339-09c4-4aa5-83fe-4cc5405607ad-c000.txt.gz b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/openorgs_dedup/organization/part-00002-5248a339-09c4-4aa5-83fe-4cc5405607ad-c000.txt.gz new file mode 100644 index 0000000000000000000000000000000000000000..6b090b9f57974446bdef603e7cd1c3b14256b36e GIT binary patch literal 3191 zcmV--42bg|iwFP!000001MS`0Zrerz#_{`o3PM-AK;&|U9CF5YN@KLzwd2B)w`hum zF-Myli8LfRUMJ1F?7QrX?Z{`RcI?oOTnMgoE|`^~lr-bq5s}G4a%*x;IrqJ{gTZ|LgWGpE_0jp_*1|LWY0=TBg2 zJKMZDhu--)Os|i*{H_li_4)Iwj(_gi$*~V-XVutFXL5Yp*(3!i)3nH#|1a~u`S~-^ z^bdXH&%jRP#i;S3@W+ut5@D?>Q>K_KjMJ=O(h@5DNi>t{u&U*#@4vzCox2ZDjnRcz)HRc_ zyzdzMAAkSF?3jPcMpn1Saz2H84SVRv|H${hD0l1DiyJ;&&&2(@`d}y1YWBbXCZ2jTmK{vOZ=QsH7Rr&I7_}d^5B9A;dU`QLr&jsC_GB`;9*O3#x%H*h=R>0!b9Q@Z z&hud;o;4DeNss;1zBqeCT}0Hnsr`Mfnjz_;l1}W@%8B+b)`zmLKFX`#2I?-N?ooAk zA9am?V|07&`akE+@AGfMrpUeY+(^8r#CD$ecv4L*nl7s8bZmXd zYX0{cgQ^RA7j`n6>c(K6E~uz(PU`u$y>s6E;TPxCpUKgEPkr&|%HIE!H}`)2{FhHp z$0U`dM(R?@IVcd98Chzo6ye4QE{MC0NxOS*dPz6=ad$j!>SXWo4Q2ACHFMc)9F1jnIv3t?wW0lMH0nQVX0Kg1V=w%maQXyj+8W@DKj}Jw zGyb}30?zou1_hkoac(4DE^y-Q!V-KFQCq~Du@?nzVz#V+H~rvgrI?dONNs2#ZC1+6 zX--o|tSKMm%}ivJ<}@(g+#l}Xb!Tz9x&OBR&#k>df7nlkTmN{wyS@8UKiTPR_kJ7n z!N@D6&Xd`AV|<3@O9W254}@<9)%0H9MA0SkCgLtfH_;Wi+4|f_+}_oj@i$$^>dpA8 zf;U&dn>)XKab7!eB>kyL2u5gwt z%X3PKz=ZR!`@@~T;l*d^#<|x^kQ&O_rshyiC}#_sLOG$FEs6`}gmT858;PNu;WmYG zLOH`P3gyIvS&ecMBc0CDLONnnt4t=TESP^t&NE+wKc<|b%!)GSVO*yElutf9OB$EF ztHw&IQ@lK_7Uw>0NWJm#6xG3VAqED|;Ueo+B*jJ60J`>d1<-|&8;JpQ;Wh=(0qDXn zdN0tu`}K?Sy2T6ty6wDP{$q7j{_a!Wl@F!baz(G z?eqqC^;zIL#4F*flB@qQkPPHRp33*gyi(?@tFfG786uoB(7Mm0lo2xo|a zA)L6-x(!8fq4j!v6MrQHaJD};5(7BHZ3^H7aE4zL+vQ-ktUxy|{CXF!Jd0a4*W2FdB|E+0)?P9g?!W2p?(OgRAMfj}OM1zh ze)4+0b;(<7%e*Y6d+1_e)nFwZR?-C;80LwUbZsb#m2|jLxOH6to?+xhVl1Q!w<*LE z;u(I?5XDn6W|bgW&WkcPoMl81m-~NBnVGcY-z=W~fS;R5_0bW@)&?--7I}E zfX=;cS+7GePo=N==b%3rZte7w!QM97uF^Q=Xe_%^e@1p(ZD{|Jjrz;8&ZWMGiM;Sy zurvCmt6$(5ebTiq@Ql9hx)yjwA2uxS{C;yIaf`h(`lb(gC*Fj$i8~`N3ho4VuAV!e zyBW+CcYecWFstOwx5Y0!Ja#5|J% z+8ZYF%t7uA6M1uhC&06XO;>(Jd-O%uc}08lak0b`-WhdnB!+i}+Z5gj?+m{v)_4Lt zSC5^~-74k=JD*`4MOln%u_AxQfrI5Fe ziM)HD+s8y+9gFQ?pe<~Q#dcV1*P^(%wio~!b8aLCfQH)?015yNzi1c$Y9{lKO}pA2 zSy{_6DKla;)k?99Ym#cJ3g#r^G`|I)Z+3fs@Af9M%Kv?5x-!@lG4Cn#ZLnz?`EPmI ze{+-YlnY7=UAi>Qb4#5mtl*A2RY+N8e}c`s4MSffpYOzT{X(ZUi4m++Z%jwUIRN(OZ~V%s9ZR+;w37_bm3&F4N)qUDbEX< z5#>ixa9U>E@JCecW>1TQtIwD`_(dMc`YyKnQ2%M-DrML zNC%{AVOStt7`c%cNEdEXARUk{{G!3^$^oVeXI{L7=_oTq5Js&fT$)T$R|+RY&ZY8S zn?HX5-Q!Gmwe@WHPPg1ma-*Je;p{~mduB*oC?f%>gVePnDWnck*TS%nx-fDhF{Ccs zrjR;FUHC;Ib(j|ml{&(h5P~?X^PFm9bna+oOPg{-(n3FWnd>d78`}4EGinZR%r0E3 zyFS!c<|S*ji9f-^+{r;EnetQ z7&I5pNPy-*bL~h9ngh+XFf3>;jNC{JnhUonXbvVNTJaU}esX5>7ai z%0jEOu*#}}*g_Ofhq=bJ0OlG8Hd-#sp)g=BmXQF=0p{9~6fg&vYhhTxTo}2L7%&%Z zQ@|WxF8rc^In0U$!kkrvE1B`!_^~RSB$m2N<&>GykD?OKg1J`*cW-~&e5eiv(zF8X zUL|*LpPzg(KY7f|B|{#{1NfpG2>>5}uN_GNd;q={h6V72ksFBteBm|)@B#S3FACtp ztXLrMQI}H9iLfM7RFX_ESrDg5ZcAQh$G#2t=ATwp2m8@=7n*c=j+kf5;yMI|%0(~| zP&uevJCZ`>pmHq?3zZ8aHxfhT!fgtbgUW?p^j?+g55G9C{!EVUdupIue3RlOD3|6$ zD#Nu9snaa;UkGJrQRXtWw4~pLa;;chhqYYGW;qmw%SAB~a5=bKJCefX;BqYt3zrKc dHxk3;!fgtdgUf|qbakuc{tx*sV!*r10RVe}T$%s? literal 0 HcmV?d00001 diff --git a/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/openorgs_dedup/relation/part-00000-94553c9f-4ae6-4db9-919d-85ddc0a60f92-c000.txt.gz b/dhp-workflows/dhp-dedup-openaire/src/test/resources/eu/dnetlib/dhp/dedup/openorgs_dedup/relation/part-00000-94553c9f-4ae6-4db9-919d-85ddc0a60f92-c000.txt.gz new file mode 100644 index 0000000000000000000000000000000000000000..080665d22d16b6550043d98dfab96209683890c0 GIT binary patch literal 683 zcmV;c0#yAUiwFP!000001MQYei`y^|$M5|V#ph7;e)v|}OHT{!-j-z4%to9cR8%FFQWqhb# z+Y+u=^uckWl~yuXPl%upIHyu_G|j4cf^LgddsBrsd0k$=hY1`Mtb3s@KlE{E+J4)0 zhx?x``zG~IsU+7`b%d_qs#+rEjNxJk*kLWx?T$Ox`)A)NZ_8Bkk~h|p=B%AP#ymXL z9*(|E-FW^7Kc%6cQ~M4|Ni|PXb|r1bAy2K(Ln&XNdu*|v(zMICPG!ow8sg~Q!v_CU zQ2b}y;4wkyacZAyczdj&Tzt+^f8ljFteTp8+K%<$a7r626Kuw&rHX1o6qlMBxoR?Y zx4Ur;G~_k(=_}0oo1fNm!mayw@e3HXV{=r#o4c!~R6eCm>LA|^v-hyg$0KAXC~Dwx zf`UqF83rww(G>^^8Vr|FMTh~dlvO-IUgLBU`ymH-IDJ(dAbL&|L>dUXz=461QWIf# zy`eKvX>Ru!gR`Z!QQ^W9O~;EJF;59=@pt!!SNnzQh3kLE^(+npR zV1wX-AX7pL9}h3jkN8Ux`VjEUYz=e)<*g zpKtxge63_f3I$NgoY5j^ufTXMxpW$QT)1Aiz7DQm&eu#S3WOuCb2U{wY9>$Jr4Wh@gKZ>x7OWhb9y-Fo6Yfbc(7GmitqSV|J#7$c&oR1*?{cyw0_!$KVJ6X z__@rdwf=`vA3wfY*|tyjclO2UX>+T`6W`vg){voukRuhS^lEj`?e`CN=N0V_Z+N%) zC!gbAfARXf!TbK#&*enr{%&=N{L}g9Y|-ZUx6SQl%ZKmx=hFN8!{sZ)M-;Z=<0Fbr zgPWw7f?`gTH0Gk%Le&{-!D+!KKD^P>WANA?gl}%%V!*IoW6r=KYQ&0=<1Co81WJA4 zd9Cs&YI8o^CkCIE*kKqhebjV0v3GhnZPa%DhaW%w?t{B_*UxdCV=x#(P1RiVPF;{? z8rLo(l%KhlJ^!7J-`MJLy{3^Tmm$!5;80V>2%R~$iX~R5?%G{1!}U4XY#$zj6IqSj z2_nr|LaeA$3s5U8ok6?1cGoZDI%LjVH7cg6eWcv0j+92K8H71~)pboZlASS2Kt`e3 zga}d<>ZHx0!CkxS7jg|c8m10~TJMm>@(G+MEH&5G5x$y-@BrzGW7gV`dQZg7D)0;# z6auNccGpXE-7O8Fl1O@+ZCY#|J!I`_v^BTvuHE%AT+Ad)Vn9RO za~fjpPiuoF0Yn9BLgHxNAL6cG%5@@`CFe0`#1RUWI%bqMi?szcQ2c8DH3K(JaY`J; z+Kq;ZB_YzNGKMyG*Y0{zu0Q1=?%G}dMqU5Pe!gqhvlWg6L3=RDWRW#$N79C63~|@) z`o;2Vf`%$3VJ!v%xNwa$^W-Voh<+izJ_p;rzg@3$ijX@9P|`edAV>(Kw*slQnS#4^ z*UPA{uU*?&D2Y=*ZYB3JN`RC)XHwvp1G;N>y+qfg6~i+%Xr)Lx0ddBTDP}>ol-pgq z>t(bXuU*FxWm?u^fKVFL+9ifqO0s&JV({yY?s`G@J$;d1N2)c(W{Qi%ZmEY*?_HtT zkE8{1*Y0|Wt_52wsig`%*RH)~hTNJpLkHE-n!9$_%emh8(Q^cqww0bwZ}$V@>d z_Wo;k?XKOmyMD28yVtH!W^FLb(40zyM4`o?17j&zrzUspu9xUqGlR_B#p>t8kkh1+ zhqh|U6=QVQ?s|!?C0L<8jB>U%Bj(s~Rvi>*NFnfXyY70Cu492QwPp>SvWN}{l}DY@ zBqrnTuHE$#U0bc(!ugua92#cr7GsXc)&vQX`hBYIdXcUx#au&~4F^NYnrE={*F%v4 zn1YXQbk~b?jiFd;v9{D;Hqa1kMSWNmP7@&dbs2a4e{lWXW?sI;)1S{S{>Nhrp2qH$ zFsQbea84QuLu1p~3t8=*Ib$YFg_Os@n9I3W|vGzH$9RMR%6z-)YZ8EdZ`{cG? zwBCNs963-!ZHRf4X`GVcl&r1JI(ud=-k0sRU$ox-b@#1akK?VDh9vtj6t!$5O@w|K zq(CBQzQOzb-1dTOU+3G*vkj;t6}E((isdtfQl~hi#TmVi#BDFg_I1AP^|ssoc5Gkg z+eiWp2vwn)CB|9WY%Nvp_TJ|faobC>egE#Tx%r%LXT>yPo)Ko^)FV`^krKyDWc9X> zr*YdaY5Q4oLUqfv?bk3_8V|wD8VkocDh87+e(lC>FUa=K`T&VqYcS6dI7Ei!+);~$ zDiHGUK0vp