refactoring, minor changes and implementation of the wf for openorgs with integration of organization phases into the scan wf

pull/102/head
miconis 3 years ago
parent 1e1aab83e3
commit 8fea29177c

@ -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<OafEntity> clazz = ModelSupport.entityTypes.get(EntityType.valueOf(subEntity));
filterEntities(spark, entityPath, clazz)
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(outputPath);
}
public static <T extends OafEntity> Dataset<T> filterEntities(
final SparkSession spark,
final String entitiesInputPath,
final Class<T> clazz) {
// <id, json_entity>
Dataset<T> entities = spark
.read()
.textFile(entitiesInputPath)
.map(
(MapFunction<String, T>) it -> {
T entity = OBJECT_MAPPER.readValue(it, clazz);
return entity;
},
Encoders.kryo(clazz));
return entities.filter(entities.col("id").contains("openorgs____"));
}
}

@ -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;
if (destination.contains("mergerel")) {
outputPath = DedupUtility.createMergeRelPath(workingPath, actionSetId, entity);
}
else {
outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, entity);
}
final String outputPath = DedupUtility.createSimRelPath(workingPath, actionSetId, subEntity);
removeOutputDir(spark, outputPath);
removeOutputDir(spark, outputPath);
final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation");
final String relationPath = DedupUtility.createEntityPath(graphBasePath, "relation");
JavaRDD<Relation> simRels = spark.read().textFile(relationPath).map(patchRelFn(), Encoders.bean(Relation.class)).toJavaRDD().filter(r -> filterRels(r, entity));
JavaRDD<Relation> simRels =
spark.read()
.textFile(relationPath)
.map(patchRelFn(), Encoders.bean(Relation.class))
.toJavaRDD()
.filter(r -> filterRels(r, entity));
simRels.saveAsTextFile(outputPath);
}
simRels.saveAsTextFile(outputPath);
}
private static MapFunction<String, Relation> patchRelFn() {

@ -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
}
]

@ -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
}
]

@ -93,13 +93,12 @@
<error to="Kill"/>
</action>
<!--extract simrels from relations and copy into the organization_simrel-->
<action name="copySimRels">
<action name="CopySimRels">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Copy Similarity Relations</name>
<class>eu.dnetlib.dhp.oa.dedup.SparkCopySimRels</class>
<name>Copy Merge Relations</name>
<class>eu.dnetlib.dhp.oa.dedup.SparkCopyRels</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
@ -113,65 +112,11 @@
</spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
<arg>--entityType</arg><arg>organization</arg>
<arg>--destination</arg><arg>simrel</arg>
<arg>--numPartitions</arg><arg>8000</arg>
</spark>
<ok to="CreateDedupRecord"/>
<error to="Kill"/>
</action>
<!-- il root record è openorgs, le simrel fra le openorgs e gli altri record restano -->
<!-- per le organizzazioni: openorgs SOLO QUELLE VALIDATE, altre TUTTE -->
<!--create the deduprecords using relations taken from openorgs: CHECK IF IT WORKS -->
<action name="CreateDedupRecord">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Create Dedup Record</name>
<class>eu.dnetlib.dhp.oa.dedup.SparkCreateDedupRecord</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
</spark>
<ok to="UpdateEntity"/>
<error to="Kill"/>
</action>
<action name="UpdateEntity">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Update Entity</name>
<class>eu.dnetlib.dhp.oa.dedup.SparkUpdateEntity</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--dedupGraphPath</arg><arg>${dedupGraphPath}</arg>
</spark>
<ok to="CreateSimRels"/>
<error to="Kill"/>
</action>

@ -169,6 +169,61 @@
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
</spark>
<ok to="CopyMergeRels"/>
<error to="Kill"/>
</action>
<!-- copy organization relations in the working dir-->
<action name="CopyMergeRels">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Copy Merge Relations</name>
<class>eu.dnetlib.dhp.oa.dedup.SparkCopyRels</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
<arg>--entityType</arg><arg>organization</arg>
<arg>--destination</arg><arg>mergerel</arg>
<arg>--numPartitions</arg><arg>8000</arg>
</spark>
<ok to="CopyEntities"/>
<error to="Kill"/>
</action>
<action name="CopyOpenorgs">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Copy Entities</name>
<class>eu.dnetlib.dhp.oa.dedup.SparkCopyOpenorgs</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-memory=${sparkExecutorMemory}
--executor-cores=${sparkExecutorCores}
--driver-memory=${sparkDriverMemory}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.shuffle.partitions=3840
</spark-opts>
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
<arg>--workingPath</arg><arg>${workingPath}</arg>
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
</spark>
<ok to="UpdateEntity"/>
<error to="Kill"/>
</action>

Loading…
Cancel
Save