forked from D-Net/dnet-hadoop
implementation of the raw wf for openorgs: still not complete, some functionalities are missing
This commit is contained in:
parent
112da6d76a
commit
1e1aab83e3
|
@ -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<Relation> simRels = spark.read().textFile(relationPath).map(patchRelFn(), Encoders.bean(Relation.class)).toJavaRDD().filter(r -> filterRels(r, entity));
|
||||
|
||||
simRels.saveAsTextFile(outputPath);
|
||||
}
|
||||
}
|
||||
|
||||
private static MapFunction<String, Relation> 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;
|
||||
}
|
||||
}
|
|
@ -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<Relation> mergeRelsRDD = spark
|
||||
.read()
|
||||
.load(mergeRelsPath)
|
||||
.as(Encoders.bean(Relation.class))
|
||||
.where("relClass == 'merges'")
|
||||
.toJavaRDD();
|
||||
|
||||
JavaRDD<Tuple2<Tuple2<String, String>, 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<Tuple2<Tuple2<String, String>, String>> flatMergeRels = mergeRelsRDD
|
||||
.mapToPair(rel -> new Tuple2<>(rel.getSource(), rel.getTarget()))
|
||||
.groupByKey()
|
||||
.flatMap(g -> {
|
||||
List<Tuple2<Tuple2<String,String>, String>> rels = new ArrayList<>();
|
||||
|
||||
List<String> 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<Relation> purgedMergeRels = flatMergeRels.union(diffRelsRDD)
|
||||
.mapToPair(rel -> new Tuple2<>(rel._1(), Arrays.asList(rel._2())))
|
||||
.reduceByKey((a, b) -> {
|
||||
List<String> list = new ArrayList<String>();
|
||||
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<Tuple2<String, String>> 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<String, Relation> 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<Relation> tupleToMergeRel(Tuple2<String, String> rel, DedupConfig dedupConf) {
|
||||
|
||||
List<Relation> 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;
|
||||
}
|
||||
}
|
|
@ -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>
|
|
@ -0,0 +1,324 @@
|
|||
<workflow-app name="Openorgs 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>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="resetOrgSimRels">
|
||||
<fs>
|
||||
<delete path="${workingPath}/${actionSetId}/organization_simrel"/>
|
||||
</fs>
|
||||
<ok to="copySimRels"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<!--extract simrels from relations and copy into the organization_simrel-->
|
||||
<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>
|
||||
<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>--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>
|
||||
|
||||
<action name="CreateSimRels">
|
||||
<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="CreateMergeRels"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="CreateMergeRels">
|
||||
<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="RemoveDifferences"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="RemoveDifferences">
|
||||
<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.SparkRemoveDiffRels</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="PrepareOrgRels"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="PrepareOrgRels">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Prepare Organization Relations</name>
|
||||
<class>eu.dnetlib.dhp.oa.dedup.SparkPrepareOrgRels</class>
|
||||
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
</spark-opts>
|
||||
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
|
||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
|
||||
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
|
||||
<arg>--dbUrl</arg><arg>${dbUrl}</arg>
|
||||
<arg>--dbTable</arg><arg>${dbTable}</arg>
|
||||
<arg>--dbUser</arg><arg>${dbUser}</arg>
|
||||
<arg>--dbPwd</arg><arg>${dbPwd}</arg>
|
||||
<arg>--numConnections</arg><arg>20</arg>
|
||||
</spark>
|
||||
<ok to="PrepareNewOrgs"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<action name="PrepareNewOrgs">
|
||||
<spark xmlns="uri:oozie:spark-action:0.2">
|
||||
<master>yarn</master>
|
||||
<mode>cluster</mode>
|
||||
<name>Prepare New Organizations</name>
|
||||
<class>eu.dnetlib.dhp.oa.dedup.SparkPrepareNewOrgs</class>
|
||||
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
|
||||
<spark-opts>
|
||||
--executor-memory=${sparkExecutorMemory}
|
||||
--executor-cores=${sparkExecutorCores}
|
||||
--driver-memory=${sparkDriverMemory}
|
||||
--conf spark.extraListeners=${spark2ExtraListeners}
|
||||
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
|
||||
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
|
||||
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
|
||||
--conf spark.sql.shuffle.partitions=3840
|
||||
</spark-opts>
|
||||
<arg>--graphBasePath</arg><arg>${graphBasePath}</arg>
|
||||
<arg>--workingPath</arg><arg>${workingPath}</arg>
|
||||
<arg>--isLookUpUrl</arg><arg>${isLookUpUrl}</arg>
|
||||
<arg>--actionSetId</arg><arg>${actionSetId}</arg>
|
||||
<arg>--apiUrl</arg><arg>${apiUrl}</arg>
|
||||
<arg>--dbUrl</arg><arg>${dbUrl}</arg>
|
||||
<arg>--dbTable</arg><arg>${dbTable}</arg>
|
||||
<arg>--dbUser</arg><arg>${dbUser}</arg>
|
||||
<arg>--dbPwd</arg><arg>${dbPwd}</arg>
|
||||
<arg>--numConnections</arg><arg>20</arg>
|
||||
</spark>
|
||||
<ok to="End"/>
|
||||
<error to="Kill"/>
|
||||
</action>
|
||||
|
||||
<end name="End"/>
|
||||
</workflow-app>
|
Loading…
Reference in New Issue