Merge pull request 'Priority to records from delegated authorities' (#187) from delegated_authorities into beta

Reviewed-on: #187
pull/192/head
Miriam Baglioni 2 years ago
commit a70b0990c9

@ -1,5 +1,5 @@
package eu.dnetlib.dhp.oa.dedup;
package eu.dnetlib.dhp.oa.merge;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
@ -38,7 +38,7 @@ public class DispatchEntitiesSparkJob {
.requireNonNull(
DispatchEntitiesSparkJob.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/dedup/dispatch_entities_parameters.json")));
"/eu/dnetlib/dhp/oa/merge/dispatch_entities_parameters.json")));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);

@ -1,5 +1,5 @@
package eu.dnetlib.dhp.oa.dedup;
package eu.dnetlib.dhp.oa.merge;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import static eu.dnetlib.dhp.utils.DHPUtils.toSeq;
@ -53,7 +53,7 @@ public class GroupEntitiesSparkJob {
.toString(
GroupEntitiesSparkJob.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/dedup/group_graph_entities_parameters.json"));
"/eu/dnetlib/dhp/oa/merge/group_graph_entities_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);

@ -47,6 +47,17 @@ public class OafMapperUtils {
}
public static Result mergeResults(Result left, Result right) {
final boolean leftFromDeletedAuthority = isFromDelegatedAuthority(left);
final boolean rightFromDeletedAuthority = isFromDelegatedAuthority(right);
if (leftFromDeletedAuthority && !rightFromDeletedAuthority) {
return left;
}
if (!leftFromDeletedAuthority && rightFromDeletedAuthority) {
return right;
}
if (new ResultTypeComparator().compare(left, right) < 0) {
left.mergeFrom(right);
return left;
@ -56,6 +67,18 @@ public class OafMapperUtils {
}
}
private static boolean isFromDelegatedAuthority(Result r) {
return Optional
.ofNullable(r.getInstance())
.map(
instance -> instance
.stream()
.filter(i -> Objects.nonNull(i.getCollectedfrom()))
.map(i -> i.getCollectedfrom().getKey())
.anyMatch(cfId -> IdentifierFactory.delegatedAuthorityDatasourceIds().contains(cfId)))
.orElse(false);
}
public static KeyValue keyValue(final String k, final String v) {
final KeyValue kv = new KeyValue();
kv.setKey(k);

@ -104,7 +104,7 @@
<master>yarn</master>
<mode>cluster</mode>
<name>group graph entities</name>
<class>eu.dnetlib.dhp.oa.dedup.GroupEntitiesSparkJob</class>
<class>eu.dnetlib.dhp.oa.merge.GroupEntitiesSparkJob</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
@ -138,7 +138,7 @@
<master>yarn</master>
<mode>cluster</mode>
<name>Dispatch publications</name>
<class>eu.dnetlib.dhp.oa.dedup.DispatchEntitiesSparkJob</class>
<class>eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
@ -163,7 +163,7 @@
<master>yarn</master>
<mode>cluster</mode>
<name>Dispatch project</name>
<class>eu.dnetlib.dhp.oa.dedup.DispatchEntitiesSparkJob</class>
<class>eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
@ -188,7 +188,7 @@
<master>yarn</master>
<mode>cluster</mode>
<name>Dispatch organization</name>
<class>eu.dnetlib.dhp.oa.dedup.DispatchEntitiesSparkJob</class>
<class>eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
@ -213,7 +213,7 @@
<master>yarn</master>
<mode>cluster</mode>
<name>Dispatch publication</name>
<class>eu.dnetlib.dhp.oa.dedup.DispatchEntitiesSparkJob</class>
<class>eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
@ -238,7 +238,7 @@
<master>yarn</master>
<mode>cluster</mode>
<name>Dispatch dataset</name>
<class>eu.dnetlib.dhp.oa.dedup.DispatchEntitiesSparkJob</class>
<class>eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
@ -263,7 +263,7 @@
<master>yarn</master>
<mode>cluster</mode>
<name>Dispatch software</name>
<class>eu.dnetlib.dhp.oa.dedup.DispatchEntitiesSparkJob</class>
<class>eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
@ -288,7 +288,7 @@
<master>yarn</master>
<mode>cluster</mode>
<name>Dispatch otherresearchproduct</name>
<class>eu.dnetlib.dhp.oa.dedup.DispatchEntitiesSparkJob</class>
<class>eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob</class>
<jar>dhp-dedup-openaire-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}

@ -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,298 @@
<workflow-app name="Group Graph Entities" xmlns="uri:oozie:workflow:0.5">
<parameters>
<property>
<name>graphBasePath</name>
<description>the input graph base path</description>
</property>
<property>
<name>workingPath</name>
<description>path of the working directory</description>
</property>
<property>
<name>graphOutputPath</name>
<description>path of the output graph</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="group_entities"/>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<action name="group_entities">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>group graph entities</name>
<class>eu.dnetlib.dhp.oa.merge.GroupEntitiesSparkJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
--executor-memory=${sparkExecutorMemory}
--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=15000
</spark-opts>
<arg>--graphInputPath</arg><arg>${graphBasePath}</arg>
<arg>--outputPath</arg><arg>${workingPath}/grouped_entities</arg>
</spark>
<ok to="fork_dispatch_entities"/>
<error to="Kill"/>
</action>
<fork name="fork_dispatch_entities">
<path start="dispatch_datasource"/>
<path start="dispatch_project"/>
<path start="dispatch_organization"/>
<path start="dispatch_publication"/>
<path start="dispatch_dataset"/>
<path start="dispatch_software"/>
<path start="dispatch_otherresearchproduct"/>
</fork>
<action name="dispatch_datasource">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dispatch publications</name>
<class>eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
--executor-memory=${sparkExecutorMemory}
--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
</spark-opts>
<arg>--inputPath</arg><arg>${workingPath}/grouped_entities</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}/datasource</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Datasource</arg>
</spark>
<ok to="wait_dispatch"/>
<error to="Kill"/>
</action>
<action name="dispatch_project">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dispatch project</name>
<class>eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
--executor-memory=${sparkExecutorMemory}
--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
</spark-opts>
<arg>--inputPath</arg><arg>${workingPath}/grouped_entities</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}/project</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Project</arg>
</spark>
<ok to="wait_dispatch"/>
<error to="Kill"/>
</action>
<action name="dispatch_organization">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dispatch organization</name>
<class>eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
--executor-memory=${sparkExecutorMemory}
--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
</spark-opts>
<arg>--inputPath</arg><arg>${workingPath}/grouped_entities</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}/organization</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Organization</arg>
</spark>
<ok to="wait_dispatch"/>
<error to="Kill"/>
</action>
<action name="dispatch_publication">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dispatch publication</name>
<class>eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
--executor-memory=${sparkExecutorMemory}
--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
</spark-opts>
<arg>--inputPath</arg><arg>${workingPath}/grouped_entities</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}/publication</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Publication</arg>
</spark>
<ok to="wait_dispatch"/>
<error to="Kill"/>
</action>
<action name="dispatch_dataset">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dispatch dataset</name>
<class>eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
--executor-memory=${sparkExecutorMemory}
--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
</spark-opts>
<arg>--inputPath</arg><arg>${workingPath}/grouped_entities</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}/dataset</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Dataset</arg>
</spark>
<ok to="wait_dispatch"/>
<error to="Kill"/>
</action>
<action name="dispatch_software">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dispatch software</name>
<class>eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
--executor-memory=${sparkExecutorMemory}
--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
</spark-opts>
<arg>--inputPath</arg><arg>${workingPath}/grouped_entities</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}/software</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.Software</arg>
</spark>
<ok to="wait_dispatch"/>
<error to="Kill"/>
</action>
<action name="dispatch_otherresearchproduct">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dispatch otherresearchproduct</name>
<class>eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob</class>
<jar>dhp-graph-mapper-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=${sparkExecutorCores}
--executor-memory=${sparkExecutorMemory}
--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
</spark-opts>
<arg>--inputPath</arg><arg>${workingPath}/grouped_entities</arg>
<arg>--outputPath</arg><arg>${graphOutputPath}/otherresearchproduct</arg>
<arg>--graphTableClassName</arg><arg>eu.dnetlib.dhp.schema.oaf.OtherResearchProduct</arg>
</spark>
<ok to="wait_dispatch"/>
<error to="Kill"/>
</action>
<join name="wait_dispatch" to="copy_relation"/>
<action name="copy_relation">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<arg>${nameNode}/${graphBasePath}/relation</arg>
<arg>${nameNode}/${graphOutputPath}/relation</arg>
</distcp>
<ok to="End"/>
<error to="Kill"/>
</action>
<end name="End"/>
</workflow-app>

@ -0,0 +1,144 @@
package eu.dnetlib.dhp.oa.graph.group;
import static org.junit.jupiter.api.Assertions.*;
import java.io.IOException;
import java.net.URISyntaxException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang3.StringUtils;
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.SparkSession;
import org.junit.jupiter.api.*;
import com.fasterxml.jackson.databind.DeserializationFeature;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists;
import eu.dnetlib.dhp.common.HdfsSupport;
import eu.dnetlib.dhp.oa.merge.DispatchEntitiesSparkJob;
import eu.dnetlib.dhp.oa.merge.GroupEntitiesSparkJob;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.Publication;
import eu.dnetlib.dhp.schema.oaf.Result;
import eu.dnetlib.dhp.utils.DHPUtils;
@TestMethodOrder(MethodOrderer.OrderAnnotation.class)
public class GroupEntitiesSparkJobTest {
private static SparkSession spark;
private static ObjectMapper mapper = new ObjectMapper()
.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
private static Path workingDir;
private Path dataInputPath;
private Path groupEntityPath;
private Path dispatchEntityPath;
@BeforeAll
public static void beforeAll() throws IOException {
workingDir = Files.createTempDirectory(GroupEntitiesSparkJob.class.getSimpleName());
SparkConf conf = new SparkConf();
conf.setAppName(GroupEntitiesSparkJob.class.getSimpleName());
conf.setMaster("local");
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
conf.registerKryoClasses(ModelSupport.getOafModelClasses());
spark = SparkSession.builder().config(conf).getOrCreate();
}
@BeforeEach
public void beforeEach() throws IOException, URISyntaxException {
dataInputPath = Paths.get(ClassLoader.getSystemResource("eu/dnetlib/dhp/oa/graph/group").toURI());
groupEntityPath = workingDir.resolve("grouped_entity");
dispatchEntityPath = workingDir.resolve("dispatched_entity");
}
@AfterAll
public static void afterAll() throws IOException {
spark.stop();
FileUtils.deleteDirectory(workingDir.toFile());
}
@Test
@Order(1)
void testGroupEntities() throws Exception {
GroupEntitiesSparkJob.main(new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-graphInputPath",
dataInputPath.toString(),
"-outputPath",
groupEntityPath.toString()
});
Dataset<Result> output = spark
.read()
.textFile(groupEntityPath.toString())
.map((MapFunction<String, String>) s -> StringUtils.substringAfter(s, "|"), Encoders.STRING())
.map((MapFunction<String, Result>) s -> mapper.readValue(s, Result.class), Encoders.bean(Result.class));
assertEquals(
1,
output
.filter(
(FilterFunction<Result>) r -> "50|doi_________::09821844208a5cd6300b2bfb13bca1b9"
.equals(r.getId()) &&
r.getCollectedfrom().stream().anyMatch(kv -> kv.getValue().equalsIgnoreCase("zenodo")))
.count());
}
@Test
@Order(2)
void testDispatchEntities() throws Exception {
for (String type : Lists
.newArrayList(
Publication.class.getCanonicalName(), eu.dnetlib.dhp.schema.oaf.Dataset.class.getCanonicalName())) {
String directory = StringUtils.substringAfterLast(type, ".").toLowerCase();
DispatchEntitiesSparkJob.main(new String[] {
"-isSparkSessionManaged",
Boolean.FALSE.toString(),
"-inputPath",
groupEntityPath.toString(),
"-outputPath",
dispatchEntityPath.resolve(directory).toString(),
"-graphTableClassName",
type
});
}
Dataset<Result> output = spark
.read()
.textFile(
DHPUtils
.toSeq(
HdfsSupport
.listFiles(dispatchEntityPath.toString(), spark.sparkContext().hadoopConfiguration())))
.map((MapFunction<String, Result>) s -> mapper.readValue(s, Result.class), Encoders.bean(Result.class));
assertEquals(3, output.count());
assertEquals(
2,
output
.map((MapFunction<Result, String>) r -> r.getResulttype().getClassid(), Encoders.STRING())
.filter((FilterFunction<String>) s -> s.equals("publication"))
.count());
assertEquals(
1,
output
.map((MapFunction<Result, String>) r -> r.getResulttype().getClassid(), Encoders.STRING())
.filter((FilterFunction<String>) s -> s.equals("dataset"))
.count());
}
}

@ -4,7 +4,7 @@
<RESOURCE_TYPE value="WorkflowDSResourceType"/>
<RESOURCE_KIND value="WorkflowDSResources"/>
<RESOURCE_URI value=""/>
<DATE_OF_CREATION value="2021-12-23T14:32:39+00:00"/>
<DATE_OF_CREATION value="2022-01-11T20:04:48+00:00"/>
</HEADER>
<BODY>
<WORKFLOW_NAME>Graph processing [EXPERIMENT]</WORKFLOW_NAME>
@ -15,7 +15,7 @@
<DESCRIPTION>set the path of unresolved entities</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">unresolvedEntityPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/data/unresolved_BETA</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/data/unresolved_BETA/content</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -51,6 +51,16 @@
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setIterationNumbers" type="SetEnvParameter">
<DESCRIPTION>set the number of iteration in affiliation propagation</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">iterations</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">1</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setMergedGraphPath" type="SetEnvParameter">
<DESCRIPTION>Set the target path to store the MERGED graph</DESCRIPTION>
<PARAMETERS>
@ -91,11 +101,21 @@
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setGroupedGraphPath" type="SetEnvParameter">
<DESCRIPTION>Set the target path to store the GROUPED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">groupedGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/05_graph_grouped</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setInferredGraphPath" type="SetEnvParameter">
<DESCRIPTION>Set the target path to store the INFERRED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">inferredGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/05_graph_inferred</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/06_graph_inferred</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -105,7 +125,7 @@
<DESCRIPTION>Set the target path to store the DEDUPED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">dedupGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/06_graph_dedup</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/07_graph_dedup</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -115,7 +135,7 @@
<DESCRIPTION>Set the target path to store the CONSISTENCY graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">consistentGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/07_graph_consistent</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/08_graph_consistent</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -125,7 +145,7 @@
<DESCRIPTION>Set the target path to store the ORCID enriched graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">orcidGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/08_graph_orcid</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/09_graph_orcid</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -135,7 +155,7 @@
<DESCRIPTION>Set the target path to store the BULK TAGGED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">bulkTaggingGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/09_graph_bulktagging</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/10_graph_bulktagging</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -145,7 +165,7 @@
<DESCRIPTION>Set the target path to store the AFFILIATION from INSTITUTIONAL REPOS graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">affiliationGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/10_graph_affiliation</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/11_graph_affiliation</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -155,7 +175,7 @@
<DESCRIPTION>Set the target path to store the AFFILIATION from SEMATIC RELATION graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">affiliationSemRelGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/11_graph_affiliationsr</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/12_graph_affiliationsr</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -165,7 +185,7 @@
<DESCRIPTION>Set the target path to store the COMMUNITY from SELECTED SOURCES graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">communityOrganizationGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/12_graph_community_organization</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/13_graph_community_organization</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -175,7 +195,7 @@
<DESCRIPTION>Set the target path to store the FUNDING from SEMANTIC RELATION graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">fundingGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/13_graph_funding</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/14_graph_funding</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -185,7 +205,7 @@
<DESCRIPTION>Set the target path to store the COMMUNITY from SEMANTIC RELATION graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">communitySemRelGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/14_graph_community_sem_rel</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/15_graph_community_sem_rel</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -195,7 +215,7 @@
<DESCRIPTION>Set the target path to store the COUNTRY enriched graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">countryGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/15_graph_country</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/16_graph_country</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -205,7 +225,7 @@
<DESCRIPTION>Set the target path to store the CLEANED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">cleanedGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/16_graph_cleaned</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/17_graph_cleaned</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -215,7 +235,7 @@
<DESCRIPTION>Set the target path to store the blacklisted graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">blacklistedGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/17_graph_blacklisted</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_experiment/graph/18_graph_blacklisted</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -548,14 +568,14 @@
'mongoURL' : 'mongodb://beta.services.openaire.eu',
'mongoDb' : 'mdstore',
'mdstoreManagerUrl' : 'https://beta.services.openaire.eu/mdstoremanager',
'postgresURL' : '',
'postgresURL' : 'jdbc:postgresql://beta.services.openaire.eu:5432/dnet_openaireplus',
'postgresUser' : '',
'postgresPassword' : '',
'postgresOpenOrgsURL' : '',
'postgresOpenOrgsURL' : 'jdbc:postgresql://10.19.65.40:5432/oa_organizations',
'postgresOpenOrgsUser' : '',
'postgresOpenOrgsPassword' : '',
'shouldHashId' : 'true',
'importOpenorgs' : 'true',
'importOpenorgs' : 'false',
'workingDir' : '/tmp/beta_experiment/working_dir/beta_aggregator'
}
</PARAM>
@ -594,10 +614,10 @@
'mongoURL' : 'mongodb://services.openaire.eu',
'mongoDb' : 'mdstore',
'mdstoreManagerUrl' : 'https://services.openaire.eu/mdstoremanager',
'postgresURL' : '',
'postgresURL' : 'jdbc:postgresql://postgresql.services.openaire.eu:5432/dnet_openaireplus',
'postgresUser' : '',
'postgresPassword' : '',
'postgresOpenOrgsURL' : '',
'postgresOpenOrgsURL' : 'jdbc:postgresql://10.19.65.39:5432/oa_organizations',
'postgresOpenOrgsUser' : '',
'postgresOpenOrgsPassword' : '',
'shouldHashId' : 'true',
@ -737,11 +757,11 @@
<PARAM managedBy="system" name="hadoopJob" required="true" type="string">executeOozieJob</PARAM>
<PARAM managedBy="system" name="cluster" required="true" type="string">IIS</PARAM>
<PARAM managedBy="system" name="envParams" required="true" type="string">
{
{
'graphBasePath':'cleanedFirstGraphPath',
'unresolvedPath' :'unresolvedEntityPath',
'targetPath':'resolvedGraphPath'
}
}
</PARAM>
<PARAM managedBy="system" name="params" required="true" type="string">
{
@ -752,6 +772,30 @@
}
</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="graphEntityGrouping"/>
</ARCS>
</NODE>
<NODE name="graphEntityGrouping" type="SubmitHadoopJob">
<DESCRIPTION>Resolve Relation</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="hadoopJob" required="true" type="string">executeOozieJob</PARAM>
<PARAM managedBy="system" name="cluster" required="true" type="string">IIS</PARAM>
<PARAM managedBy="system" name="envParams" required="true" type="string">
{
'graphBasePath':'resolvedGraphPath',
'targetPath':'groupedGraphPath'
}
</PARAM>
<PARAM managedBy="system" name="params" required="true" type="string">
{
'oozie.wf.application.path' : '/lib/dnet/BETA/oa/graph/group/oozie_app',
'workingDir' : '/tmp/beta_experiment/working_dir/grouping',
'sparkExecutorCores' : '4',
'sparkExecutorMemory' : '7G'
}
</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="prepareIISActionSets"/>
</ARCS>
@ -867,9 +911,9 @@
<PARAM managedBy="system" name="mdStoreStoreLocationParam" required="true" type="string">import_mdstore_service_location</PARAM>
<PARAM managedBy="system" name="mdStoreDatasetParam" required="true" type="string">import_dataset_mdstore_ids_csv</PARAM>
<PARAM managedBy="system" name="oozieWfAppPathParam" required="true" type="string">oozie.wf.application.path</PARAM>
<PARAM managedBy="user" name="oozieWfAppPath" required="true" type="string">/lib/iis/primary/snapshots/2021-09-24</PARAM>
<PARAM managedBy="user" name="oozieWfAppPath" required="true" type="string">/lib/iis/primary/snapshots/2021-12-09</PARAM>
<PARAM function="validValues(['IIS','DM'])" managedBy="user" name="clusterName" required="true" type="string">IIS</PARAM>
<PARAM managedBy="user" name="importHbaseDumpLocation" required="true" type="string">/tmp/beta_inference/graph/07_graph_cleaned</PARAM>
<PARAM managedBy="user" name="importHbaseDumpLocation" required="true" type="string">deprecated - not used</PARAM>
<PARAM managedBy="system" name="importHbaseDumpLocationParam" required="true" type="string">import_infospace_graph_location</PARAM>
<PARAM managedBy="user" name="objectStoreBlacklistCSV" required="false" type="string"/>
<PARAM managedBy="system" name="importProjectConceptsContextCSVParam" required="true" type="string">import_project_concepts_context_ids_csv</PARAM>
@ -908,7 +952,7 @@
'import_islookup_service_location' : 'import_islookup_service_location',
'import_project_concepts_context_ids_csv' : 'import_project_concepts_context_ids_csv',
'import_dataset_mdstore_ids_csv' : 'import_dataset_mdstore_ids_csv',
'import_infospace_graph_location' : 'import_infospace_graph_location',
'import_infospace_graph_location' : 'groupedGraphPath',
'export_action_set_id_matched_doc_organizations' : 'export_action_set_id_matched_doc_organizations',
'export_action_set_id_document_referencedDatasets' : 'export_action_set_id_document_referencedDatasets',
@ -958,7 +1002,7 @@
<PARAM managedBy="system" name="envParams" required="true" type="string">
{
'inputActionSetIds' : 'actionSetIdsIISGraph',
'inputGraphRootPath' : 'resolvedGraphPath',
'inputGraphRootPath' : 'groupedGraphPath',
'outputGraphRootPath' : 'inferredGraphPath',
'isLookupUrl' : 'isLookUpUrl'
}
@ -1125,7 +1169,8 @@
<PARAM managedBy="system" name="envParams" required="true" type="string">
{
'sourcePath' : 'affiliationGraphPath',
'outputPath': 'affiliationSemRelGraphPath'
'outputPath': 'affiliationSemRelGraphPath',
'iterations':'iterations'
}
</PARAM>
<PARAM managedBy="system" name="params" required="true" type="string">
@ -1283,7 +1328,7 @@
{
'oozie.wf.application.path' : '/lib/dnet/BETA/oa/enrichment/blacklist/oozie_app',
'workingDir' : '/tmp/beta_experiment/working_dir/blacklist',
'postgresURL' : '',
'postgresURL' : 'jdbc:postgresql://beta.services.openaire.eu:5432/dnet_openaireplus',
'postgresUser' : '',
'postgresPassword' : ''
}
@ -1296,10 +1341,10 @@
</NODE>
</CONFIGURATION>
<STATUS>
<LAST_EXECUTION_ID>wf_20211206_093743_83</LAST_EXECUTION_ID>
<LAST_EXECUTION_DATE>2021-12-06T10:12:32+00:00</LAST_EXECUTION_DATE>
<LAST_EXECUTION_STATUS>SUCCESS</LAST_EXECUTION_STATUS>
<LAST_EXECUTION_ERROR/>
<LAST_EXECUTION_ID>wf_20220111_200505_785</LAST_EXECUTION_ID>
<LAST_EXECUTION_DATE>2022-01-11T20:08:53+00:00</LAST_EXECUTION_DATE>
<LAST_EXECUTION_STATUS></LAST_EXECUTION_STATUS>
<LAST_EXECUTION_ERROR></LAST_EXECUTION_ERROR>
</STATUS>
</BODY>
</RESOURCE_PROFILE>
</RESOURCE_PROFILE>

@ -11,6 +11,16 @@
<WORKFLOW_TYPE>IIS</WORKFLOW_TYPE>
<WORKFLOW_PRIORITY>30</WORKFLOW_PRIORITY>
<CONFIGURATION start="manual">
<NODE isStart="true" name="setUnresolvedEntityPath" type="SetEnvParameter">
<DESCRIPTION>set the path of unresolved entities</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">unresolvedEntityPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/data/unresolved_BETA/content</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setNsPrefixBlacklistForBETA" type="SetEnvParameter">
<DESCRIPTION>set blacklist of funder nsPrefixes from the beta aggregator</DESCRIPTION>
<PARAMETERS>
@ -71,11 +81,31 @@
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setResolvedGraphPath" type="SetEnvParameter">
<DESCRIPTION>Set the target path to store the CLEANED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">resolvedGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_inference/graph/04_graph_resolved</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setGroupedGraphPath" type="SetEnvParameter">
<DESCRIPTION>Set the target path to store the GROUPED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">groupedGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_inference/graph/05_graph_grouped</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setDedupGraphPath" type="SetEnvParameter">
<DESCRIPTION>Set the target path to store the DEDUPED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">dedupGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_inference/graph/04_graph_dedup</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_inference/graph/06_graph_dedup</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -85,7 +115,7 @@
<DESCRIPTION>Set the target path to store the CONSISTENCY graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">consistentGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_inference/graph/05_graph_consistent</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_inference/graph/07_graph_consistent</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -95,7 +125,7 @@
<DESCRIPTION>Set the target path to store the CLEANED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">cleanedGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_inference/graph/06_graph_cleaned</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_inference/graph/08_graph_cleaned</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -548,6 +578,55 @@
</PARAM>
<PARAM managedBy="system" name="oozieReportActionsCsv" required="true" type="string">build-report</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="graphResolution"/>
</ARCS>
</NODE>
<NODE name="graphResolution" type="SubmitHadoopJob">
<DESCRIPTION>Resolve Relation</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="hadoopJob" required="true" type="string">executeOozieJob</PARAM>
<PARAM managedBy="system" name="cluster" required="true" type="string">IIS</PARAM>
<PARAM managedBy="system" name="envParams" required="true" type="string">
{
'graphBasePath':'cleanedFirstGraphPath',
'unresolvedPath' :'unresolvedEntityPath',
'targetPath':'resolvedGraphPath'
}
</PARAM>
<PARAM managedBy="system" name="params" required="true" type="string">
{
'oozie.wf.application.path' : '/lib/dnet/BETA/oa/graph/resolution/oozie_app',
'workingDir' : '/tmp/beta_inference/working_dir/relation_resolution',
'sparkExecutorCores' : '2',
'sparkExecutorMemory' : '12G'
}
</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="graphEntityGrouping"/>
</ARCS>
</NODE>
<NODE name="graphEntityGrouping" type="SubmitHadoopJob">
<DESCRIPTION>Resolve Relation</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="hadoopJob" required="true" type="string">executeOozieJob</PARAM>
<PARAM managedBy="system" name="cluster" required="true" type="string">IIS</PARAM>
<PARAM managedBy="system" name="envParams" required="true" type="string">
{
'graphBasePath':'resolvedGraphPath',
'targetPath':'groupedGraphPath'
}
</PARAM>
<PARAM managedBy="system" name="params" required="true" type="string">
{
'oozie.wf.application.path' : '/lib/dnet/BETA/oa/graph/group/oozie_app',
'workingDir' : '/tmp/beta_inference/working_dir/grouping',
'sparkExecutorCores' : '4',
'sparkExecutorMemory' : '7G'
}
</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="duplicateScan"/>
</ARCS>
@ -560,7 +639,7 @@
<PARAM managedBy="system" name="envParams" required="true" type="string">
{
'actionSetId' : 'dedupConfig',
'graphBasePath' : 'cleanedFirstGraphPath',
'graphBasePath' : 'groupedGraphPath',
'dedupGraphPath': 'dedupGraphPath',
'isLookUpUrl' : 'isLookUpUrl'
}

@ -11,6 +11,16 @@
<WORKFLOW_TYPE>IIS</WORKFLOW_TYPE>
<WORKFLOW_PRIORITY>30</WORKFLOW_PRIORITY>
<CONFIGURATION start="manual">
<NODE isStart="true" name="setUnresolvedEntityPath" type="SetEnvParameter">
<DESCRIPTION>set the path of unresolved entities</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">unresolvedEntityPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/data/unresolved_PROD/content</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setNsPrefixBlacklist" type="SetEnvParameter">
<DESCRIPTION>set blacklist of funder nsPrefixes</DESCRIPTION>
<PARAMETERS>
@ -61,11 +71,21 @@
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setResolvedGraphPath" type="SetEnvParameter">
<DESCRIPTION>Set the target path to store the CLEANED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">resolvedGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_inference/graph/03_graph_resolved</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setDedupGraphPath" type="SetEnvParameter">
<DESCRIPTION>Set the target path to store the DEDUPED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">dedupGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_inference/graph/03_graph_dedup</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_inference/graph/04_graph_dedup</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -75,7 +95,7 @@
<DESCRIPTION>Set the target path to store the CONSISTENCY graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">consistentGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_inference/graph/04_graph_consistent</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_inference/graph/05_graph_consistent</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -85,7 +105,7 @@
<DESCRIPTION>Set the target path to store the CLEANED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">cleanedGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_inference/graph/05_graph_cleaned</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_inference/graph/06_graph_cleaned</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -347,6 +367,31 @@
</PARAM>
<PARAM managedBy="system" name="oozieReportActionsCsv" required="true" type="string">build-report</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="graphResolution"/>
</ARCS>
</NODE>
<NODE name="graphResolution" type="SubmitHadoopJob">
<DESCRIPTION>Resolve Relation</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="hadoopJob" required="true" type="string">executeOozieJob</PARAM>
<PARAM managedBy="system" name="cluster" required="true" type="string">IIS</PARAM>
<PARAM managedBy="system" name="envParams" required="true" type="string">
{
'graphBasePath':'cleanedFirstGraphPath',
'unresolvedPath' :'unresolvedEntityPath',
'targetPath':'resolvedGraphPath'
}
</PARAM>
<PARAM managedBy="system" name="params" required="true" type="string">
{
'oozie.wf.application.path' : '/lib/dnet/PROD/oa/graph/resolution/oozie_app',
'workingDir' : '/tmp/prod_inference/working_dir/relation_resolution',
'sparkExecutorCores' : '2',
'sparkExecutorMemory' : '12G'
}
</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="duplicateScan"/>
</ARCS>
@ -359,7 +404,7 @@
<PARAM managedBy="system" name="envParams" required="true" type="string">
{
'actionSetId' : 'dedupConfig',
'graphBasePath' : 'cleanedFirstGraphPath',
'graphBasePath' : 'resolvedGraphPath',
'dedupGraphPath': 'dedupGraphPath',
'isLookUpUrl' : 'isLookUpUrl'
}

@ -11,6 +11,16 @@
<WORKFLOW_TYPE>Data Provision</WORKFLOW_TYPE>
<WORKFLOW_PRIORITY>30</WORKFLOW_PRIORITY>
<CONFIGURATION start="manual">
<NODE isStart="true" name="setUnresolvedEntityPath" type="SetEnvParameter">
<DESCRIPTION>set the path of unresolved entities</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">unresolvedEntityPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/data/unresolved_BETA/content</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setNsPrefixBlacklistForBETA" type="SetEnvParameter">
<DESCRIPTION>set blacklist of funder nsPrefixes from the beta aggregator</DESCRIPTION>
<PARAMETERS>
@ -71,11 +81,31 @@
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setResolvedGraphPath" type="SetEnvParameter">
<DESCRIPTION>Set the target path to store the CLEANED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">resolvedGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/04_graph_resolved</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setGroupedGraphPath" type="SetEnvParameter">
<DESCRIPTION>Set the target path to store the GROUPED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">groupedGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/05_graph_grouped</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setDedupGraphPath" type="SetEnvParameter">
<DESCRIPTION>Set the target path to store the DEDUPED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">dedupGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/04_graph_dedup</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/06_graph_dedup</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -85,7 +115,7 @@
<DESCRIPTION>Set the target path to store the INFERRED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">inferredGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/05_graph_inferred</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/07_graph_inferred</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -95,7 +125,7 @@
<DESCRIPTION>Set the target path to store the CONSISTENCY graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">consistentGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/06_graph_consistent</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/08_graph_consistent</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -105,7 +135,7 @@
<DESCRIPTION>Set the target path to store the ORCID enriched graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">orcidGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/07_graph_orcid</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/09_graph_orcid</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -115,7 +145,7 @@
<DESCRIPTION>Set the target path to store the BULK TAGGED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">bulkTaggingGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/08_graph_bulktagging</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/10_graph_bulktagging</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -125,7 +155,17 @@
<DESCRIPTION>Set the target path to store the AFFILIATION from INSTITUTIONAL REPOS graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">affiliationGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/09_graph_affiliation</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/11_graph_affiliation</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setAffiliationSemRelGraphPath" type="SetEnvParameter">
<DESCRIPTION>Set the target path to store the AFFILIATION from SEMATIC RELATION graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">affiliationSemRelGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/12_graph_affiliationsr</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -135,7 +175,7 @@
<DESCRIPTION>Set the target path to store the COMMUNITY from SELECTED SOURCES graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">communityOrganizationGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/10_graph_comunity_organization</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/13_graph_comunity_organization</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -145,7 +185,7 @@
<DESCRIPTION>Set the target path to store the FUNDING from SEMANTIC RELATION graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">fundingGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/11_graph_funding</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/14_graph_funding</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -155,7 +195,7 @@
<DESCRIPTION>Set the target path to store the COMMUNITY from SEMANTIC RELATION graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">communitySemRelGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/12_graph_comunity_sem_rel</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/15_graph_comunity_sem_rel</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -165,7 +205,7 @@
<DESCRIPTION>Set the target path to store the COUNTRY enriched graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">countryGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/13_graph_country</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/16_graph_country</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -175,7 +215,7 @@
<DESCRIPTION>Set the target path to store the CLEANED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">cleanedGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/14_graph_cleaned</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/17_graph_cleaned</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -185,7 +225,7 @@
<DESCRIPTION>Set the target path to store the blacklisted graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">blacklistedGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/15_graph_blacklisted</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_provision/graph/18_graph_blacklisted</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -695,6 +735,55 @@
</PARAM>
<PARAM managedBy="system" name="oozieReportActionsCsv" required="true" type="string">build-report</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="graphResolution"/>
</ARCS>
</NODE>
<NODE name="graphResolution" type="SubmitHadoopJob">
<DESCRIPTION>Resolve Relation</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="hadoopJob" required="true" type="string">executeOozieJob</PARAM>
<PARAM managedBy="system" name="cluster" required="true" type="string">IIS</PARAM>
<PARAM managedBy="system" name="envParams" required="true" type="string">
{
'graphBasePath':'cleanedFirstGraphPath',
'unresolvedPath' :'unresolvedEntityPath',
'targetPath':'resolvedGraphPath'
}
</PARAM>
<PARAM managedBy="system" name="params" required="true" type="string">
{
'oozie.wf.application.path' : '/lib/dnet/BETA/oa/graph/resolution/oozie_app',
'workingDir' : '/tmp/beta_provision/working_dir/relation_resolution',
'sparkExecutorCores' : '2',
'sparkExecutorMemory' : '12G'
}
</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="graphEntityGrouping"/>
</ARCS>
</NODE>
<NODE name="graphEntityGrouping" type="SubmitHadoopJob">
<DESCRIPTION>Resolve Relation</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="hadoopJob" required="true" type="string">executeOozieJob</PARAM>
<PARAM managedBy="system" name="cluster" required="true" type="string">IIS</PARAM>
<PARAM managedBy="system" name="envParams" required="true" type="string">
{
'graphBasePath':'resolvedGraphPath',
'targetPath':'groupedGraphPath'
}
</PARAM>
<PARAM managedBy="system" name="params" required="true" type="string">
{
'oozie.wf.application.path' : '/lib/dnet/BETA/oa/graph/group/oozie_app',
'workingDir' : '/tmp/beta_provision/working_dir/grouping',
'sparkExecutorCores' : '4',
'sparkExecutorMemory' : '7G'
}
</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="duplicateScan"/>
</ARCS>
@ -707,7 +796,7 @@
<PARAM managedBy="system" name="envParams" required="true" type="string">
{
'actionSetId' : 'dedupConfig',
'graphBasePath' : 'cleanedFirstGraphPath',
'graphBasePath' : 'groupedGraphPath',
'dedupGraphPath': 'dedupGraphPath',
'isLookUpUrl' : 'isLookUpUrl'
}

@ -11,6 +11,16 @@
<WORKFLOW_TYPE>Data Provision</WORKFLOW_TYPE>
<WORKFLOW_PRIORITY>30</WORKFLOW_PRIORITY>
<CONFIGURATION start="manual">
<NODE isStart="true" name="setUnresolvedEntityPath" type="SetEnvParameter">
<DESCRIPTION>set the path of unresolved entities</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">unresolvedEntityPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/data/unresolved_PROD/content</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setNsPrefixBlacklist" type="SetEnvParameter">
<DESCRIPTION>set blacklist of funder nsPrefixes</DESCRIPTION>
<PARAMETERS>
@ -51,11 +61,21 @@
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setResolvedGraphPath" type="SetEnvParameter">
<DESCRIPTION>Set the target path to store the CLEANED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">resolvedGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/beta_inference/graph/03_graph_resolved</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
</ARCS>
</NODE>
<NODE isStart="true" name="setDedupGraphPath" type="SetEnvParameter">
<DESCRIPTION>Set the target path to store the DEDUPED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">dedupGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/03_graph_dedup</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/04_graph_dedup</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -65,7 +85,7 @@
<DESCRIPTION>Set the target path to store the INFERRED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">inferredGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/04_graph_inferred</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/05_graph_inferred</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -75,7 +95,7 @@
<DESCRIPTION>Set the target path to store the CONSISTENCY graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">consistentGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/05_graph_consistent</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/06_graph_consistent</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -85,7 +105,7 @@
<DESCRIPTION>Set the target path to store the ORCID enriched graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">orcidGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/06_graph_orcid</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/07_graph_orcid</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -95,7 +115,7 @@
<DESCRIPTION>Set the target path to store the BULK TAGGED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">bulkTaggingGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/07_graph_bulktagging</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/08_graph_bulktagging</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -105,7 +125,7 @@
<DESCRIPTION>Set the target path to store the AFFILIATION from INSTITUTIONAL REPOS graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">affiliationGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/08_graph_affiliation</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/09_graph_affiliation</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -115,7 +135,7 @@
<DESCRIPTION>Set the target path to store the COMMUNITY from SELECTED SOURCES graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">communityOrganizationGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/09_graph_comunity_organization</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/10_graph_comunity_organization</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -125,7 +145,7 @@
<DESCRIPTION>Set the target path to store the FUNDING from SEMANTIC RELATION graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">fundingGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/10_graph_funding</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/11_graph_funding</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -135,7 +155,7 @@
<DESCRIPTION>Set the target path to store the COMMUNITY from SEMANTIC RELATION graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">communitySemRelGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/11_graph_comunity_sem_rel</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/12_graph_comunity_sem_rel</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -145,7 +165,7 @@
<DESCRIPTION>Set the target path to store the COUNTRY enriched graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">countryGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/12_graph_country</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/13_graph_country</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -155,7 +175,7 @@
<DESCRIPTION>Set the target path to store the CLEANED graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">cleanedGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/13_graph_cleaned</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/14_graph_cleaned</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -165,7 +185,7 @@
<DESCRIPTION>Set the target path to store the blacklisted graph</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="parameterName" required="true" type="string">blacklistedGraphPath</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/14_graph_blacklisted</PARAM>
<PARAM managedBy="user" name="parameterValue" required="true" type="string">/tmp/prod_provision/graph/15_graph_blacklisted</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="waitConfig"/>
@ -446,6 +466,59 @@
</PARAM>
<PARAM managedBy="system" name="oozieReportActionsCsv" required="true" type="string">build-report</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="patchHostedBy"/>
</ARCS>
</NODE>
<NODE name="patchHostedBy" type="SubmitHadoopJob">
<DESCRIPTION>updates publication's hostedby info according to the ISSNs available from DOAJ and UNIBI</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="hadoopJob" required="true" type="string">executeOozieJob</PARAM>
<PARAM managedBy="system" name="cluster" required="true" type="string">IIS</PARAM>
<PARAM managedBy="system" name="envParams" required="true" type="string">
{
'sourcePath' : 'cleanedFirstGraphPath'
}
</PARAM>
<PARAM managedBy="system" name="params" required="true" type="string">
{
'resumeFrom' : 'prepareInfo',
'hostedByMapPath' : '/user/dnet.production/data/hostedByMap',
'oozie.wf.application.path' : '/lib/dnet/PROD/oa/graph/hostedbymap/oozie_app',
'workingDir' : '/tmp/prod_provision/working_dir/hostedbymap',
'outputPath' : '/tmp/prod_provision/working_dir/hostedbymap',
'sparkExecutorCores' : '3',
'sparkExecutorMemory' : '10G'
}
</PARAM>
<PARAM managedBy="system" name="oozieReportActionsCsv" required="true" type="string">build-report</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="graphResolution"/>
</ARCS>
</NODE>
<NODE name="graphResolution" type="SubmitHadoopJob">
<DESCRIPTION>Graph resolution</DESCRIPTION>
<PARAMETERS>
<PARAM managedBy="system" name="hadoopJob" required="true" type="string">executeOozieJob</PARAM>
<PARAM managedBy="system" name="cluster" required="true" type="string">IIS</PARAM>
<PARAM managedBy="system" name="envParams" required="true" type="string">
{
'graphBasePath':'cleanedFirstGraphPath',
'unresolvedPath' :'unresolvedEntityPath',
'targetPath':'resolvedGraphPath'
}
</PARAM>
<PARAM managedBy="system" name="params" required="true" type="string">
{
'oozie.wf.application.path' : '/lib/dnet/PROD/oa/graph/resolution/oozie_app',
'workingDir' : '/tmp/prod_provision/working_dir/relation_resolution',
'shouldResolveEntities' : 'false',
'sparkExecutorCores' : '4',
'sparkExecutorMemory' : '9G'
}
</PARAM>
</PARAMETERS>
<ARCS>
<ARC to="duplicateScan"/>
</ARCS>
@ -458,7 +531,7 @@
<PARAM managedBy="system" name="envParams" required="true" type="string">
{
'actionSetId' : 'dedupConfig',
'graphBasePath' : 'cleanedFirstGraphPath',
'graphBasePath' : 'resolvedGraphPath',
'dedupGraphPath': 'dedupGraphPath',
'isLookUpUrl' : 'isLookUpUrl'
}

@ -797,7 +797,7 @@
<mockito-core.version>3.3.3</mockito-core.version>
<mongodb.driver.version>3.4.2</mongodb.driver.version>
<vtd.version>[2.12,3.0)</vtd.version>
<dhp-schemas.version>[2.10.24]</dhp-schemas.version>
<dhp-schemas.version>[2.10.26-SNAPSHOT]</dhp-schemas.version>
<dnet-actionmanager-api.version>[4.0.3]</dnet-actionmanager-api.version>
<dnet-actionmanager-common.version>[6.0.5]</dnet-actionmanager-common.version>
<dnet-openaire-broker-common.version>[3.1.6]</dnet-openaire-broker-common.version>

Loading…
Cancel
Save