[SKG-IF] selection of subset of relevant results from the set provided via input

This commit is contained in:
Miriam Baglioni 2024-03-13 15:22:56 +01:00
parent 2811e2ebd7
commit e8f19ad003
19 changed files with 578 additions and 19 deletions

View File

@ -0,0 +1,93 @@
package eu.dnetlib.dhp.oa.graph.dump.filterentities;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.Serializable;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.oa.graph.dump.skgif.EmitFromEntities;
import eu.dnetlib.dhp.oa.graph.dump.skgif.Utils;
import eu.dnetlib.dhp.oa.graph.dump.skgif.beans.EmitPerManifestation;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.Publication;
import eu.dnetlib.dhp.schema.oaf.Result;
import scala.Tuple2;
/**
* @author miriam.baglioni
* @Date 12/03/24
*/
public class FilterEntities implements Serializable {
private static final Logger log = LoggerFactory.getLogger(EmitFromEntities.class);
public static void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils
.toString(
FilterEntities.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/graph/dump/filter_entities_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String inputPath = parser.get("sourcePath");
log.info("inputPath: {}", inputPath);
final String filterPath = parser.get("filterPath");
log.info("filterPath: {}", filterPath);
final String workingDir = parser.get("workingDir");
log.info("workingDir: {}", workingDir);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
filterEntities(spark, inputPath, filterPath, workingDir);
});
}
private static <R extends Result> void filterEntities(SparkSession spark, String inputPath, String filterPath,
String workingDir) {
ModelSupport.entityTypes.keySet().forEach(e -> {
if (ModelSupport.isResult(e)) {
Class<R> resultClazz = ModelSupport.entityTypes.get(e);
Dataset<R> result = Utils
.readPath(spark, inputPath + e.name(), resultClazz);
Dataset<Row> filterIds = spark.read().parquet(filterPath + e.name() + "_ids");
result
.joinWith(filterIds, result.col("id").equalTo(filterIds.col("id")))
.map((MapFunction<Tuple2<R, Row>, R>) t2 -> t2._1(), Encoders.bean(resultClazz))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(workingDir + e.name());
}
});
}
}

View File

@ -0,0 +1,30 @@
package eu.dnetlib.dhp.oa.graph.dump.filterentities;
import java.io.Serializable;
/**
* @author miriam.baglioni
* @Date 13/03/24
*/
public class Identifiers implements Serializable {
private String id;
private String CCL;
public String getId() {
return id;
}
public void setId(String id) {
this.id = id;
}
public String getCCL() {
return CCL;
}
public void setCCL(String CCL) {
this.CCL = CCL;
}
}

View File

@ -0,0 +1,215 @@
package eu.dnetlib.dhp.oa.graph.dump.filterentities;
import static eu.dnetlib.dhp.common.SparkSessionSupport.runWithSparkSession;
import java.io.Serializable;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.commons.io.IOUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.function.FilterFunction;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.MapFunction;
import org.apache.spark.sql.*;
import org.apache.spark.sql.Dataset;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import eu.dnetlib.dhp.application.ArgumentApplicationParser;
import eu.dnetlib.dhp.oa.graph.dump.skgif.EmitFromEntities;
import eu.dnetlib.dhp.oa.graph.dump.skgif.Utils;
import eu.dnetlib.dhp.schema.common.EntityType;
import eu.dnetlib.dhp.schema.common.ModelSupport;
import eu.dnetlib.dhp.schema.oaf.*;
import scala.Tuple2;
/**
* @author miriam.baglioni
* @Date 12/03/24
*/
public class SelectConnectedEntities implements Serializable {
private static final Logger log = LoggerFactory.getLogger(EmitFromEntities.class);
public static void main(String[] args) throws Exception {
String jsonConfiguration = IOUtils
.toString(
SelectConnectedEntities.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/graph/dump/select_connected_entities_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);
Boolean isSparkSessionManaged = Optional
.ofNullable(parser.get("isSparkSessionManaged"))
.map(Boolean::valueOf)
.orElse(Boolean.TRUE);
log.info("isSparkSessionManaged: {}", isSparkSessionManaged);
final String inputPath = parser.get("sourcePath");
log.info("inputPath: {}", inputPath);
final String filterPath = parser.get("filterPath");
log.info("filterPath: {}", filterPath);
final String workingDir = parser.get("workingDir");
log.info("workingDir: {}", workingDir);
SparkConf conf = new SparkConf();
runWithSparkSession(
conf,
isSparkSessionManaged,
spark -> {
selectConnectedEntities(spark, inputPath, filterPath, workingDir);
});
}
private static <R extends Result> void selectConnectedEntities(SparkSession spark, String inputPath,
String filterPath,
String workingDir) {
Dataset<Identifiers> resultIds = spark.emptyDataset(Encoders.bean(Identifiers.class));
for (EntityType entity : ModelSupport.entityTypes.keySet())
if (ModelSupport.isResult(entity))
resultIds = resultIds
.union(
spark
.read()
.parquet(filterPath + entity.name() + "_ids")
.as(Encoders.bean(Identifiers.class)));
Dataset<Relation> relation = Utils
.readPath(spark, inputPath + "relation", Relation.class)
.filter((FilterFunction<Relation>) r -> !r.getDataInfo().getDeletedbyinference());
Dataset<Organization> organizations = Utils
.readPath(spark, inputPath + "organization", Organization.class)
.filter((FilterFunction<Organization>) o -> !o.getDataInfo().getDeletedbyinference());
Dataset<Project> projects = Utils
.readPath(spark, inputPath + "project", Project.class)
.filter((FilterFunction<Project>) p -> !p.getDataInfo().getDeletedbyinference())
.filter(
(FilterFunction<Project>) p -> Optional.ofNullable(p.getFundingtree()).isPresent() &&
p.getFundingtree().size() > 0 &&
Utils
.getFunderName(p.getFundingtree().get(0).getValue())
.equalsIgnoreCase("European Commission"));
Dataset<Datasource> datasources = Utils
.readPath(spark, inputPath + "datasource", Datasource.class)
.filter((FilterFunction<Datasource>) d -> !d.getDataInfo().getDeletedbyinference());
// select relations having source in the set of identifiers selected for eosc
Dataset<Relation> resultSource = resultIds
.joinWith(relation, resultIds.col("id").equalTo(relation.col("source")))
.map((MapFunction<Tuple2<Identifiers, Relation>, Relation>) t2 -> t2._2(), Encoders.bean(Relation.class));
// write relations having sorce and target in the set
resultIds
.joinWith(resultSource, resultIds.col("id").equalTo(resultSource.col("target")))
.map((MapFunction<Tuple2<Identifiers, Relation>, Relation>) t2 -> t2._2(), Encoders.bean(Relation.class))
.write()
.option("compression", "gzip")
.mode(SaveMode.Overwrite)
.json(workingDir + "relation");
// write relations between results and organizations
resultSource
.joinWith(organizations, resultSource.col("target").equalTo(organizations.col("id")))
.map((MapFunction<Tuple2<Relation, Organization>, Relation>) t2 -> t2._1(), Encoders.bean(Relation.class))
.write()
.mode(SaveMode.Append)
.option("compression", "gzip")
.json(workingDir + "relation");
// write relations between results and projects
resultSource
.joinWith(projects, resultSource.col("target").equalTo(projects.col("id")))
.map((MapFunction<Tuple2<Relation, Project>, Relation>) t2 -> t2._1(), Encoders.bean(Relation.class))
.write()
.mode(SaveMode.Append)
.option("compression", "gzip")
.json(workingDir + "relation");
// write organizations linked to results in the set
resultSource
.joinWith(organizations, resultSource.col("target").equalTo(organizations.col("id")))
.map(
(MapFunction<Tuple2<Relation, Organization>, Organization>) t2 -> t2._2(),
Encoders.bean(Organization.class))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(workingDir + "organization");
// write projects linked to results in the set
resultSource
.joinWith(projects, resultSource.col("target").equalTo(projects.col("id")))
.map((MapFunction<Tuple2<Relation, Project>, Project>) t2 -> t2._2(), Encoders.bean(Project.class))
.write()
.mode(SaveMode.Append)
.option("compression", "gzip")
.json(workingDir + "project");
// read the results and select all the distinct instance.hostedbykey
Dataset<String> hostedbyIds = spark.emptyDataset(Encoders.STRING());
for (EntityType entity : ModelSupport.entityTypes.keySet())
if (ModelSupport.isResult(entity)) {
Class<R> resultClazz = ModelSupport.entityTypes.get(entity);
hostedbyIds = hostedbyIds
.union(
Utils
.readPath(spark, workingDir + entity.name(), resultClazz)
.flatMap(
(FlatMapFunction<R, String>) r -> r
.getInstance()
.stream()
.map(i -> i.getHostedby().getKey())
.collect(Collectors.toList())
.iterator(),
Encoders.STRING()));
}
// join with the datasources and write the datasource in the join
hostedbyIds
.joinWith(datasources, hostedbyIds.col("value").equalTo(datasources.col("id")))
.map((MapFunction<Tuple2<String, Datasource>, Datasource>) t2 -> t2._2(), Encoders.bean(Datasource.class))
.write()
.mode(SaveMode.Overwrite)
.option("compression", "gzip")
.json(workingDir + "datasource");
// selecting relations between organizations and projects in the selected set
Dataset<Organization> organizationSbs = Utils.readPath(spark, workingDir + "organization", Organization.class);
Dataset<Project> projectSbs = Utils.readPath(spark, workingDir + "project", Project.class);
Dataset<Relation> orgSourceRels = organizationSbs
.joinWith(relation, organizationSbs.col("id").equalTo(relation.col("source")))
.map((MapFunction<Tuple2<Organization, Relation>, Relation>) t2 -> t2._2(), Encoders.bean(Relation.class));
orgSourceRels
.joinWith(projectSbs, orgSourceRels.col("target").equalTo(projectSbs.col("id")))
.map((MapFunction<Tuple2<Relation, Project>, Relation>) t2 -> t2._1(), Encoders.bean(Relation.class))
.write()
.mode(SaveMode.Append)
.option("compression", "gzip")
.json(workingDir + "relation");
// selecting relations between datasources and organizations in the selected set
Dataset<Datasource> datasourceSbs = Utils.readPath(spark, workingDir + "datasource", Datasource.class);
Dataset<Relation> dsSourceRels = datasourceSbs
.joinWith(relation, datasourceSbs.col("id").equalTo(relation.col("source")))
.map((MapFunction<Tuple2<Datasource, Relation>, Relation>) t2 -> t2._2(), Encoders.bean(Relation.class));
dsSourceRels
.joinWith(organizationSbs, dsSourceRels.col("target").equalTo(organizations.col("id")))
.map((MapFunction<Tuple2<Relation, Organization>, Relation>) t2 -> t2._1(), Encoders.bean(Relation.class))
.write()
.mode(SaveMode.Append)
.option("compression", "gzip")
.json(workingDir + "relation");
/**
* DATASOURCE_PROVIDED_BY_ORGANIZATION(
* "isProvidedBy"),
*/
}
}

View File

@ -42,7 +42,7 @@ public class DumpDatasource implements Serializable {
.toString(
DumpDatasource.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/graph/dump/dump_datasource_parameters.json"));
"/eu/dnetlib/dhp/oa/graph/dump/skgif/dump_datasource_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);

View File

@ -45,7 +45,7 @@ public class DumpGrant implements Serializable {
.toString(
DumpGrant.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/graph/dump/dump_grant_parameters.json"));
"/eu/dnetlib/dhp/oa/graph/dump/skgif/dump_grant_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);

View File

@ -36,7 +36,7 @@ public class DumpOrganization implements Serializable {
.toString(
DumpOrganization.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/graph/dump/dump_organization_parameters.json"));
"/eu/dnetlib/dhp/oa/graph/dump/skgif/dump_organization_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);

View File

@ -43,7 +43,7 @@ public class DumpResult implements Serializable {
.toString(
DumpResult.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/graph/dump/dump_result_parameters.json"));
"/eu/dnetlib/dhp/oa/graph/dump/skgif/dump_result_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);

View File

@ -38,7 +38,7 @@ public class DumpVenue implements Serializable {
.toString(
DumpVenue.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/graph/dump/dump_datasource_parameters.json"));
"/eu/dnetlib/dhp/oa/graph/dump/skgif/dump_datasource_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);

View File

@ -44,7 +44,7 @@ public class EmitFromEntities implements Serializable {
.toString(
EmitFromEntities.class
.getResourceAsStream(
"/eu/dnetlib/dhp/oa/graph/dump/emit_biblio_parameters.json"));
"/eu/dnetlib/dhp/oa/graph/dump/skgif/emit_biblio_parameters.json"));
final ArgumentApplicationParser parser = new ArgumentApplicationParser(jsonConfiguration);
parser.parseArgument(args);

View File

@ -123,7 +123,7 @@ public class Utils implements Serializable {
mg.setLocal_identifier(p.getId());
if (Optional.ofNullable(p.getCode()).isPresent())
mg.setCode(p.getCode().getValue());
if (Optional.ofNullable(p.getFundingtree()).isPresent())
if (Optional.ofNullable(p.getFundingtree()).isPresent() && p.getFundingtree().size() > 0)
mg.setFunder(getFunderName(p.getFundingtree().get(0).getValue()));
if (Optional.ofNullable(p.getAcronym()).isPresent())
mg.setTitle(p.getAcronym().getValue());

View File

@ -0,0 +1,26 @@
[
{
"paramName":"s",
"paramLongName":"sourcePath",
"paramDescription": "the path of the sequencial file to read",
"paramRequired": true
},
{
"paramName": "fp",
"paramLongName": "filterPath",
"paramDescription": "the path used to store temporary output files",
"paramRequired": true
},
{
"paramName": "ssm",
"paramLongName": "isSparkSessionManaged",
"paramDescription": "true if the spark session is managed, false otherwise",
"paramRequired": false
},
{
"paramName": "wd",
"paramLongName": "workingDir",
"paramDescription": "the relationPath",
"paramRequired": false
}
]

View File

@ -0,0 +1,26 @@
[
{
"paramName":"s",
"paramLongName":"sourcePath",
"paramDescription": "the path of the sequencial file to read",
"paramRequired": true
},
{
"paramName": "fp",
"paramLongName": "filterPath",
"paramDescription": "the path used to store temporary output files",
"paramRequired": true
},
{
"paramName": "ssm",
"paramLongName": "isSparkSessionManaged",
"paramDescription": "true if the spark session is managed, false otherwise",
"paramRequired": false
},
{
"paramName": "wd",
"paramLongName": "workingDir",
"paramDescription": "the relationPath",
"paramRequired": false
}
]

View File

@ -62,15 +62,178 @@
</property>
</configuration>
</global>
<start to="emit_from_result"/>
<start to="select_relevant_graph_subset"/>
<decision name="filter_set">
<switch>
<case to="filter">${wf:conf('filter') eq "true"}</case>
<default to="copy_graph"/>
</switch>
</decision>
<fork name="copy_graph">
<path start="copy_relation"/>
<path start="copy_organization"/>
<path start="copy_projects"/>
<path start="copy_datasources"/>
<path start="copy_publication"/>
<path start="copy_dataset"/>
<path start="copy_software"/>
<path start="copy_otherresearchproduct"/>
</fork>
<action name="copy_relation">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<arg>${nameNode}/${sourcePath}/relation</arg>
<arg>${nameNode}/${workingDir}/graph/relation</arg>
</distcp>
<ok to="copy_wait"/>
<error to="Kill"/>
</action>
<action name="copy_organization">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<arg>${nameNode}/${sourcePath}/organization</arg>
<arg>${nameNode}/${workingDir}/graph/organization</arg>
</distcp>
<ok to="copy_wait"/>
<error to="Kill"/>
</action>
<action name="copy_projects">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<arg>${nameNode}/${sourcePath}/project</arg>
<arg>${nameNode}/${workingDir}/graph/project</arg>
</distcp>
<ok to="copy_wait"/>
<error to="Kill"/>
</action>
<action name="copy_datasources">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<arg>${nameNode}/${sourcePath}/datasource</arg>
<arg>${nameNode}/${workingDir}/graph/datasource</arg>
</distcp>
<ok to="copy_wait"/>
<error to="Kill"/>
</action>
<action name="copy_publication">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<arg>${nameNode}/${sourcePath}/publication</arg>
<arg>${nameNode}/${workingDir}/graph/publication</arg>
</distcp>
<ok to="copy_wait"/>
<error to="Kill"/>
</action>
<action name="copy_dataset">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<arg>${nameNode}/${sourcePath}/dataset</arg>
<arg>${nameNode}/${workingDir}/graph/dataset</arg>
</distcp>
<ok to="copy_wait"/>
<error to="Kill"/>
</action>
<action name="copy_software">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<arg>${nameNode}/${sourcePath}/software</arg>
<arg>${nameNode}/${workingDir}/graph/software</arg>
</distcp>
<ok to="copy_wait"/>
<error to="Kill"/>
</action>
<action name="copy_otherresearchproduct">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<arg>${nameNode}/${sourcePath}/otherresearchproduct</arg>
<arg>${nameNode}/${workingDir}/graph/otherresearchproduct</arg>
</distcp>
<ok to="copy_wait"/>
<error to="Kill"/>
</action>
<kill name="Kill">
<message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
</kill>
<join name="copy_wait" to="emit_from_result"/>
<action name="filter">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Selecting subset of results</name>
<class>eu.dnetlib.dhp.oa.graph.dump.filterentities.FilterEntities</class>
<jar>dump-${projectVersion}.jar</jar>
<spark-opts>
--executor-cores=4
--executor-memory=4G
--driver-memory=${sparkDriverMemory}
--conf spark.executor.memoryOverhead=5G
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.extraListeners=${spark2ExtraListeners}
--conf spark.sql.queryExecutionListeners=${spark2SqlQueryExecutionListeners}
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
--conf spark.sql.shuffle.partitions=15000
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--workingDir</arg><arg>${workingDir}/graph/</arg>
<arg>--filterPath</arg><arg>${filterPath}</arg>
</spark>
<ok to="select_relevant_graph_subset"/>
<error to="Kill"/>
</action>
<action name="select_relevant_graph_subset">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Selecting relevant linked relations and entities</name>
<class>eu.dnetlib.dhp.oa.graph.dump.filterentities.SelectConnectedEntities</class>
<jar>dump-${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.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<!-- <arg>&#45;&#45;workingDir</arg><arg>${workingDir}/graph/</arg>-->
<arg>--workingDir</arg><arg>${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/</arg>
<arg>--filterPath</arg><arg>${filterPath}</arg>
</spark>
<!-- <ok to="emit_from_result"/>-->
<ok to="copy_graph_subset"/>
<error to="Kill"/>
</action>
<action name="copy_graph_subset">
<distcp xmlns="uri:oozie:distcp-action:0.2">
<job-tracker>${jobTracker}</job-tracker>
<name-node>${nameNode}</name-node>
<arg>${nameNode}/user/miriam.baglioni/oa/graph/dump/temp/graph/*</arg>
<arg>${nameNode}/${workingDir}/graph/</arg>
</distcp>
<ok to="emit_from_result"/>
<error to="Kill"/>
</action>
<action name="emit_from_result">
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Extraction</name>
<name>Emit from Result</name>
<class>eu.dnetlib.dhp.oa.graph.dump.skgif.EmitFromEntities</class>
<jar>dump-${projectVersion}.jar</jar>
<spark-opts>
@ -83,7 +246,7 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--sourcePath</arg><arg>${workingDir}/graph/</arg>
<arg>--workingDir</arg><arg>${workingDir}/</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
</spark>
@ -94,7 +257,7 @@
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table project </name>
<name>Dump table results </name>
<class>eu.dnetlib.dhp.oa.graph.dump.skgif.DumpResult</class>
<jar>dump-${projectVersion}.jar</jar>
<spark-opts>
@ -107,7 +270,7 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--sourcePath</arg><arg>${workingDir}/graph/</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
<arg>--workingDir</arg><arg>${workingDir}/</arg>
</spark>
@ -118,7 +281,7 @@
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table project </name>
<name>Dump table datasource </name>
<class>eu.dnetlib.dhp.oa.graph.dump.skgif.DumpDatasource</class>
<jar>dump-${projectVersion}.jar</jar>
<spark-opts>
@ -131,7 +294,7 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--sourcePath</arg><arg>${workingDir}/graph/</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
<arg>--workingDir</arg><arg>${workingDir}/</arg>
</spark>
@ -142,7 +305,7 @@
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table project </name>
<name>Dump table venues </name>
<class>eu.dnetlib.dhp.oa.graph.dump.skgif.DumpVenue</class>
<jar>dump-${projectVersion}.jar</jar>
<spark-opts>
@ -155,7 +318,7 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--sourcePath</arg><arg>${workingDir}/graph/</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
<arg>--workingDir</arg><arg>${workingDir}/</arg>
</spark>
@ -167,7 +330,7 @@
<spark xmlns="uri:oozie:spark-action:0.2">
<master>yarn</master>
<mode>cluster</mode>
<name>Dump table project </name>
<name>Dump table organization </name>
<class>eu.dnetlib.dhp.oa.graph.dump.skgif.DumpOrganization</class>
<jar>dump-${projectVersion}.jar</jar>
<spark-opts>
@ -180,7 +343,7 @@
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--sourcePath</arg><arg>${workingDir}/graph/</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
<arg>--workingDir</arg><arg>${workingDir}/</arg>
</spark>
@ -203,8 +366,9 @@
--conf spark.yarn.historyServer.address=${spark2YarnHistoryServerAddress}
--conf spark.eventLog.dir=${nameNode}${spark2EventLogDir}
--conf spark.sql.warehouse.dir=${sparkSqlWarehouseDir}
</spark-opts>
<arg>--sourcePath</arg><arg>${sourcePath}</arg>
<arg>--sourcePath</arg><arg>${workingDir}/graph/</arg>
<arg>--outputPath</arg><arg>${outputPath}</arg>
<arg>--workingDir</arg><arg>${workingDir}/</arg>
</spark>

View File

@ -148,4 +148,9 @@ public class DumpGrantTest implements Serializable {
grant.foreach(g -> System.out.println(OBJECT_MAPPER.writeValueAsString(g)));
}
@Test
public void testDumpFunder() throws Exception {
}
}