[SKG-IF] selection of subset of relevant results from the set provided via input
This commit is contained in:
parent
2811e2ebd7
commit
e8f19ad003
|
@ -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());
|
||||
}
|
||||
|
||||
});
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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"),
|
||||
*/
|
||||
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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
|
||||
}
|
||||
]
|
|
@ -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
|
||||
}
|
||||
]
|
|
@ -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>--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>
|
||||
|
|
|
@ -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 {
|
||||
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue