implementation of the workflow for new organizations in openorgs

This commit is contained in:
miconis 2020-10-06 13:58:09 +02:00
parent e3f7798d1b
commit a2ac7e52fb
7 changed files with 439 additions and 18 deletions

View File

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

View File

@ -1,10 +1,11 @@
package eu.dnetlib.dhp.oa.dedup;
import static jdk.nashorn.internal.objects.NativeDebug.map;
import java.io.IOException;
import java.util.*;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.Properties;
import org.apache.commons.io.IOUtils;
import org.apache.http.client.methods.CloseableHttpResponse;
@ -12,26 +13,20 @@ 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.JavaRDD;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.api.java.function.MapGroupsFunction;
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 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.*;
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.ISLookUpException;
import eu.dnetlib.enabling.is.lookup.rmi.ISLookUpService;
import eu.dnetlib.pace.config.DedupConfig;
import scala.Tuple2;
public class SparkPrepareOrgRels extends AbstractSparkAction {

View File

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

View File

@ -0,0 +1,18 @@
<configuration>
<property>
<name>jobTracker</name>
<value>yarnRM</value>
</property>
<property>
<name>nameNode</name>
<value>hdfs://nameservice1</value>
</property>
<property>
<name>oozie.use.system.libpath</name>
<value>true</value>
</property>
<property>
<name>oozie.action.sharelib.for.spark</name>
<value>spark2</value>
</property>
</configuration>

View File

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

View File

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

View File

@ -223,11 +223,11 @@ public class SparkDedupTest implements Serializable {
});
new SparkCollectSimRels(
parser,
spark,
spark.read().load(testDedupAssertionsBasePath + "/similarity_groups"),
spark.read().load(testDedupAssertionsBasePath + "/groups"))
.run(isLookUpService);
parser,
spark,
spark.read().load(testDedupAssertionsBasePath + "/similarity_groups"),
spark.read().load(testDedupAssertionsBasePath + "/groups"))
.run(isLookUpService);
long orgs_simrel = spark
.read()